From 1cea80d29f4f1c61ed56ad1261b74ed42611bf64 Mon Sep 17 00:00:00 2001 From: Ilya Lantukh Date: Fri, 6 Apr 2018 13:49:10 +0300 Subject: [PATCH 0001/1463] IGNITE-8018 Optimized GridCacheMapEntry initialValue() - Fixes #3686. Signed-off-by: Alexey Goncharuk --- .../processors/cache/GridCacheMapEntry.java | 158 ++++++++++++++---- .../colocated/GridDhtDetachedCacheEntry.java | 3 +- .../distributed/near/GridNearCacheEntry.java | 3 +- 3 files changed, 131 insertions(+), 33 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 74dabe9a1f7dd..a6ef0d284d6b4 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 @@ -77,6 +77,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.IgnitePredicate; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_EXPIRED; @@ -2699,40 +2700,80 @@ protected final boolean hasValueUnlocked() { ) throws IgniteCheckedException, GridCacheEntryRemovedException { ensureFreeSpace(); + boolean deferred = false; + boolean obsolete = false; + + GridCacheVersion oldVer = null; + lockEntry(); try { checkObsolete(); + boolean walEnabled = !cctx.isNear() && cctx.group().persistenceEnabled() && cctx.group().walEnabled(); + + long expTime = expireTime < 0 ? CU.toExpireTime(ttl) : expireTime; + + val = cctx.kernalContext().cacheObjects().prepareForCache(val, cctx); + + final boolean unswapped = ((flags & IS_UNSWAPPED_MASK) != 0); + boolean update; - boolean walEnabled = !cctx.isNear() && cctx.group().persistenceEnabled() && cctx.group().walEnabled(); + IgnitePredicate p = new IgnitePredicate() { + @Override public boolean apply(@Nullable CacheDataRow row) { + boolean update0; + + GridCacheVersion currentVer = row != null ? row.version() : GridCacheMapEntry.this.ver; - if (cctx.group().persistenceEnabled()) { - unswap(false); + boolean isStartVer = currentVer.nodeOrder() == cctx.localNode().order() + && currentVer.order() == startVer; - if (!isNew()) { - if (cctx.atomic()) - update = ATOMIC_VER_COMPARATOR.compare(this.ver, ver) < 0; + if (cctx.group().persistenceEnabled()) { + if (!isStartVer) { + if (cctx.atomic()) + update0 = ATOMIC_VER_COMPARATOR.compare(currentVer, ver) < 0; + else + update0 = currentVer.compareTo(ver) < 0; + } + else + update0 = true; + } else - update = this.ver.compareTo(ver) < 0; + update0 = isStartVer; + + update0 |= (!preload && deletedUnlocked()); + + return update0; } - else - update = true; - } - else - update = isNew() && !cctx.offheap().containsKey(this); + }; - update |= !preload && deletedUnlocked(); + if (unswapped) { + update = p.apply(null); - if (update) { - long expTime = expireTime < 0 ? CU.toExpireTime(ttl) : expireTime; + if (update) { + // 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()); - val = cctx.kernalContext().cacheObjects().prepareForCache(val, cctx); + if (delta < 0) { + if (onExpired(this.val, null)) { + if (cctx.deferredDelete()) { + deferred = true; + oldVer = this.ver; + } + else if (val == null) + obsolete = true; + } + } - if (val != null) storeValue(val, expTime, ver, null); + } + } + else // Optimization to access storage only once. + update = storeValue(val, expTime, ver, null, p); + if (update) { update(val, expTime, ttl, ver, true); boolean skipQryNtf = false; @@ -2797,6 +2838,20 @@ else if (deletedUnlocked()) } finally { unlockEntry(); + + // It is necessary to execute these callbacks outside of lock to avoid deadlocks. + + if (obsolete) { + onMarkedObsolete(); + + cctx.cache().removeEntry(this); + } + + if (deferred) { + assert oldVer != null; + + cctx.onDeferredDelete(this, oldVer); + } } } @@ -3516,14 +3571,39 @@ private IgniteTxLocalAdapter currentTx() { * @param oldRow Old row if available. * @throws IgniteCheckedException If update failed. */ - protected void storeValue(CacheObject val, + protected boolean storeValue(CacheObject val, long expireTime, GridCacheVersion ver, @Nullable CacheDataRow oldRow) throws IgniteCheckedException { - assert lock.isHeldByCurrentThread(); assert val != null : "null values in update for key: " + key; - cctx.offheap().invoke(cctx, key, localPartition(), new UpdateClosure(this, val, ver, expireTime)); + return storeValue(val, expireTime, ver, oldRow, null); + } + + /** + * Stores value in offheap. + * + * @param val Value. + * @param expireTime Expire time. + * @param ver New entry version. + * @param oldRow Old row if available. + * @param predicate Optional predicate. + * @throws IgniteCheckedException If update failed. + * @return {@code True} if storage was modified. + */ + protected boolean storeValue( + @Nullable CacheObject val, + long expireTime, + GridCacheVersion ver, + @Nullable CacheDataRow oldRow, + @Nullable IgnitePredicate predicate) throws IgniteCheckedException { + assert lock.isHeldByCurrentThread(); + + UpdateClosure closure = new UpdateClosure(this, val, ver, expireTime, predicate); + + cctx.offheap().invoke(cctx, key, localPartition(), closure); + + return closure.treeOp != IgniteTree.OperationType.NOOP; } /** @@ -4295,7 +4375,7 @@ private static class UpdateClosure implements IgniteCacheOffheapManager.OffheapI private final GridCacheMapEntry entry; /** */ - private final CacheObject val; + @Nullable private final CacheObject val; /** */ private final GridCacheVersion ver; @@ -4303,6 +4383,9 @@ private static class UpdateClosure implements IgniteCacheOffheapManager.OffheapI /** */ private final long expireTime; + /** */ + @Nullable private final IgnitePredicate predicate; + /** */ private CacheDataRow newRow; @@ -4317,31 +4400,44 @@ private static class UpdateClosure implements IgniteCacheOffheapManager.OffheapI * @param val New value. * @param ver New version. * @param expireTime New expire time. + * @param predicate Optional predicate. */ - UpdateClosure(GridCacheMapEntry entry, CacheObject val, GridCacheVersion ver, long expireTime) { + UpdateClosure(GridCacheMapEntry entry, @Nullable CacheObject val, GridCacheVersion ver, long expireTime, + @Nullable IgnitePredicate predicate) { this.entry = entry; this.val = val; this.ver = ver; this.expireTime = expireTime; + this.predicate = predicate; } /** {@inheritDoc} */ @Override public void call(@Nullable CacheDataRow oldRow) throws IgniteCheckedException { this.oldRow = oldRow; + if (predicate != null && !predicate.apply(oldRow)) { + treeOp = IgniteTree.OperationType.NOOP; + + return; + } + if (oldRow != null) oldRow.key(entry.key); - newRow = entry.cctx.offheap().dataStore(entry.localPartition()).createRow( - entry.cctx, - entry.key, - val, - ver, - expireTime, - oldRow); + if (val != null) { + newRow = entry.cctx.offheap().dataStore(entry.localPartition()).createRow( + entry.cctx, + entry.key, + val, + ver, + expireTime, + oldRow); - treeOp = oldRow != null && oldRow.link() == newRow.link() ? - IgniteTree.OperationType.NOOP : IgniteTree.OperationType.PUT; + treeOp = oldRow != null && oldRow.link() == newRow.link() ? + IgniteTree.OperationType.NOOP : IgniteTree.OperationType.PUT; + } + else + treeOp = oldRow != null ? IgniteTree.OperationType.REMOVE : IgniteTree.OperationType.NOOP; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java index 3536908823d2d..d02015b665489 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java @@ -65,10 +65,11 @@ public void resetFromPrimary(CacheObject val, GridCacheVersion ver) { } /** {@inheritDoc} */ - @Override protected void storeValue(CacheObject val, + @Override protected boolean storeValue(CacheObject val, long expireTime, GridCacheVersion ver, CacheDataRow oldRow) throws IgniteCheckedException { + return false; // No-op for detached entries, index is updated on primary nodes. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java index 322e63ce5f7c8..fb41f5c2e306f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java @@ -458,7 +458,8 @@ public boolean loadedValue(@Nullable IgniteInternalTx tx, } /** {@inheritDoc} */ - @Override protected void storeValue(CacheObject val, long expireTime, GridCacheVersion ver, CacheDataRow oldRow) { + @Override protected boolean storeValue(CacheObject val, long expireTime, GridCacheVersion ver, CacheDataRow oldRow) { + return false; // No-op: queries are disabled for near cache. } From 37fc72542eb6baa8be8b41aecd08a194102d13c1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=90=D0=BB=D0=B5=D0=BA=D1=81=D0=B5=D0=B9=20=D0=A1=D1=82?= =?UTF-8?q?=D0=B5=D0=BB=D1=8C=D0=BC=D0=B0=D0=BA?= Date: Fri, 6 Apr 2018 18:28:22 +0300 Subject: [PATCH 0002/1463] IGNITE-8049 Limit the number of operation cycles in B+Tree - Fixes #3769. Signed-off-by: dpavlov (cherry picked from commit e491f10) --- .../apache/ignite/IgniteSystemProperties.java | 5 +++ .../cache/persistence/tree/BPlusTree.java | 41 ++++++++++++++++++- .../database/BPlusTreeSelfTest.java | 29 +++++++++++++ 3 files changed, 74 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index f7128c0ed1fc9..152d845afbac5 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -844,6 +844,11 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_READ_LOAD_BALANCING = "IGNITE_READ_LOAD_BALANCING"; + /** + * Number of repetitions to capture a lock in the B+Tree. + */ + public static final String IGNITE_BPLUS_TREE_LOCK_RETRIES = "IGNITE_BPLUS_TREE_LOCK_RETRIES"; + /** * Enforces singleton. */ 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 a520dce829c04..4d050952b4be6 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 @@ -27,6 +27,7 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.pagemem.PageIdUtils; import org.apache.ignite.internal.pagemem.PageMemory; @@ -86,6 +87,13 @@ public abstract class BPlusTree extends DataStructure implements /** */ private static volatile boolean interrupted; + /** */ + private static final int IGNITE_BPLUS_TREE_LOCK_RETRIES_DEFAULT = 1000; + + /** */ + private static final int LOCK_RETRIES = IgniteSystemProperties.getInteger( + IgniteSystemProperties.IGNITE_BPLUS_TREE_LOCK_RETRIES, IGNITE_BPLUS_TREE_LOCK_RETRIES_DEFAULT); + /** */ private final AtomicBoolean destroyed = new AtomicBoolean(false); @@ -1123,6 +1131,8 @@ private Result findDown(final Get g, final long pageId, final long fwdId, final try { for (;;) { + g.checkLockRetry(); + // Init args. g.pageId = pageId; g.fwdId = fwdId; @@ -1657,13 +1667,18 @@ private Result invokeDown(final Invoke x, final long pageId, final long backId, long page = acquirePage(pageId); try { + Result res = RETRY; + for (;;) { + if (res == RETRY) + x.checkLockRetry(); + // Init args. x.pageId(pageId); x.fwdId(fwdId); x.backId(backId); - Result res = read(pageId, page, search, x, lvl, RETRY); + res = read(pageId, page, search, x, lvl, RETRY); switch (res) { case GO_DOWN_X: @@ -1813,6 +1828,8 @@ private Result removeDown(final Remove r, final long pageId, final long backId, try { for (;;) { + r.checkLockRetry(); + // Init args. r.pageId = pageId; r.fwdId = fwdId; @@ -2310,6 +2327,8 @@ private Result putDown(final Put p, final long pageId, final long fwdId, final i try { for (;;) { + p.checkLockRetry(); + // Init args. p.pageId = pageId; p.fwdId = fwdId; @@ -2422,6 +2441,9 @@ private abstract class Get { /** Ignore row passed, find last row */ boolean findLast; + /** Number of repetitions to capture a lock in the B+Tree (countdown). */ + int lockRetriesCnt = getLockRetries(); + /** * @param row Row. * @param findLast find last row. @@ -2534,6 +2556,16 @@ void fwdId(long fwdId) { boolean isFinished() { throw new IllegalStateException(); } + + /** + * @throws IgniteCheckedException If the operation can not be retried. + */ + final void checkLockRetry() throws IgniteCheckedException { + if (lockRetriesCnt == 0) + throw new IgniteCheckedException("Maximum of retries " + getLockRetries() + " reached."); + + lockRetriesCnt--; + } } /** @@ -4911,4 +4943,11 @@ public interface TreeRowClosure { public boolean apply(BPlusTree tree, BPlusIO io, long pageAddr, int idx) throws IgniteCheckedException; } + + /** + * @return Return number of retries. + */ + protected int getLockRetries() { + return LOCK_RETRIES; + } } 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 72d2e0ff92d6c..83d0ddd278a79 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 @@ -227,6 +227,26 @@ public void testFind() throws IgniteCheckedException { checkCursor(tree.find(10L, 70L), map.subMap(10L, true, 70L, true).values().iterator()); } + /** + * @throws IgniteCheckedException If failed. + */ + public void testRetries() throws IgniteCheckedException { + TestTree tree = createTestTree(true); + + tree.numRetries = 1; + + long size = CNT * CNT; + + try { + for (long i = 1; i <= size; i++) + tree.put(i); + + fail(); + } + catch (IgniteCheckedException ignored) { + } + } + /** * @throws IgniteCheckedException If failed. */ @@ -2349,6 +2369,9 @@ protected static class TestTree extends BPlusTree { /** */ private static ConcurrentMap> writeLocks = new ConcurrentHashMap<>(); + /** Number of retries. */ + private int numRetries = super.getLockRetries(); + /** * @param reuseList Reuse list. * @param canGetRow Can get row from inner page. @@ -2536,6 +2559,12 @@ static String printLocks() { return b.toString(); } + + /** {@inheritDoc} */ + @Override + protected int getLockRetries() { + return numRetries; + } } /** From 76e293654e34c927d6c9efc85a12e736b58a21f2 Mon Sep 17 00:00:00 2001 From: Eduard Shangareev Date: Fri, 6 Apr 2018 19:22:07 +0300 Subject: [PATCH 0003/1463] IGNITE-8114 Add fail recovery mechanism to tracking pages - Fixes #3734. Signed-off-by: dpavlov (cherry picked from commit 0829397) --- .../TrackingPageIsCorruptedException.java | 60 +++++++ .../cache/persistence/tree/io/PageMetaIO.java | 6 +- .../persistence/tree/io/TrackingPageIO.java | 156 +++++++++++++++--- .../tree/io/TrackingPageIOTest.java | 116 ++++++++++--- 4 files changed, 289 insertions(+), 49 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/TrackingPageIsCorruptedException.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/TrackingPageIsCorruptedException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/TrackingPageIsCorruptedException.java new file mode 100644 index 0000000000000..2b82bff27f95e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/TrackingPageIsCorruptedException.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.cache.persistence.snapshot; + +import org.apache.ignite.IgniteCheckedException; + +/** + * Thrown when corrupted tracking page was queried. + */ +public class TrackingPageIsCorruptedException extends IgniteCheckedException { + /** */ + private static final long serialVersionUID = 0L; + + /** Instance. */ + public static final TrackingPageIsCorruptedException INSTANCE = new TrackingPageIsCorruptedException(-1, -1); + + /** Last tag. */ + private final long lastTag; + + /** Passed tag. */ + private final long passedTag; + + /** + * @param lastTag Last tag. + * @param passedTag Passed tag. + */ + public TrackingPageIsCorruptedException(long lastTag, long passedTag) { + this.lastTag = lastTag; + this.passedTag = passedTag; + } + + /** + * @return Last tag. + */ + public long lastTag() { + return lastTag; + } + + /** + * @return Passed tag. + */ + public long passedTag() { + return passedTag; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageMetaIO.java index d2921eec3f776..d676cfd3f6805 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageMetaIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageMetaIO.java @@ -152,10 +152,10 @@ public long getLastSuccessfulFullSnapshotId(long pageAddr) { /** * @param pageAddr Page address. - * @param nextSnapshotId Next snapshot id. + * @param nextSnapshotTag Next snapshot tag. */ - public void setNextSnapshotTag(long pageAddr, long nextSnapshotId) { - PageUtils.putLong(pageAddr, NEXT_SNAPSHOT_TAG_OFF, nextSnapshotId); + public void setNextSnapshotTag(long pageAddr, long nextSnapshotTag) { + PageUtils.putLong(pageAddr, NEXT_SNAPSHOT_TAG_OFF, nextSnapshotTag); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIO.java index 1bd70f8f5ec66..94885e4889fc8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIO.java @@ -20,6 +20,7 @@ import java.nio.ByteBuffer; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.pagemem.PageIdUtils; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.TrackingPageIsCorruptedException; import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler; import org.apache.ignite.internal.util.GridStringBuilder; import org.apache.ignite.internal.util.GridUnsafe; @@ -50,6 +51,12 @@ public class TrackingPageIO extends PageIO { new TrackingPageIO(1) ); + /** Corrupt flag mask. */ + public static final long CORRUPT_FLAG_MASK = 1L << 63; + + /** Corrupt flag mask. */ + public static final long CORRUPT_FLAG_FILTER_MASK = ~CORRUPT_FLAG_MASK; + /** Last snapshot offset. */ public static final int LAST_SNAPSHOT_TAG_OFFSET = COMMON_HEADER_END; @@ -77,11 +84,11 @@ protected TrackingPageIO(int ver) { * * @param buf Buffer. * @param pageId Page id. - * @param nextSnapshotTag tag of next snapshot. + * @param nextSnapshotTag Tag of next snapshot. * @param pageSize Page size. */ - public boolean markChanged(ByteBuffer buf, long pageId, long nextSnapshotTag, long lastSuccessfulSnapshotTag, int pageSize) { - validateSnapshotId(buf, nextSnapshotTag, lastSuccessfulSnapshotTag, pageSize); + public void markChanged(ByteBuffer buf, long pageId, long nextSnapshotTag, long lastSuccessfulSnapshotTag, int pageSize) { + validateSnapshotTag(buf, nextSnapshotTag, lastSuccessfulSnapshotTag, pageSize); int cntOfPage = countOfPageToTrack(pageSize); @@ -98,7 +105,7 @@ public boolean markChanged(ByteBuffer buf, long pageId, long nextSnapshotTag, lo byte newVal = (byte) (byteToUpdate | updateTemplate); if (byteToUpdate == newVal) - return false; + return; buf.put(idx, newVal); @@ -107,8 +114,6 @@ public boolean markChanged(ByteBuffer buf, long pageId, long nextSnapshotTag, lo buf.putShort(sizeOff, newSize); assert newSize == countOfChangedPage(buf, nextSnapshotTag, pageSize); - - return true; } /** @@ -116,22 +121,30 @@ public boolean markChanged(ByteBuffer buf, long pageId, long nextSnapshotTag, lo * @param nextSnapshotTag Next snapshot id. * @param lastSuccessfulSnapshotTag Last successful snapshot id. * @param pageSize Page size. + * + * @return -1 if everything is ok, otherwise last saved tag. */ - private void validateSnapshotId(ByteBuffer buf, long nextSnapshotTag, long lastSuccessfulSnapshotTag, int pageSize) { + private long validateSnapshotTag(ByteBuffer buf, long nextSnapshotTag, long lastSuccessfulSnapshotTag, int pageSize) { assert nextSnapshotTag != lastSuccessfulSnapshotTag : "nextSnapshotTag = " + nextSnapshotTag + ", lastSuccessfulSnapshotId = " + lastSuccessfulSnapshotTag; long last = getLastSnapshotTag(buf); - assert last <= nextSnapshotTag : "last = " + last + ", nextSnapshotTag = " + nextSnapshotTag; + if(last > nextSnapshotTag) { //we have lost snapshot tag therefore should mark this tracking as corrupted + PageHandler.zeroMemory(buf, LAST_SNAPSHOT_TAG_OFFSET, buf.capacity() - LAST_SNAPSHOT_TAG_OFFSET); + + setLastSnasphotTag(buf, nextSnapshotTag | CORRUPT_FLAG_MASK); + + return last; + } if (nextSnapshotTag == last) //everything is ok - return; + return -1; int cntOfPage = countOfPageToTrack(pageSize); if (last <= lastSuccessfulSnapshotTag) { //we can drop our data - buf.putLong(LAST_SNAPSHOT_TAG_OFFSET, nextSnapshotTag); + setLastSnasphotTag(buf, nextSnapshotTag); PageHandler.zeroMemory(buf, SIZE_FIELD_OFFSET, buf.capacity() - SIZE_FIELD_OFFSET); } else { //we can't drop data, it is still necessary for incremental snapshots @@ -167,37 +180,119 @@ private void validateSnapshotId(ByteBuffer buf, long nextSnapshotTag, long lastS buf.putShort(sizeOff2, (short)newSize); } - buf.putLong(LAST_SNAPSHOT_TAG_OFFSET, nextSnapshotTag); + setLastSnasphotTag(buf, nextSnapshotTag); PageHandler.zeroMemory(buf, sizeOff, len + SIZE_FIELD_SIZE); } + + return -1; + } + + /** + * @param buf Buffer. + * @param nextSnapshotTag Next snapshot tag. + */ + private void setLastSnasphotTag(ByteBuffer buf, long nextSnapshotTag) { + if (isCorrupted(buf)) + nextSnapshotTag = nextSnapshotTag | CORRUPT_FLAG_MASK; + + buf.putLong(LAST_SNAPSHOT_TAG_OFFSET, nextSnapshotTag); } /** * @param buf Buffer. + * @param nextSnapshotTag Next snapshot tag. */ - long getLastSnapshotTag(ByteBuffer buf) { - return buf.getLong(LAST_SNAPSHOT_TAG_OFFSET); + private void setLastSnasphotTag0(ByteBuffer buf, long nextSnapshotTag) { + buf.putLong(LAST_SNAPSHOT_TAG_OFFSET, nextSnapshotTag); } /** - * @param addr Address. + * @param addr address. + * @param nextSnapshotTag Next snapshot tag. */ - long getLastSnapshotTag(long addr) { + private void setLastSnasphotTag0(long addr, long nextSnapshotTag) { + GridUnsafe.putLong(addr + LAST_SNAPSHOT_TAG_OFFSET, nextSnapshotTag); + } + + /** + * @param buf Buffer. + * @return Saved max seen snapshot tag. + */ + private long getLastSnapshotTag(ByteBuffer buf) { + return getLastSnapshotTag0(buf) & CORRUPT_FLAG_FILTER_MASK; + } + + /** + * @param buf Buffer. + * @return Saved value in {@link TrackingPageIO#LAST_SNAPSHOT_TAG_OFFSET}. + */ + private long getLastSnapshotTag0(ByteBuffer buf) { + return buf.getLong(LAST_SNAPSHOT_TAG_OFFSET) ; + } + + /** + * @param addr Address of buffer start. + * @return Saved max seen snapshot tag. + */ + private long getLastSnapshotTag(long addr) { + return getLastSnapshotTag0(addr) & CORRUPT_FLAG_FILTER_MASK; + } + + /** + * @param addr Address of buffer start. + * @return Saved value in {@link TrackingPageIO#LAST_SNAPSHOT_TAG_OFFSET}. + */ + private long getLastSnapshotTag0(long addr) { return GridUnsafe.getLong(addr + LAST_SNAPSHOT_TAG_OFFSET); } /** - * Check that pageId was marked as changed between previous snapshot finish and current snapshot start. + * @param buf Buffer. + * @return Was tracking page marked as corrupted or not. + */ + public boolean isCorrupted(ByteBuffer buf) { + return getLastSnapshotTag0(buf) < 0; //if true it means that first bit set to 1 + } + + /** + * Reset corrupted flag to false. + * + * @param buf Buffer. + */ + public void resetCorruptFlag(ByteBuffer buf) { + setLastSnasphotTag0(buf, getLastSnapshotTag(buf)); + } + + /** + * Reset corrupted flag to false. + * + * @param addr Buffer. + */ + public void resetCorruptFlag(long addr) { + setLastSnasphotTag0(addr, getLastSnapshotTag(addr)); + } + + /** * * @param buf Buffer. * @param pageId Page id. * @param curSnapshotTag Snapshot tag. * @param lastSuccessfulSnapshotTag Last successful snapshot id. * @param pageSize Page size. + * + * @return Was that pageId marked as changed between previous snapshot finish and current snapshot start or not. + * @throws TrackingPageIsCorruptedException if this tracking page was marked as corrupted. */ - public boolean wasChanged(ByteBuffer buf, long pageId, long curSnapshotTag, long lastSuccessfulSnapshotTag, int pageSize) { - validateSnapshotId(buf, curSnapshotTag + 1, lastSuccessfulSnapshotTag, pageSize); + public boolean wasChanged(ByteBuffer buf, long pageId, long curSnapshotTag, long lastSuccessfulSnapshotTag, int pageSize) + throws TrackingPageIsCorruptedException { + if (isCorrupted(buf)) + throw TrackingPageIsCorruptedException.INSTANCE; + + long lastTag = validateSnapshotTag(buf, curSnapshotTag + 1, lastSuccessfulSnapshotTag, pageSize); + + if (lastTag >= 0) + throw new TrackingPageIsCorruptedException(lastTag, curSnapshotTag); if (countOfChangedPage(buf, curSnapshotTag, pageSize) < 1) return false; @@ -223,7 +318,7 @@ public boolean wasChanged(ByteBuffer buf, long pageId, long curSnapshotTag, long * @param snapshotTag Snapshot tag. * @param pageSize Page size. * - * @return count of pages which were marked as change for given snapshotTag + * @return Count of pages which were marked as change for given snapshotTag. */ public short countOfChangedPage(ByteBuffer buf, long snapshotTag, int pageSize) { long dif = getLastSnapshotTag(buf) - snapshotTag; @@ -240,16 +335,16 @@ public short countOfChangedPage(ByteBuffer buf, long snapshotTag, int pageSize) /** * @param snapshotTag Snapshot id. * - * @return true if snapshotTag is odd, otherwise - false + * @return true if snapshotTag is odd, otherwise - false. */ - boolean useLeftHalf(long snapshotTag) { + private boolean useLeftHalf(long snapshotTag) { return (snapshotTag & 0b1) == 0; } /** * @param pageId Page id. * @param pageSize Page size. - * @return pageId of tracking page which set pageId belongs to + * @return Page id of tracking page which set pageId belongs to. */ public long trackingPageFor(long pageId, int pageSize) { assert PageIdUtils.pageIndex(pageId) > 0; @@ -267,7 +362,7 @@ public long trackingPageFor(long pageId, int pageSize) { /** * @param pageSize Page size. * - * @return how many page we can track with 1 page + * @return How many page we can track with 1 page. */ public int countOfPageToTrack(int pageSize) { return ((pageSize - SIZE_FIELD_OFFSET) / 2 - SIZE_FIELD_SIZE) << 3; @@ -279,11 +374,18 @@ public int countOfPageToTrack(int pageSize) { * @param curSnapshotTag Snapshot id. * @param lastSuccessfulSnapshotTag Last successful snapshot id. * @param pageSize Page size. - * @return set pageId if it was changed or next closest one, if there is no changed page {@code null} will be returned + * @return Passed pageId if it was changed or next closest one, if there is no changed page {@code null} will be returned. + * @throws TrackingPageIsCorruptedException if this tracking page was marked as corrupted. */ @Nullable public Long findNextChangedPage(ByteBuffer buf, long start, long curSnapshotTag, - long lastSuccessfulSnapshotTag, int pageSize) { - validateSnapshotId(buf, curSnapshotTag + 1, lastSuccessfulSnapshotTag, pageSize); + long lastSuccessfulSnapshotTag, int pageSize) throws TrackingPageIsCorruptedException { + if (isCorrupted(buf)) + throw TrackingPageIsCorruptedException.INSTANCE; + + long lastTag = validateSnapshotTag(buf, curSnapshotTag + 1, lastSuccessfulSnapshotTag, pageSize); + + if (lastTag >= 0) + throw new TrackingPageIsCorruptedException(lastTag, curSnapshotTag); int cntOfPage = countOfPageToTrack(pageSize); @@ -332,6 +434,8 @@ public int countOfPageToTrack(int pageSize) { /** * @param byteToTest Byte to test. * @param firstBitToTest First bit to test. + * + * @return Index of bit which is set to 1, if there is no such index then -1. */ private static int foundSetBit(byte byteToTest, int firstBitToTest) { assert firstBitToTest < 8; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIOTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIOTest.java index b50f0262fdab1..cacea48c5289c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIOTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIOTest.java @@ -26,14 +26,19 @@ import java.util.TreeSet; import java.util.concurrent.ThreadLocalRandom; import junit.framework.TestCase; +import org.apache.ignite.internal.pagemem.PageIdAllocator; +import org.apache.ignite.internal.pagemem.PageIdUtils; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.TrackingPageIsCorruptedException; import org.apache.ignite.internal.util.GridUnsafe; +import org.jetbrains.annotations.NotNull; /** * */ public class TrackingPageIOTest extends TestCase { /** Page size. */ - public static final int PAGE_SIZE = 2048; + public static final int PAGE_SIZE = 4096; + /** */ private final TrackingPageIO io = TrackingPageIO.VERSIONS.latest(); @@ -41,9 +46,8 @@ public class TrackingPageIOTest extends TestCase { /** * */ - public void testBasics() { - ByteBuffer buf = ByteBuffer.allocateDirect(PAGE_SIZE); - buf.order(ByteOrder.nativeOrder()); + public void testBasics() throws Exception { + ByteBuffer buf = createBuffer(); io.markChanged(buf, 2, 0, -1, PAGE_SIZE); @@ -55,13 +59,21 @@ public void testBasics() { } /** - * + * @return byte buffer with right order. */ - public void testMarkingRandomly() { + @NotNull private ByteBuffer createBuffer() { ByteBuffer buf = ByteBuffer.allocateDirect(PAGE_SIZE); + buf.order(ByteOrder.nativeOrder()); - int cntOfPageToTrack = io.countOfPageToTrack(PAGE_SIZE); + return buf; + } + + /** + * + */ + public void testMarkingRandomly() throws Exception { + ByteBuffer buf = createBuffer(); for (int i = 0; i < 1001; i++) checkMarkingRandomly(buf, i, false); @@ -70,9 +82,8 @@ public void testMarkingRandomly() { /** * */ - public void testZeroingRandomly() { - ByteBuffer buf = ByteBuffer.allocateDirect(PAGE_SIZE); - buf.order(ByteOrder.nativeOrder()); + public void testZeroingRandomly() throws Exception { + ByteBuffer buf = createBuffer(); for (int i = 0; i < 1001; i++) checkMarkingRandomly(buf, i, true); @@ -82,7 +93,7 @@ public void testZeroingRandomly() { * @param buf Buffer. * @param backupId Backup id. */ - private void checkMarkingRandomly(ByteBuffer buf, int backupId, boolean testZeroing) { + private void checkMarkingRandomly(ByteBuffer buf, int backupId, boolean testZeroing) throws Exception { ThreadLocalRandom rand = ThreadLocalRandom.current(); int track = io.countOfPageToTrack(PAGE_SIZE); @@ -132,8 +143,7 @@ private void checkMarkingRandomly(ByteBuffer buf, int backupId, boolean testZero * @throws Exception If failed. */ public void testFindNextChangedPage() throws Exception { - ByteBuffer buf = ByteBuffer.allocateDirect(PAGE_SIZE); - buf.order(ByteOrder.nativeOrder()); + ByteBuffer buf = createBuffer(); for (int i = 0; i < 101; i++) checkFindingRandomly(buf, i); @@ -143,7 +153,7 @@ public void testFindNextChangedPage() throws Exception { * @param buf Buffer. * @param backupId Backup id. */ - private void checkFindingRandomly(ByteBuffer buf, int backupId) { + private void checkFindingRandomly(ByteBuffer buf, int backupId) throws Exception { ThreadLocalRandom rand = ThreadLocalRandom.current(); int track = io.countOfPageToTrack(PAGE_SIZE); @@ -187,9 +197,8 @@ else if (setIdx.contains(pageId)) /** * */ - public void testMerging() { - ByteBuffer buf = ByteBuffer.allocateDirect(PAGE_SIZE); - buf.order(ByteOrder.nativeOrder()); + public void testMerging() throws Exception { + ByteBuffer buf = createBuffer(); ThreadLocalRandom rand = ThreadLocalRandom.current(); @@ -226,9 +235,8 @@ public void testMerging() { /** * */ - public void testMerging_MarksShouldBeDropForSuccessfulBackup() { - ByteBuffer buf = ByteBuffer.allocateDirect(PAGE_SIZE); - buf.order(ByteOrder.nativeOrder()); + public void testMerging_MarksShouldBeDropForSuccessfulBackup() throws Exception { + ByteBuffer buf = createBuffer(); ThreadLocalRandom rand = ThreadLocalRandom.current(); @@ -260,6 +268,15 @@ public void testMerging_MarksShouldBeDropForSuccessfulBackup() { assertEquals("pageId = " + i, setIdx2.contains(i), io.wasChanged(buf, i, 5, 4, PAGE_SIZE)); } + /** + * @param buf Buffer. + * @param track Track. + * @param basePageId Base page id. + * @param maxPageId Max page id. + * @param setIdx Set index. + * @param backupId Backup id. + * @param successfulBackupId Successful backup id. + */ private void generateMarking( ByteBuffer buf, int track, @@ -280,4 +297,63 @@ private void generateMarking( } } } + + /** + * We should handle case when we lost snapshot tag and now it's lower than saved. + * + * @throws Exception if failed. + */ + public void testThatWeDontFailIfSnapshotTagWasLost() throws Exception { + ByteBuffer buf = createBuffer(); + + long basePageId = PageIdUtils.pageId(0, PageIdAllocator.FLAG_IDX, 1); + + assert basePageId >= 0; + + PageIO.setPageId(GridUnsafe.bufferAddress(buf), basePageId); + + int oldTag = 10; + + io.markChanged(buf, basePageId + 1, oldTag, oldTag - 1, PAGE_SIZE); + + for (int i = 1; i < 100; i++) + io.markChanged(buf, basePageId + i, oldTag - 1, oldTag - 2, PAGE_SIZE); + + assertTrue(io.isCorrupted(buf)); + + for (int i = 1; i < 100; i++) { + try { + long id = basePageId + i; + + io.wasChanged(buf, id, oldTag - 1, oldTag - 2, PAGE_SIZE); + + fail(); + } + catch (TrackingPageIsCorruptedException e) { + //ignore + } + } + + for (int i = 1; i < 100; i++) { + long id = basePageId + i + 1000; + + io.markChanged(buf, id, oldTag, oldTag - 2, PAGE_SIZE); + } + + io.resetCorruptFlag(buf); + + assertFalse(io.isCorrupted(buf)); + + for (int i = 1; i < 100; i++) { + long id = basePageId + i + 1000; + + assertTrue(io.wasChanged(buf, id, oldTag, oldTag - 1, PAGE_SIZE)); + } + + for (int i = 1; i < 100; i++) { + long id = basePageId + i; + + assertFalse(io.wasChanged(buf, id, oldTag, oldTag - 1, PAGE_SIZE)); + } + } } \ No newline at end of file From 49f11db727febc83297c7f0f5de9e6f98f0197fa Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Mon, 9 Apr 2018 09:25:50 +0700 Subject: [PATCH 0004/1463] IGNITE-8159 control.sh: Fixed NPE on adding nodes on empty baseline and not active cluster. (cherry picked from commit 834869c) --- .../visor/baseline/VisorBaselineTask.java | 24 ++++++----- .../ignite/util/GridCommandHandlerTest.java | 43 +++++++++++++++++++ 2 files changed, 56 insertions(+), 11 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/baseline/VisorBaselineTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/baseline/VisorBaselineTask.java index 56c2dd94c53d3..721b4b3d7519d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/baseline/VisorBaselineTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/baseline/VisorBaselineTask.java @@ -71,10 +71,10 @@ private VisorBaselineTaskResult collect() { Collection baseline = cluster.currentBaselineTopology(); - Collection servers = cluster.forServers().nodes(); + Collection srvrs = cluster.forServers().nodes(); - return new VisorBaselineTaskResult(ignite.active(), cluster.topologyVersion(), - F.isEmpty(baseline) ? null : baseline, servers); + return new VisorBaselineTaskResult(ignite.cluster().active(), cluster.topologyVersion(), + F.isEmpty(baseline) ? null : baseline, srvrs); } /** @@ -93,12 +93,14 @@ private VisorBaselineTaskResult set0(Collection baselineTop) { * @return Current baseline. */ private Map currentBaseLine() { - Collection baseline = ignite.cluster().currentBaselineTopology(); - Map nodes = new HashMap<>(); - for (BaselineNode node : baseline) - nodes.put(node.consistentId().toString(), node); + Collection baseline = ignite.cluster().currentBaselineTopology(); + + if (!F.isEmpty(baseline)) { + for (BaselineNode node : baseline) + nodes.put(node.consistentId().toString(), node); + } return nodes; } @@ -122,12 +124,12 @@ private Map currentServers() { * @return New baseline. */ private VisorBaselineTaskResult set(List consistentIds) { - Map servers = currentServers(); + Map srvrs = currentServers(); Collection baselineTop = new ArrayList<>(); for (String consistentId : consistentIds) { - BaselineNode node = servers.get(consistentId); + BaselineNode node = srvrs.get(consistentId); if (node == null) throw new IllegalStateException("Node not found for consistent ID: " + consistentId); @@ -146,10 +148,10 @@ private VisorBaselineTaskResult set(List consistentIds) { */ private VisorBaselineTaskResult add(List consistentIds) { Map baseline = currentBaseLine(); - Map servers = currentServers(); + Map srvrs = currentServers(); for (String consistentId : consistentIds) { - BaselineNode node = servers.get(consistentId); + BaselineNode node = srvrs.get(consistentId); if (node == null) throw new IllegalStateException("Node not found for consistent ID: " + consistentId); diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java index d5051d7ba0bc7..eb18e29680082 100644 --- a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java @@ -17,7 +17,9 @@ package org.apache.ignite.util; +import java.io.ByteArrayOutputStream; import java.io.File; +import java.io.PrintStream; import java.util.ArrayList; import java.util.Arrays; import org.apache.ignite.Ignite; @@ -61,6 +63,11 @@ protected File folder(String folder) throws IgniteCheckedException { cleanPersistenceDir(); } + /** {@inheritDoc} */ + @Override public String getTestIgniteInstanceName() { + return "bltTest"; + } + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); @@ -76,6 +83,8 @@ protected File folder(String folder) throws IgniteCheckedException { dsCfg.setWalMode(WALMode.LOG_ONLY); dsCfg.getDefaultDataRegionConfiguration().setPersistenceEnabled(true); + cfg.setConsistentId(igniteInstanceName); + return cfg; } @@ -205,6 +214,40 @@ public void testBaselineAdd() throws Exception { assertEquals(2, ignite.cluster().currentBaselineTopology().size()); } + /** + * Test baseline add items works via control.sh + * + * @throws Exception If failed. + */ + public void testBaselineAddOnNotActiveCluster() throws Exception { + try { + Ignite ignite = startGrid(1); + + assertFalse(ignite.cluster().active()); + + String consistentIDs = getTestIgniteInstanceName(1); + + ByteArrayOutputStream out = new ByteArrayOutputStream(4096); + System.setOut(new PrintStream(out)); + + assertEquals(EXIT_CODE_UNEXPECTED_ERROR, execute("--baseline", "add", consistentIDs)); + + assertTrue(out.toString().contains("Changing BaselineTopology on inactive cluster is not allowed.")); + + consistentIDs = + getTestIgniteInstanceName(1) + ", " + + getTestIgniteInstanceName(2) + "," + + getTestIgniteInstanceName(3); + + assertEquals(EXIT_CODE_UNEXPECTED_ERROR, execute("--baseline", "add", consistentIDs)); + + assertTrue(out.toString().contains("Node not found for consistent ID: bltTest2")); + } + finally { + System.setOut(System.out); + } + } + /** * Test baseline remove works via control.sh * From 9ad7be2f51b6dcdcdf43fedb298cd4e240f0adab Mon Sep 17 00:00:00 2001 From: Ilya Borisov Date: Mon, 9 Apr 2018 20:59:32 +0700 Subject: [PATCH 0005/1463] IGNITE-8155 Web Console: Fixed number pattern warning in browser console. (cherry picked from commit 5d8f570) --- .../app/modules/states/configuration/clusters/communication.pug | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.pug b/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.pug index bd8971ad9f193..8b43521ff5a0a 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.pug +++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.pug @@ -94,6 +94,7 @@ panel-collapsible(ng-form=form on-open=`ui.loadPanel('${form}')`) tip: 'Message queue limit for incoming and outgoing messages' }) .pc-form-grid-col-30 + //- allowInvalid: true prevents from infinite digest loop when old value was 0 and becomes less than allowed minimum +sane-ignite-form-field-number({ label: 'Unacknowledged messages:', model: `${communication}.unacknowledgedMessagesBufferSize`, @@ -111,7 +112,6 @@ panel-collapsible(ng-form=form on-open=`ui.loadPanel('${form}')`)
  • At least message queue limit * {{ ::$ctrl.Clusters.unacknowledgedMessagesBufferSize.validRatio }}
  • ` })( - //- allowInvalid: true prevents from infinite digest loop when old value was 0 and becomes less than allowed minimum ng-model-options=`{ allowInvalid: true }` From 4aa56751906e5db7aad025a7193933fa929aae26 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Mon, 9 Apr 2018 22:13:21 +0700 Subject: [PATCH 0006/1463] IGNITE-7940 Visor CMD: Added "cache -slp" and "cache -rlp" commands to show and reset lost partitions for specified cache. (cherry picked from commit abfa0f5) --- .../cache/VisorCacheLostPartitionsTask.java | 85 +++++++++ .../VisorCacheLostPartitionsTaskArg.java | 73 ++++++++ .../VisorCacheLostPartitionsTaskResult.java | 74 ++++++++ .../VisorCacheResetLostPartitionsTask.java | 65 +++++++ .../VisorCacheResetLostPartitionsTaskArg.java | 73 ++++++++ .../commands/cache/VisorCacheCommand.scala | 34 +++- .../VisorCacheLostPartitionsCommand.scala | 170 ++++++++++++++++++ ...VisorCacheResetLostPartitionsCommand.scala | 132 ++++++++++++++ 8 files changed, 702 insertions(+), 4 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLostPartitionsTask.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLostPartitionsTaskArg.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLostPartitionsTaskResult.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetLostPartitionsTask.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetLostPartitionsTaskArg.java create mode 100644 modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheLostPartitionsCommand.scala create mode 100644 modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheResetLostPartitionsCommand.scala diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLostPartitionsTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLostPartitionsTask.java new file mode 100644 index 0000000000000..24b406915a02b --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLostPartitionsTask.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.internal.visor.cache; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.ignite.internal.processors.cache.IgniteInternalCache; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology; +import org.apache.ignite.internal.processors.task.GridInternal; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.visor.VisorJob; +import org.apache.ignite.internal.visor.VisorOneNodeTask; + +/** + * Collect list of lost partitions. + */ +@GridInternal +public class VisorCacheLostPartitionsTask + extends VisorOneNodeTask { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override protected VisorCacheLostPartitionsJob job(VisorCacheLostPartitionsTaskArg arg) { + return new VisorCacheLostPartitionsJob(arg, debug); + } + + /** + * Job that collect list of lost partitions. + */ + private static class VisorCacheLostPartitionsJob + extends VisorJob { + /** */ + private static final long serialVersionUID = 0L; + + /** + * @param arg Object with list of cache names to collect lost partitions. + * @param debug Debug flag. + */ + private VisorCacheLostPartitionsJob(VisorCacheLostPartitionsTaskArg arg, boolean debug) { + super(arg, debug); + } + + /** {@inheritDoc} */ + @Override protected VisorCacheLostPartitionsTaskResult run(VisorCacheLostPartitionsTaskArg arg) { + Map> res = new HashMap<>(); + + for (String cacheName: arg.getCacheNames()) { + IgniteInternalCache cache = ignite.cachex(cacheName); + + if (cache != null) { + GridDhtPartitionTopology topology = cache.context().topology(); + List lostPartitions = new ArrayList<>(topology.lostPartitions()); + + if (!lostPartitions.isEmpty()) + res.put(cacheName, lostPartitions); + } + } + + return new VisorCacheLostPartitionsTaskResult(res); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorCacheLostPartitionsJob.class, this); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLostPartitionsTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLostPartitionsTaskArg.java new file mode 100644 index 0000000000000..d6404bfdeb5ec --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLostPartitionsTaskArg.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.visor.cache; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.List; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * Argument for {@link VisorCacheLostPartitionsTask}. + */ +public class VisorCacheLostPartitionsTaskArg extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** List of cache names. */ + private List cacheNames; + + /** + * Default constructor. + */ + public VisorCacheLostPartitionsTaskArg() { + // No-op. + } + + /** + * @param cacheNames List of cache names. + */ + public VisorCacheLostPartitionsTaskArg(List cacheNames) { + this.cacheNames = cacheNames; + } + + /** + * @return List of cache names. + */ + public List getCacheNames() { + return cacheNames; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + U.writeCollection(out, cacheNames); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + cacheNames = U.readList(in); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorCacheLostPartitionsTaskArg.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLostPartitionsTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLostPartitionsTaskResult.java new file mode 100644 index 0000000000000..b9a0e6b13ad81 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLostPartitionsTaskResult.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.visor.cache; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.List; +import java.util.Map; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * Result for {@link VisorCacheLostPartitionsTask}. + */ +public class VisorCacheLostPartitionsTaskResult extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** List of lost partitions by caches. */ + private Map> lostPartitions; + + /** + * Default constructor. + */ + public VisorCacheLostPartitionsTaskResult() { + // No-op. + } + + /** + * @param lostPartitions List of lost partitions by caches. + */ + public VisorCacheLostPartitionsTaskResult(Map> lostPartitions) { + this.lostPartitions = lostPartitions; + } + + /** + * @return List of lost partitions by caches. + */ + public Map> getLostPartitions() { + return lostPartitions; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + U.writeMap(out, lostPartitions); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + lostPartitions = U.readMap(in); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorCacheLostPartitionsTaskResult.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetLostPartitionsTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetLostPartitionsTask.java new file mode 100644 index 0000000000000..eb48cd2efd458 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetLostPartitionsTask.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.visor.cache; + +import org.apache.ignite.internal.processors.task.GridInternal; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.visor.VisorJob; +import org.apache.ignite.internal.visor.VisorOneNodeTask; + +/** + * Reset lost partitions for caches. + */ +@GridInternal +public class VisorCacheResetLostPartitionsTask extends VisorOneNodeTask { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override protected VisorCacheResetLostPartitionsJob job(VisorCacheResetLostPartitionsTaskArg arg) { + return new VisorCacheResetLostPartitionsJob(arg, debug); + } + + /** + * Job that reset lost partitions for caches. + */ + private static class VisorCacheResetLostPartitionsJob extends VisorJob { + /** */ + private static final long serialVersionUID = 0L; + + /** + * @param arg Object with list cache names to reset lost partitons. + * @param debug Debug flag. + */ + private VisorCacheResetLostPartitionsJob(VisorCacheResetLostPartitionsTaskArg arg, boolean debug) { + super(arg, debug); + } + + /** {@inheritDoc} */ + @Override protected Void run(VisorCacheResetLostPartitionsTaskArg arg) { + ignite.resetLostPartitions(arg.getCacheNames()); + + return null; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorCacheResetLostPartitionsJob.class, this); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetLostPartitionsTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetLostPartitionsTaskArg.java new file mode 100644 index 0000000000000..2f365c82d6fba --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheResetLostPartitionsTaskArg.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.visor.cache; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.List; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * Argument for {@link VisorCacheResetLostPartitionsTask}. + */ +public class VisorCacheResetLostPartitionsTaskArg extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** List of cache names. */ + private List cacheNames; + + /** + * Default constructor. + */ + public VisorCacheResetLostPartitionsTaskArg() { + // No-op. + } + + /** + * @param cacheNames List of cache names. + */ + public VisorCacheResetLostPartitionsTaskArg(List cacheNames) { + this.cacheNames = cacheNames; + } + + /** + * @return List of cache names. + */ + public List getCacheNames() { + return cacheNames; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + U.writeCollection(out, cacheNames); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + cacheNames = U.readList(in); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorCacheResetLostPartitionsTaskArg.class, this); + } +} diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala index d55fed1e72402..e3e200148a252 100755 --- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala +++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala @@ -62,6 +62,10 @@ import scala.language.{implicitConversions, reflectiveCalls} * +-------------------------------------------------------------------------------------------+ * | cache -rebalance | Re-balance partitions for cache with specified name. | * +-------------------------------------------------------------------------------------------+ + * | cache -slp | Show list of lost partitions for specified cache. | + * +-------------------------------------------------------------------------------------------+ + * | cache -rlp | Reset lost partitions for specified cache. | + * +-------------------------------------------------------------------------------------------+ * * }}} * @@ -75,6 +79,8 @@ import scala.language.{implicitConversions, reflectiveCalls} * cache -stop -c= * cache -reset -c= * cache -rebalance -c= + * cache -slp -c= + * cache -rlp -c= * }}} * * ====Arguments==== @@ -123,6 +129,10 @@ import scala.language.{implicitConversions, reflectiveCalls} * Reset metrics for cache with specified name. * -rebalance * Re-balance partitions for cache with specified name. + * -slp + * Show list of lost partitions for specified cache. + * -rlp + * Reset lost partitions for specified cache. * -p= * Number of object to fetch from cache at once. * Valid range from 1 to 100. @@ -163,6 +173,10 @@ import scala.language.{implicitConversions, reflectiveCalls} * Reset metrics for cache with name 'cache'. * cache -rebalance -c=cache * Re-balance partitions for cache with name 'cache'. + * cache -slp -c=cache + * Show list of lost partitions for cache with name 'cache'. + * cache -rlp -c=cache + * Reset lost partitions for cache with name 'cache'. * * }}} */ @@ -265,7 +279,7 @@ class VisorCacheCommand extends VisorConsoleCommand { // Get cache stats data from all nodes. val aggrData = cacheData(node, cacheName, showSystem) - if (hasArgFlagIn("clear", "scan", "stop", "reset", "rebalance")) { + if (hasArgFlagIn("clear", "scan", "stop", "reset", "rebalance", "slp", "rlp")) { if (cacheName.isEmpty) askForCache("Select cache from:", node, showSystem && !hasArgFlagIn("clear", "stop", "reset", "rebalance"), aggrData) match { @@ -291,6 +305,10 @@ class VisorCacheCommand extends VisorConsoleCommand { VisorCacheResetCommand().reset(argLst, node) else if (hasArgFlag("rebalance", argLst)) VisorCacheRebalanceCommand().rebalance(argLst, node) + else if (hasArgFlag("slp", argLst)) + VisorCacheLostPartitionsCommand().showLostPartitions(argLst, node) + else if (hasArgFlag("rlp", argLst)) + VisorCacheResetLostPartitionsCommand().resetLostPartitions(argLst, node) } else { if (hasArgFlag("scan", argLst)) @@ -716,7 +734,9 @@ object VisorCacheCommand { " ", "Clears cache.", " ", - "Prints list of all entries from cache." + "Prints list of all entries from cache.", + " ", + "Prints or clear list lost partitions from cache." ), spec = Seq( "cache", @@ -726,7 +746,9 @@ object VisorCacheCommand { "cache -scan -c= {-near} {-id=|id8=} {-p=}", "cache -stop -c=", "cache -reset -c=", - "cache -rebalance -c=" + "cache -rebalance -c=", + "cache -slp -c=", + "cache -rlp -c=" ), args = Seq( "-id8=" -> Seq( @@ -752,6 +774,8 @@ object VisorCacheCommand { "-near" -> "Prints list of all entries from near cache of cache.", "-stop" -> "Stop cache with specified name.", "-reset" -> "Reset metrics of cache with specified name.", + "-slp" -> "Show list of lost partitions for specified cache.", + "-rlp" -> "Reset lost partitions for specified cache.", "-rebalance" -> "Re-balance partitions for cache with specified name.", "-s=hi|mi|rd|wr|cn" -> Seq( "Defines sorting type. Sorted by:", @@ -812,7 +836,9 @@ object VisorCacheCommand { "Prints list entries from near cache of cache with name 'cache' and node '12345678' ID8.", "cache -stop -c=@c0" -> "Stop cache with name taken from 'c0' memory variable.", "cache -reset -c=@c0" -> "Reset metrics for cache with name taken from 'c0' memory variable.", - "cache -rebalance -c=cache" -> "Re-balance partitions for cache with name 'cache'." + "cache -rebalance -c=cache" -> "Re-balance partitions for cache with name 'cache'.", + "cache -slp -c=@c0" -> "Show list of lost partitions for cache with name taken from 'c0' memory variable.", + "cache -rlp -c=@c0" -> "Reset lost partitions for cache with name taken from 'c0' memory variable." ), emptyArgs = cmd.cache, withArgs = cmd.cache diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheLostPartitionsCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheLostPartitionsCommand.scala new file mode 100644 index 0000000000000..d6830310d11d1 --- /dev/null +++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheLostPartitionsCommand.scala @@ -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.visor.commands.cache + +import java.util.Collections + +import org.apache.ignite.cluster.{ClusterGroupEmptyException, ClusterNode} +import org.apache.ignite.internal.visor.cache.{VisorCacheLostPartitionsTask, VisorCacheLostPartitionsTaskArg} +import org.apache.ignite.visor.commands.common.VisorTextTable +import org.apache.ignite.visor.visor._ +import org.apache.ignite.internal.visor.util.VisorTaskUtils._ + +import scala.collection.JavaConversions._ +import scala.collection.mutable + +/** + * ==Overview== + * Visor 'lost partitions' command implementation. + * + * ====Specification==== + * {{{ + * cache -slp -c= + * }}} + * + * ====Arguments==== + * {{{ + * + * Name of the cache. + * }}} + * + * ====Examples==== + * {{{ + * cache -slp -c=cache + * Show list of lost partitions for cache with name 'cache'. + * }}} + */ +class VisorCacheLostPartitionsCommand { + /** + * Prints error message and advise. + * + * @param errMsgs Error messages. + */ + private def scold(errMsgs: Any*) { + assert(errMsgs != null) + + warn(errMsgs: _*) + warn("Type 'help cache' to see how to use this command.") + } + + private def error(e: Throwable) { + var cause: Throwable = e + + while (cause.getCause != null) + cause = cause.getCause + + scold(cause.getMessage) + } + + /** + * ===Command=== + * Show list of lost partitions in cache with specified name. + * + * ===Examples=== + * cache -slp -c=cache + * Show list of lost partitions from cache with name 'cache'. + * + * @param argLst Command arguments. + */ + def showLostPartitions(argLst: ArgList, node: Option[ClusterNode]) { + val cacheArg = argValue("c", argLst) + val cacheName = cacheArg match { + case None => null // default cache. + + case Some(s) if s.startsWith("@") => + warn("Can't find cache variable with specified name: " + s, + "Type 'cache' to see available cache variables." + ) + + return + + case Some(name) => name + } + + val lostPartitions = + try + executeRandom(groupForDataNode(node, cacheName), classOf[VisorCacheLostPartitionsTask], + new VisorCacheLostPartitionsTaskArg(Collections.singletonList(cacheName))) + catch { + case _: ClusterGroupEmptyException => + scold(messageNodeNotFound(node, cacheName)) + + return + case e: Throwable => + error(e) + + return + } + + if (lostPartitions.getLostPartitions.isEmpty) { + println(s"""Lost partitions for cache: "${escapeName(cacheName)}" is not found""") + + return + } + + lostPartitions.getLostPartitions.foreach(cacheLostPartitions => { + val t = VisorTextTable() + + t #= ("Interval", "Partitions") + + val partitions = cacheLostPartitions._2.toIndexedSeq + val partitionCnt = partitions.size + + val indexes = mutable.ArrayBuffer.empty[String] + val partitionRows = mutable.ArrayBuffer.empty[String] + var startIdx = 0 + var idx = 0 + val b = new StringBuilder + + partitions.foreach((part) => { + if (idx % 10 == 0) + startIdx = part + + b.append(part) + idx += 1 + + if (idx != partitionCnt) + b.append(", ") + + + if (idx % 10 == 0 || idx == partitionCnt) { + indexes += startIdx + "-" + part + partitionRows += b.toString().trim + b.clear() + } + }) + + t += (indexes, partitionRows) + println(s"Lost partitions for cache: ${escapeName(cacheLostPartitions._1)} ($partitionCnt)") + t.render() + }) + } +} + +/** + * Companion object that does initialization of the command. + */ +object VisorCacheLostPartitionsCommand { + /** Singleton command. */ + private val cmd = new VisorCacheLostPartitionsCommand + + /** + * Singleton. + */ + def apply(): VisorCacheLostPartitionsCommand = cmd +} diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheResetLostPartitionsCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheResetLostPartitionsCommand.scala new file mode 100644 index 0000000000000..a72a80f46ff4d --- /dev/null +++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheResetLostPartitionsCommand.scala @@ -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.visor.commands.cache + +import java.util.Collections + +import org.apache.ignite.cluster.{ClusterGroupEmptyException, ClusterNode} +import org.apache.ignite.internal.visor.cache.{VisorCacheResetLostPartitionsTask, VisorCacheResetLostPartitionsTaskArg} +import org.apache.ignite.internal.visor.util.VisorTaskUtils._ +import org.apache.ignite.visor.visor._ + +import scala.language.reflectiveCalls + +/** + * ==Overview== + * Visor 'lost partition reset' command implementation. + * + * ====Specification==== + * {{{ + * cache -rlp -c= + * }}} + * + * ====Arguments==== + * {{{ + * + * Name of the cache. + * }}} + * + * ====Examples==== + * {{{ + * cache -rlp -c=@c0 + * Reset lost partition for cache with name taken from 'c0' memory variable. + * }}} + */ +class VisorCacheResetLostPartitionsCommand { + /** + * Prints error message and advise. + * + * @param errMsgs Error messages. + */ + private def scold(errMsgs: Any*) { + assert(errMsgs != null) + + warn(errMsgs: _*) + warn("Type 'help cache' to see how to use this command.") + } + + private def error(e: Exception) { + var cause: Throwable = e + + while (cause.getCause != null) + cause = cause.getCause + + scold(cause.getMessage) + } + + /** + * ===Command=== + * Reset lost partitions for cache with specified name. + * + * ===Examples=== + * cache -c=cache -rlp + * Reset lost partitions for cache with name 'cache'. + * + * @param argLst Command arguments. + */ + def resetLostPartitions(argLst: ArgList, node: Option[ClusterNode]) { + val cacheArg = argValue("c", argLst) + + val cacheName = cacheArg match { + case None => null // default cache. + + case Some(s) if s.startsWith("@") => + warn("Can't find cache variable with specified name: " + s, + "Type 'cache' to see available cache variables." + ) + + return + + case Some(name) => name + } + + val grp = try { + groupForDataNode(node, cacheName) + } + catch { + case _: ClusterGroupEmptyException => + scold(messageNodeNotFound(node, cacheName)) + + return + } + + try { + executeRandom(grp, classOf[VisorCacheResetLostPartitionsTask], + new VisorCacheResetLostPartitionsTaskArg(Collections.singletonList(cacheName))) + + println("Visor successfully reset lost partitions for cache: " + escapeName(cacheName)) + } + catch { + case _: ClusterGroupEmptyException => scold(messageNodeNotFound(node, cacheName)) + case e: Exception => error(e) + } + } +} + +/** + * Companion object that does initialization of the command. + */ +object VisorCacheResetLostPartitionsCommand { + /** Singleton command. */ + private val cmd = new VisorCacheResetLostPartitionsCommand + + /** + * Singleton. + */ + def apply() = cmd +} From cc04c5c70af1bdbba834f73330e73277b60e23fc Mon Sep 17 00:00:00 2001 From: Eduard Shangareev Date: Mon, 9 Apr 2018 19:15:50 +0300 Subject: [PATCH 0007/1463] IGNITE-8114 Additional fix for Add fail recovery mechanism to tracking pages (cherry picked from commit 961fc35) --- .../cache/persistence/tree/io/TrackingPageIO.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIO.java index 94885e4889fc8..80dbf253e84b3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIO.java @@ -86,9 +86,11 @@ protected TrackingPageIO(int ver) { * @param pageId Page id. * @param nextSnapshotTag Tag of next snapshot. * @param pageSize Page size. + * + * @return -1 if everything is ok, otherwise last saved tag. */ - public void markChanged(ByteBuffer buf, long pageId, long nextSnapshotTag, long lastSuccessfulSnapshotTag, int pageSize) { - validateSnapshotTag(buf, nextSnapshotTag, lastSuccessfulSnapshotTag, pageSize); + public long markChanged(ByteBuffer buf, long pageId, long nextSnapshotTag, long lastSuccessfulSnapshotTag, int pageSize) { + long tag = validateSnapshotTag(buf, nextSnapshotTag, lastSuccessfulSnapshotTag, pageSize); int cntOfPage = countOfPageToTrack(pageSize); @@ -105,7 +107,7 @@ public void markChanged(ByteBuffer buf, long pageId, long nextSnapshotTag, long byte newVal = (byte) (byteToUpdate | updateTemplate); if (byteToUpdate == newVal) - return; + return tag; buf.put(idx, newVal); @@ -114,6 +116,8 @@ public void markChanged(ByteBuffer buf, long pageId, long nextSnapshotTag, long buf.putShort(sizeOff, newSize); assert newSize == countOfChangedPage(buf, nextSnapshotTag, pageSize); + + return tag; } /** From c70d85aa36c702ea0f29bd8668e9bf0790f9ba11 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Tue, 10 Apr 2018 15:42:24 +0700 Subject: [PATCH 0008/1463] IGNITE-8126 Web Console: Fixed code generation for cache load. (cherry picked from commit a0a187b) --- .../configuration/summary/summary.worker.js | 2 +- .../web-console/frontend/package-lock.json | 929 ++++++++---------- 2 files changed, 409 insertions(+), 522 deletions(-) diff --git a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js index c80d698dd588c..b3b0bce4bf3f8 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js +++ b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js @@ -109,7 +109,7 @@ onmessage = function(e) { } // Generate loader for caches with configured store. - const cachesToLoad = filter(cluster.caches, (cache) => nonNil(cache.cacheStoreFactory)); + const cachesToLoad = filter(cluster.caches, (cache) => nonNil(_.get(cache, 'cacheStoreFactory.kind'))); if (nonEmpty(cachesToLoad)) zip.file(`${srcPath}/load/LoadCaches.java`, java.loadCaches(cachesToLoad, 'load', 'LoadCaches', `"${clientXml}"`)); diff --git a/modules/web-console/frontend/package-lock.json b/modules/web-console/frontend/package-lock.json index 071dec058b2c3..1834621f6b316 100644 --- a/modules/web-console/frontend/package-lock.json +++ b/modules/web-console/frontend/package-lock.json @@ -95,9 +95,9 @@ "dev": true }, "@types/node": { - "version": "9.6.2", - "resolved": "https://registry.npmjs.org/@types/node/-/node-9.6.2.tgz", - "integrity": "sha512-UWkRY9X7RQHp5OhhRIIka58/gVVycL1zHZu0OTsT5LI86ABaMOSbUjAl+b0FeDhQcxclrkyft3kW5QWdMRs8wQ==", + "version": "7.0.60", + "resolved": "https://registry.npmjs.org/@types/node/-/node-7.0.60.tgz", + "integrity": "sha512-ZfCUDgCOPBDn0aAsyBOcNh1nLksuGp3LAL+8GULccZN2IkMBG2KfiwFIRrIuQkLKg1W1dIB9kQZ9MIF3IgAqlw==", "dev": true }, "@types/sinon": { @@ -113,9 +113,9 @@ "dev": true }, "@types/uglify-js": { - "version": "3.0.1", - "resolved": "https://registry.npmjs.org/@types/uglify-js/-/uglify-js-3.0.1.tgz", - "integrity": "sha512-eWwNO88HxJonNKyxZ3dR62yle3N+aBPIsjTrPtoMcldLXGeIKAIlewNIWT4cxjZ4gy3YdBobkaKSv74HJXSzRg==", + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/@types/uglify-js/-/uglify-js-3.0.2.tgz", + "integrity": "sha512-o8hU2+4xsyGC27Vujoklvxl88Ew5zmJuTBYMX1Uro2rYUt4HEFJKL6fuq8aGykvS+ssIsIzerWWP2DRxonownQ==", "dev": true, "requires": { "source-map": "0.6.1" @@ -145,9 +145,9 @@ "integrity": "sha512-NoGVTCumOsyFfuy3934f3ktiJi+wcXHJFxT47tby3iCpuo6M/WjFA9VqT5bYO+FE46i3R0N00RpJX75HxHKDaQ==", "dev": true, "requires": { - "@types/node": "9.6.2", + "@types/node": "7.0.60", "@types/tapable": "1.0.1", - "@types/uglify-js": "3.0.1", + "@types/uglify-js": "3.0.2", "source-map": "0.6.1" }, "dependencies": { @@ -205,6 +205,16 @@ "preact": "7.2.1" } }, + "JSONStream": { + "version": "1.3.2", + "resolved": "https://registry.npmjs.org/JSONStream/-/JSONStream-1.3.2.tgz", + "integrity": "sha1-wQI3G27Dp887hHygDCC7D85Mbeo=", + "dev": true, + "requires": { + "jsonparse": "1.3.1", + "through": "2.3.8" + } + }, "abbrev": { "version": "1.1.1", "resolved": "https://registry.npmjs.org/abbrev/-/abbrev-1.1.1.tgz", @@ -455,9 +465,9 @@ "integrity": "sha1-ClsQZgGLQOcAuMbuNLDJf8MhlSs=" }, "angular-ui-grid": { - "version": "4.4.5", - "resolved": "https://registry.npmjs.org/angular-ui-grid/-/angular-ui-grid-4.4.5.tgz", - "integrity": "sha512-ZwnDi4+6oh1A089nKJao8Q9XQseDAsHFYYwT/felyUeCpW+5izq8pJqQszM0HLqNeQj08+IJVQFbWto+UlH07A==", + "version": "4.4.6", + "resolved": "https://registry.npmjs.org/angular-ui-grid/-/angular-ui-grid-4.4.6.tgz", + "integrity": "sha512-0d14HDY4XeqFHI508thxeufiR0AlFoZQ8ihk0x8TRCQc+b9CCk1/F63W2zihirxF0cdOAqBCY2pVSM7vfZvXBQ==", "requires": { "angular": "1.6.6" } @@ -517,17 +527,15 @@ "integrity": "sha1-qJS3XUvE9s1nnvMkSp/Y9Gri1Cg=" }, "braces": { - "version": "2.3.1", - "resolved": "https://registry.npmjs.org/braces/-/braces-2.3.1.tgz", - "integrity": "sha512-SO5lYHA3vO6gz66erVvedSCkp7AKWdv6VcQ2N4ysXfPxdAlxAMMAdwegGGcv1Bqwm7naF1hNdk5d6AAIEHV2nQ==", + "version": "2.3.2", + "resolved": "https://registry.npmjs.org/braces/-/braces-2.3.2.tgz", + "integrity": "sha512-aNdbnj9P8PjdXU4ybaWLK2IF3jc/EoDYbC7AazW6to3TRsfXxscC9UXOB5iDiEQrkyIbWp2SLQda4+QAa7nc3w==", "requires": { "arr-flatten": "1.1.0", "array-unique": "0.3.2", - "define-property": "1.0.0", "extend-shallow": "2.0.1", "fill-range": "4.0.0", "isobject": "3.0.1", - "kind-of": "6.0.2", "repeat-element": "1.1.2", "snapdragon": "0.8.2", "snapdragon-node": "2.1.1", @@ -535,14 +543,6 @@ "to-regex": "3.0.2" }, "dependencies": { - "define-property": { - "version": "1.0.0", - "resolved": "https://registry.npmjs.org/define-property/-/define-property-1.0.0.tgz", - "integrity": "sha1-dp66rz9KY6rTr56NMEybvnm/sOY=", - "requires": { - "is-descriptor": "1.0.2" - } - }, "extend-shallow": { "version": "2.0.1", "resolved": "https://registry.npmjs.org/extend-shallow/-/extend-shallow-2.0.1.tgz", @@ -592,6 +592,42 @@ "is-extendable": "0.1.1" } }, + "is-accessor-descriptor": { + "version": "0.1.6", + "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-0.1.6.tgz", + "integrity": "sha1-qeEss66Nh2cn7u84Q/igiXtcmNY=", + "requires": { + "kind-of": "3.2.2" + }, + "dependencies": { + "kind-of": { + "version": "3.2.2", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", + "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", + "requires": { + "is-buffer": "1.1.6" + } + } + } + }, + "is-data-descriptor": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-0.1.4.tgz", + "integrity": "sha1-C17mSDiOLIYCgueT8YVv7D8wG1Y=", + "requires": { + "kind-of": "3.2.2" + }, + "dependencies": { + "kind-of": { + "version": "3.2.2", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", + "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", + "requires": { + "is-buffer": "1.1.6" + } + } + } + }, "is-descriptor": { "version": "0.1.6", "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-0.1.6.tgz", @@ -683,39 +719,29 @@ } }, "is-accessor-descriptor": { - "version": "0.1.6", - "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-0.1.6.tgz", - "integrity": "sha1-qeEss66Nh2cn7u84Q/igiXtcmNY=", + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-1.0.0.tgz", + "integrity": "sha512-m5hnHTkcVsPfqx3AKlyttIPb7J+XykHvJP2B9bZDjlhLIoEq4XoK64Vg7boZlVWYK6LUY94dYPEE7Lh0ZkZKcQ==", "requires": { - "kind-of": "3.2.2" - }, - "dependencies": { - "kind-of": { - "version": "3.2.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", - "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", - "requires": { - "is-buffer": "1.1.6" - } - } + "kind-of": "6.0.2" } }, "is-data-descriptor": { - "version": "0.1.4", - "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-0.1.4.tgz", - "integrity": "sha1-C17mSDiOLIYCgueT8YVv7D8wG1Y=", + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-1.0.0.tgz", + "integrity": "sha512-jbRXy1FmtAoCjQkVmIVYwuuqDFUbaOeDjmed1tOGPrsMhtJA4rD9tkgA0F1qJ3gRFRXcHYVkdeaP50Q5rE/jLQ==", "requires": { - "kind-of": "3.2.2" - }, - "dependencies": { - "kind-of": { - "version": "3.2.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", - "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", - "requires": { - "is-buffer": "1.1.6" - } - } + "kind-of": "6.0.2" + } + }, + "is-descriptor": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-1.0.2.tgz", + "integrity": "sha512-2eis5WqQGV7peooDyLmNEPUrps9+SXX5c9pL3xEB+4e9HnGuDa7mB7kHxHw4CbqS9k1T2hOH3miL8n8WtiYVtg==", + "requires": { + "is-accessor-descriptor": "1.0.0", + "is-data-descriptor": "1.0.0", + "kind-of": "6.0.2" } }, "is-number": { @@ -753,7 +779,7 @@ "requires": { "arr-diff": "4.0.0", "array-unique": "0.3.2", - "braces": "2.3.1", + "braces": "2.3.2", "define-property": "2.0.2", "extend-shallow": "3.0.2", "extglob": "2.0.4", @@ -996,7 +1022,7 @@ "integrity": "sha1-Hb0cg1ZY41zj+ZhAmdsAWFx4IBQ=", "requires": { "browserslist": "1.7.7", - "caniuse-db": "1.0.30000823", + "caniuse-db": "1.0.30000827", "normalize-range": "0.1.2", "num2fraction": "1.2.2", "postcss": "5.2.18", @@ -1009,9 +1035,9 @@ "integrity": "sha1-FDQt0428yU0OW4fXY81jYSwOeU8=" }, "aws4": { - "version": "1.6.0", - "resolved": "https://registry.npmjs.org/aws4/-/aws4-1.6.0.tgz", - "integrity": "sha1-g+9cqGCysy5KDe7e6MdxudtXRx4=" + "version": "1.7.0", + "resolved": "https://registry.npmjs.org/aws4/-/aws4-1.7.0.tgz", + "integrity": "sha512-32NDda82rhwD9/JBCCkB+MRYDp0oSvlo2IL6rQWA10PQi7tDUM3eqMSltXmY+Oyl/7N3P3qNtAlv7X0d9bI28w==" }, "babel-code-frame": { "version": "6.26.0", @@ -1654,7 +1680,7 @@ "integrity": "sha1-N5k3q8Z9eJWXCtxiHyhM2WbPIVM=", "requires": { "babel-runtime": "6.26.0", - "core-js": "2.5.4", + "core-js": "2.5.5", "regenerator-runtime": "0.10.5" }, "dependencies": { @@ -1736,7 +1762,7 @@ "requires": { "babel-core": "6.26.0", "babel-runtime": "6.26.0", - "core-js": "2.5.4", + "core-js": "2.5.5", "home-or-tmp": "2.0.0", "lodash": "4.17.5", "mkdirp": "0.5.1", @@ -1748,7 +1774,7 @@ "resolved": "https://registry.npmjs.org/babel-runtime/-/babel-runtime-6.26.0.tgz", "integrity": "sha1-llxwWGaOgrVde/4E/yM3vItWR/4=", "requires": { - "core-js": "2.5.4", + "core-js": "2.5.5", "regenerator-runtime": "0.11.1" } }, @@ -1916,6 +1942,11 @@ "type-is": "1.6.16" }, "dependencies": { + "iconv-lite": { + "version": "0.4.19", + "resolved": "https://registry.npmjs.org/iconv-lite/-/iconv-lite-0.4.19.tgz", + "integrity": "sha512-oTZqweIP51xaGPI4uPa56/Pri/480R+mo7SeU+YETByQNhDG55ycFyNLIgta9vXhILrxXDmF7ZGhqZIcuN0gJQ==" + }, "qs": { "version": "6.5.1", "resolved": "https://registry.npmjs.org/qs/-/qs-6.5.1.tgz", @@ -1992,9 +2023,9 @@ "integrity": "sha512-erYug8XoqzU3IfcU8fUgyHqyOXqIE4tUTTQ+7mqUjQlvnXkOO6OlT9c/ZoJVHYoAaqGxr09CN53G7XIsO4KtWA==", "dev": true, "requires": { + "JSONStream": "1.3.2", "combine-source-map": "0.8.0", "defined": "1.0.0", - "JSONStream": "1.3.2", "safe-buffer": "5.1.1", "through2": "2.0.3", "umd": "3.0.3" @@ -2034,6 +2065,7 @@ "integrity": "sha512-gKfOsNQv/toWz+60nSPfYzuwSEdzvV2WdxrVPUbPD/qui44rAkB3t3muNtmmGYHqrG56FGwX9SUEQmzNLAeS7g==", "dev": true, "requires": { + "JSONStream": "1.3.2", "assert": "1.4.1", "browser-pack": "6.1.0", "browser-resolve": "1.11.2", @@ -2054,8 +2086,7 @@ "htmlescape": "1.1.1", "https-browserify": "1.0.0", "inherits": "2.0.3", - "insert-module-globals": "7.0.5", - "JSONStream": "1.3.2", + "insert-module-globals": "7.0.6", "labeled-stream-splicer": "2.0.1", "module-deps": "4.1.1", "os-browserify": "0.3.0", @@ -2066,7 +2097,7 @@ "querystring-es3": "0.2.1", "read-only-stream": "2.0.0", "readable-stream": "2.3.6", - "resolve": "1.6.0", + "resolve": "1.7.0", "shasum": "1.0.2", "shell-quote": "1.6.1", "stream-browserify": "2.0.1", @@ -2233,7 +2264,7 @@ "resolved": "https://registry.npmjs.org/browserslist/-/browserslist-1.7.7.tgz", "integrity": "sha1-C9dnBCWL6CmyOYu1Dkti0aFmsLk=", "requires": { - "caniuse-db": "1.0.30000823", + "caniuse-db": "1.0.30000827", "electron-to-chromium": "1.3.42" } }, @@ -2445,15 +2476,15 @@ "integrity": "sha1-tTTnxzTE+B7F++isoq0kNUuWLGw=", "requires": { "browserslist": "1.7.7", - "caniuse-db": "1.0.30000823", + "caniuse-db": "1.0.30000827", "lodash.memoize": "4.1.2", "lodash.uniq": "4.5.0" } }, "caniuse-db": { - "version": "1.0.30000823", - "resolved": "https://registry.npmjs.org/caniuse-db/-/caniuse-db-1.0.30000823.tgz", - "integrity": "sha1-5o5fjHB4PvQFnS6g3oH1UWUdpvw=" + "version": "1.0.30000827", + "resolved": "https://registry.npmjs.org/caniuse-db/-/caniuse-db-1.0.30000827.tgz", + "integrity": "sha1-vSg53Rlgk7RMKMF/k1ExQMnZJYg=" }, "caseless": { "version": "0.11.0", @@ -2546,7 +2577,7 @@ "requires": { "anymatch": "2.0.0", "async-each": "1.0.1", - "braces": "2.3.1", + "braces": "2.3.2", "glob-parent": "3.1.0", "inherits": "2.0.3", "is-binary-path": "1.0.1", @@ -2563,17 +2594,15 @@ "integrity": "sha1-qJS3XUvE9s1nnvMkSp/Y9Gri1Cg=" }, "braces": { - "version": "2.3.1", - "resolved": "https://registry.npmjs.org/braces/-/braces-2.3.1.tgz", - "integrity": "sha512-SO5lYHA3vO6gz66erVvedSCkp7AKWdv6VcQ2N4ysXfPxdAlxAMMAdwegGGcv1Bqwm7naF1hNdk5d6AAIEHV2nQ==", + "version": "2.3.2", + "resolved": "https://registry.npmjs.org/braces/-/braces-2.3.2.tgz", + "integrity": "sha512-aNdbnj9P8PjdXU4ybaWLK2IF3jc/EoDYbC7AazW6to3TRsfXxscC9UXOB5iDiEQrkyIbWp2SLQda4+QAa7nc3w==", "requires": { "arr-flatten": "1.1.0", "array-unique": "0.3.2", - "define-property": "1.0.0", "extend-shallow": "2.0.1", "fill-range": "4.0.0", "isobject": "3.0.1", - "kind-of": "6.0.2", "repeat-element": "1.1.2", "snapdragon": "0.8.2", "snapdragon-node": "2.1.1", @@ -2630,27 +2659,12 @@ "integrity": "sha1-JP1iAaR4LPUFYcgQJ2r8fRLXEZU=", "requires": { "kind-of": "3.2.2" - }, - "dependencies": { - "kind-of": { - "version": "3.2.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", - "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", - "requires": { - "is-buffer": "1.1.6" - } - } } }, "isobject": { "version": "3.0.1", "resolved": "https://registry.npmjs.org/isobject/-/isobject-3.0.1.tgz", "integrity": "sha1-TkMekrEalzFjaqH5yNHMvP2reN8=" - }, - "kind-of": { - "version": "6.0.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-6.0.2.tgz", - "integrity": "sha512-s5kLOcnH0XqDO+FvuaLX8DDjZ18CGFk7VygH40QoKPUQhW4e2rvM0rwUq0t8IQDOwYSeLK01U90OjzBTme2QqA==" } } }, @@ -2719,61 +2733,10 @@ "is-descriptor": "0.1.6" } }, - "is-accessor-descriptor": { - "version": "0.1.6", - "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-0.1.6.tgz", - "integrity": "sha1-qeEss66Nh2cn7u84Q/igiXtcmNY=", - "requires": { - "kind-of": "3.2.2" - }, - "dependencies": { - "kind-of": { - "version": "3.2.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", - "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", - "requires": { - "is-buffer": "1.1.6" - } - } - } - }, - "is-data-descriptor": { - "version": "0.1.4", - "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-0.1.4.tgz", - "integrity": "sha1-C17mSDiOLIYCgueT8YVv7D8wG1Y=", - "requires": { - "kind-of": "3.2.2" - }, - "dependencies": { - "kind-of": { - "version": "3.2.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", - "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", - "requires": { - "is-buffer": "1.1.6" - } - } - } - }, - "is-descriptor": { - "version": "0.1.6", - "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-0.1.6.tgz", - "integrity": "sha512-avDYr0SB3DwO9zsMov0gKCESFYqCnE4hq/4z3TdUlukEy5t9C0YRq7HLrsN52NAcqXKaepeCD0n+B0arnVG3Hg==", - "requires": { - "is-accessor-descriptor": "0.1.6", - "is-data-descriptor": "0.1.4", - "kind-of": "5.1.0" - } - }, "isobject": { "version": "3.0.1", "resolved": "https://registry.npmjs.org/isobject/-/isobject-3.0.1.tgz", "integrity": "sha1-TkMekrEalzFjaqH5yNHMvP2reN8=" - }, - "kind-of": { - "version": "5.1.0", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-5.1.0.tgz", - "integrity": "sha512-NGEErnH6F2vUuXDh+OlbcKW7/wOcfdRHaZ7VWtqCztfHri/++YKmP51OdWeGPuqCOba6kk2OTe5d02VmTB80Pw==" } } }, @@ -3261,9 +3224,9 @@ } }, "core-js": { - "version": "2.5.4", - "resolved": "https://registry.npmjs.org/core-js/-/core-js-2.5.4.tgz", - "integrity": "sha1-8si/GB8qgLkvNgEhQpzmOi8K6uA=" + "version": "2.5.5", + "resolved": "https://registry.npmjs.org/core-js/-/core-js-2.5.5.tgz", + "integrity": "sha1-sU3ek2xkDAV5prUMq8wTLdYSfjs=" }, "core-util-is": { "version": "1.0.2", @@ -3689,6 +3652,39 @@ "integrity": "sha1-dp66rz9KY6rTr56NMEybvnm/sOY=", "requires": { "is-descriptor": "1.0.2" + }, + "dependencies": { + "is-accessor-descriptor": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-1.0.0.tgz", + "integrity": "sha512-m5hnHTkcVsPfqx3AKlyttIPb7J+XykHvJP2B9bZDjlhLIoEq4XoK64Vg7boZlVWYK6LUY94dYPEE7Lh0ZkZKcQ==", + "requires": { + "kind-of": "6.0.2" + } + }, + "is-data-descriptor": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-1.0.0.tgz", + "integrity": "sha512-jbRXy1FmtAoCjQkVmIVYwuuqDFUbaOeDjmed1tOGPrsMhtJA4rD9tkgA0F1qJ3gRFRXcHYVkdeaP50Q5rE/jLQ==", + "requires": { + "kind-of": "6.0.2" + } + }, + "is-descriptor": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-1.0.2.tgz", + "integrity": "sha512-2eis5WqQGV7peooDyLmNEPUrps9+SXX5c9pL3xEB+4e9HnGuDa7mB7kHxHw4CbqS9k1T2hOH3miL8n8WtiYVtg==", + "requires": { + "is-accessor-descriptor": "1.0.0", + "is-data-descriptor": "1.0.0", + "kind-of": "6.0.2" + } + }, + "kind-of": { + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-6.0.2.tgz", + "integrity": "sha512-s5kLOcnH0XqDO+FvuaLX8DDjZ18CGFk7VygH40QoKPUQhW4e2rvM0rwUq0t8IQDOwYSeLK01U90OjzBTme2QqA==" + } } }, "defined": { @@ -4019,7 +4015,7 @@ "integrity": "sha1-U4tm8+5izRq1HsMjgp0flIDHS+s=", "dev": true, "requires": { - "iconv-lite": "0.4.19" + "iconv-lite": "0.4.21" } }, "end-of-stream": { @@ -4591,7 +4587,7 @@ "integrity": "sha512-bSn6gvGxKt+b7+6TKEv1ZycHleA7aHhRHyAqJyp5pbUFuYYNIzpZnQDk7AsYckyWdEnTeAnay0aCy2aV6iTk9A==", "requires": { "chardet": "0.4.2", - "iconv-lite": "0.4.19", + "iconv-lite": "0.4.21", "tmp": "0.0.33" } }, @@ -4650,18 +4646,16 @@ "dev": true }, "braces": { - "version": "2.3.1", - "resolved": "https://registry.npmjs.org/braces/-/braces-2.3.1.tgz", - "integrity": "sha512-SO5lYHA3vO6gz66erVvedSCkp7AKWdv6VcQ2N4ysXfPxdAlxAMMAdwegGGcv1Bqwm7naF1hNdk5d6AAIEHV2nQ==", + "version": "2.3.2", + "resolved": "https://registry.npmjs.org/braces/-/braces-2.3.2.tgz", + "integrity": "sha512-aNdbnj9P8PjdXU4ybaWLK2IF3jc/EoDYbC7AazW6to3TRsfXxscC9UXOB5iDiEQrkyIbWp2SLQda4+QAa7nc3w==", "dev": true, "requires": { "arr-flatten": "1.1.0", "array-unique": "0.3.2", - "define-property": "1.0.0", "extend-shallow": "2.0.1", "fill-range": "4.0.0", "isobject": "3.0.1", - "kind-of": "6.0.2", "repeat-element": "1.1.2", "snapdragon": "0.8.2", "snapdragon-node": "2.1.1", @@ -4669,15 +4663,6 @@ "to-regex": "3.0.2" }, "dependencies": { - "define-property": { - "version": "1.0.0", - "resolved": "https://registry.npmjs.org/define-property/-/define-property-1.0.0.tgz", - "integrity": "sha1-dp66rz9KY6rTr56NMEybvnm/sOY=", - "dev": true, - "requires": { - "is-descriptor": "1.0.2" - } - }, "extend-shallow": { "version": "2.0.1", "resolved": "https://registry.npmjs.org/extend-shallow/-/extend-shallow-2.0.1.tgz", @@ -4732,6 +4717,46 @@ "is-extendable": "0.1.1" } }, + "is-accessor-descriptor": { + "version": "0.1.6", + "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-0.1.6.tgz", + "integrity": "sha1-qeEss66Nh2cn7u84Q/igiXtcmNY=", + "dev": true, + "requires": { + "kind-of": "3.2.2" + }, + "dependencies": { + "kind-of": { + "version": "3.2.2", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", + "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", + "dev": true, + "requires": { + "is-buffer": "1.1.6" + } + } + } + }, + "is-data-descriptor": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-0.1.4.tgz", + "integrity": "sha1-C17mSDiOLIYCgueT8YVv7D8wG1Y=", + "dev": true, + "requires": { + "kind-of": "3.2.2" + }, + "dependencies": { + "kind-of": { + "version": "3.2.2", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", + "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", + "dev": true, + "requires": { + "is-buffer": "1.1.6" + } + } + } + }, "is-descriptor": { "version": "0.1.6", "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-0.1.6.tgz", @@ -4853,43 +4878,32 @@ } }, "is-accessor-descriptor": { - "version": "0.1.6", - "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-0.1.6.tgz", - "integrity": "sha1-qeEss66Nh2cn7u84Q/igiXtcmNY=", + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-1.0.0.tgz", + "integrity": "sha512-m5hnHTkcVsPfqx3AKlyttIPb7J+XykHvJP2B9bZDjlhLIoEq4XoK64Vg7boZlVWYK6LUY94dYPEE7Lh0ZkZKcQ==", "dev": true, "requires": { - "kind-of": "3.2.2" - }, - "dependencies": { - "kind-of": { - "version": "3.2.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", - "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", - "dev": true, - "requires": { - "is-buffer": "1.1.6" - } - } + "kind-of": "6.0.2" } }, "is-data-descriptor": { - "version": "0.1.4", - "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-0.1.4.tgz", - "integrity": "sha1-C17mSDiOLIYCgueT8YVv7D8wG1Y=", + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-1.0.0.tgz", + "integrity": "sha512-jbRXy1FmtAoCjQkVmIVYwuuqDFUbaOeDjmed1tOGPrsMhtJA4rD9tkgA0F1qJ3gRFRXcHYVkdeaP50Q5rE/jLQ==", "dev": true, "requires": { - "kind-of": "3.2.2" - }, - "dependencies": { - "kind-of": { - "version": "3.2.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", - "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", - "dev": true, - "requires": { - "is-buffer": "1.1.6" - } - } + "kind-of": "6.0.2" + } + }, + "is-descriptor": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-1.0.2.tgz", + "integrity": "sha512-2eis5WqQGV7peooDyLmNEPUrps9+SXX5c9pL3xEB+4e9HnGuDa7mB7kHxHw4CbqS9k1T2hOH3miL8n8WtiYVtg==", + "dev": true, + "requires": { + "is-accessor-descriptor": "1.0.0", + "is-data-descriptor": "1.0.0", + "kind-of": "6.0.2" } }, "is-extglob": { @@ -4947,7 +4961,7 @@ "requires": { "arr-diff": "4.0.0", "array-unique": "0.3.2", - "braces": "2.3.1", + "braces": "2.3.2", "define-property": "2.0.2", "extend-shallow": "3.0.2", "extglob": "2.0.4", @@ -5816,9 +5830,9 @@ } }, "html-minifier": { - "version": "3.5.13", - "resolved": "https://registry.npmjs.org/html-minifier/-/html-minifier-3.5.13.tgz", - "integrity": "sha512-B7P99uf0LPQ5lslyhrAZAXE7Lk1tpiv52KVapKbeFhgqNMUI7JBd/fYLX55imu3Rz7sCTzZM6r/IBe4oT7qCjg==", + "version": "3.5.14", + "resolved": "https://registry.npmjs.org/html-minifier/-/html-minifier-3.5.14.tgz", + "integrity": "sha512-sZjw6zhQgyUnIlIPU+W80XpRjWjdxHtNcxjfyOskOsCTDKytcfLY04wsQY/83Yqb4ndoiD2FtauiL7Yg6uUQFQ==", "requires": { "camel-case": "3.0.0", "clean-css": "4.1.11", @@ -5826,7 +5840,7 @@ "he": "1.1.1", "param-case": "2.1.1", "relateurl": "0.2.7", - "uglify-js": "3.3.18" + "uglify-js": "3.3.20" }, "dependencies": { "source-map": { @@ -5835,9 +5849,9 @@ "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==" }, "uglify-js": { - "version": "3.3.18", - "resolved": "https://registry.npmjs.org/uglify-js/-/uglify-js-3.3.18.tgz", - "integrity": "sha512-VhjIFv93KnTx/ntNi9yTBbfrsWnQnqUy02MT32uqU/5i2oEJ8GAEJ0AwYV206JeOmIzSjm41Ba0iXVKv6j7y9g==", + "version": "3.3.20", + "resolved": "https://registry.npmjs.org/uglify-js/-/uglify-js-3.3.20.tgz", + "integrity": "sha512-WpLkWCf9sGvGZnIvBV0PNID9BATQNT/IXKAmqegfKzIPcTmTV3FP8NQpoogQkt/Y402x2sOFdaHUmqFY9IZp+g==", "requires": { "commander": "2.15.1", "source-map": "0.6.1" @@ -5850,7 +5864,7 @@ "resolved": "https://registry.npmjs.org/html-webpack-plugin/-/html-webpack-plugin-3.2.0.tgz", "integrity": "sha1-sBq71yOsqqeze2r0SS69oD2d03s=", "requires": { - "html-minifier": "3.5.13", + "html-minifier": "3.5.14", "loader-utils": "0.2.17", "lodash": "4.17.5", "pretty-error": "2.1.1", @@ -5888,7 +5902,7 @@ "posthtml": "0.11.3", "posthtml-render": "1.1.3", "svgo": "1.0.5", - "uglify-js": "3.3.18" + "uglify-js": "3.3.20" }, "dependencies": { "coa": { @@ -5954,9 +5968,9 @@ } }, "uglify-js": { - "version": "3.3.18", - "resolved": "https://registry.npmjs.org/uglify-js/-/uglify-js-3.3.18.tgz", - "integrity": "sha512-VhjIFv93KnTx/ntNi9yTBbfrsWnQnqUy02MT32uqU/5i2oEJ8GAEJ0AwYV206JeOmIzSjm41Ba0iXVKv6j7y9g==", + "version": "3.3.20", + "resolved": "https://registry.npmjs.org/uglify-js/-/uglify-js-3.3.20.tgz", + "integrity": "sha512-WpLkWCf9sGvGZnIvBV0PNID9BATQNT/IXKAmqegfKzIPcTmTV3FP8NQpoogQkt/Y402x2sOFdaHUmqFY9IZp+g==", "requires": { "commander": "2.15.1", "source-map": "0.6.1" @@ -6061,9 +6075,12 @@ "integrity": "sha1-7AbBDgo0wPL68Zn3/X/Hj//QPHM=" }, "iconv-lite": { - "version": "0.4.19", - "resolved": "https://registry.npmjs.org/iconv-lite/-/iconv-lite-0.4.19.tgz", - "integrity": "sha512-oTZqweIP51xaGPI4uPa56/Pri/480R+mo7SeU+YETByQNhDG55ycFyNLIgta9vXhILrxXDmF7ZGhqZIcuN0gJQ==" + "version": "0.4.21", + "resolved": "https://registry.npmjs.org/iconv-lite/-/iconv-lite-0.4.21.tgz", + "integrity": "sha512-En5V9za5mBt2oUA03WGD3TwDv0MKAruqsuxstbMUZaj9W9k/m1CV/9py3l0L5kw9Bln8fdHQmzHSYtvpvTLpKw==", + "requires": { + "safer-buffer": "2.1.2" + } }, "icss-replace-symbols": { "version": "1.1.0", @@ -6290,16 +6307,17 @@ } }, "insert-module-globals": { - "version": "7.0.5", - "resolved": "https://registry.npmjs.org/insert-module-globals/-/insert-module-globals-7.0.5.tgz", - "integrity": "sha512-wgRtrCpMm0ruH2hgLUIx+9YfJsgJQmU1KkPUzTuatW9dbH19yPRqAQhFX1HJU6zbmg2IMmt80BgSE5MWuksw3Q==", + "version": "7.0.6", + "resolved": "https://registry.npmjs.org/insert-module-globals/-/insert-module-globals-7.0.6.tgz", + "integrity": "sha512-R3sidKJr3SsggqQQ5cEwQb3pWG8RNx0UnpyeiOSR6jorRIeAOzH2gkTWnNdMnyRiVbjrG047K7UCtlMkQ1Mo9w==", "dev": true, "requires": { + "JSONStream": "1.3.2", "combine-source-map": "0.8.0", "concat-stream": "1.6.2", "is-buffer": "1.1.6", - "JSONStream": "1.3.2", "lexical-scope": "1.2.0", + "path-is-absolute": "1.0.1", "process": "0.11.10", "through2": "2.0.3", "xtend": "4.0.1" @@ -6356,18 +6374,11 @@ "integrity": "sha1-UFMN+4T8yap9vnhS6Do3uTufKqY=" }, "is-accessor-descriptor": { - "version": "1.0.0", - "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-1.0.0.tgz", - "integrity": "sha512-m5hnHTkcVsPfqx3AKlyttIPb7J+XykHvJP2B9bZDjlhLIoEq4XoK64Vg7boZlVWYK6LUY94dYPEE7Lh0ZkZKcQ==", + "version": "0.1.6", + "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-0.1.6.tgz", + "integrity": "sha1-qeEss66Nh2cn7u84Q/igiXtcmNY=", "requires": { - "kind-of": "6.0.2" - }, - "dependencies": { - "kind-of": { - "version": "6.0.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-6.0.2.tgz", - "integrity": "sha512-s5kLOcnH0XqDO+FvuaLX8DDjZ18CGFk7VygH40QoKPUQhW4e2rvM0rwUq0t8IQDOwYSeLK01U90OjzBTme2QqA==" - } + "kind-of": "3.2.2" } }, "is-arrayish": { @@ -6409,18 +6420,11 @@ "integrity": "sha1-hut1OSgF3cM69xySoO7fdO52BLI=" }, "is-data-descriptor": { - "version": "1.0.0", - "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-1.0.0.tgz", - "integrity": "sha512-jbRXy1FmtAoCjQkVmIVYwuuqDFUbaOeDjmed1tOGPrsMhtJA4rD9tkgA0F1qJ3gRFRXcHYVkdeaP50Q5rE/jLQ==", + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-0.1.4.tgz", + "integrity": "sha1-C17mSDiOLIYCgueT8YVv7D8wG1Y=", "requires": { - "kind-of": "6.0.2" - }, - "dependencies": { - "kind-of": { - "version": "6.0.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-6.0.2.tgz", - "integrity": "sha512-s5kLOcnH0XqDO+FvuaLX8DDjZ18CGFk7VygH40QoKPUQhW4e2rvM0rwUq0t8IQDOwYSeLK01U90OjzBTme2QqA==" - } + "kind-of": "3.2.2" } }, "is-date-object": { @@ -6429,19 +6433,19 @@ "integrity": "sha1-mqIOtq7rv/d/vTPnTKAbM1gdOhY=" }, "is-descriptor": { - "version": "1.0.2", - "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-1.0.2.tgz", - "integrity": "sha512-2eis5WqQGV7peooDyLmNEPUrps9+SXX5c9pL3xEB+4e9HnGuDa7mB7kHxHw4CbqS9k1T2hOH3miL8n8WtiYVtg==", + "version": "0.1.6", + "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-0.1.6.tgz", + "integrity": "sha512-avDYr0SB3DwO9zsMov0gKCESFYqCnE4hq/4z3TdUlukEy5t9C0YRq7HLrsN52NAcqXKaepeCD0n+B0arnVG3Hg==", "requires": { - "is-accessor-descriptor": "1.0.0", - "is-data-descriptor": "1.0.0", - "kind-of": "6.0.2" + "is-accessor-descriptor": "0.1.6", + "is-data-descriptor": "0.1.4", + "kind-of": "5.1.0" }, "dependencies": { "kind-of": { - "version": "6.0.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-6.0.2.tgz", - "integrity": "sha512-s5kLOcnH0XqDO+FvuaLX8DDjZ18CGFk7VygH40QoKPUQhW4e2rvM0rwUq0t8IQDOwYSeLK01U90OjzBTme2QqA==" + "version": "5.1.0", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-5.1.0.tgz", + "integrity": "sha512-NGEErnH6F2vUuXDh+OlbcKW7/wOcfdRHaZ7VWtqCztfHri/++YKmP51OdWeGPuqCOba6kk2OTe5d02VmTB80Pw==" } } }, @@ -6937,16 +6941,6 @@ "resolved": "https://registry.npmjs.org/jsonpointer/-/jsonpointer-4.0.1.tgz", "integrity": "sha1-T9kss04OnbPInIYi7PUfm5eMbLk=" }, - "JSONStream": { - "version": "1.3.2", - "resolved": "https://registry.npmjs.org/JSONStream/-/JSONStream-1.3.2.tgz", - "integrity": "sha1-wQI3G27Dp887hHygDCC7D85Mbeo=", - "dev": true, - "requires": { - "jsonparse": "1.3.1", - "through": "2.3.8" - } - }, "jsprim": { "version": "1.4.1", "resolved": "https://registry.npmjs.org/jsprim/-/jsprim-1.4.1.tgz", @@ -7029,7 +7023,7 @@ "colors": "1.1.2", "combine-lists": "1.0.1", "connect": "3.6.6", - "core-js": "2.5.4", + "core-js": "2.5.5", "di": "0.0.1", "dom-serialize": "2.2.1", "expand-braces": "0.1.2", @@ -7742,9 +7736,9 @@ "integrity": "sha1-4PyVEztu8nbNyIh82vJKpvFW+Po=" }, "loglevelnext": { - "version": "1.0.3", - "resolved": "https://registry.npmjs.org/loglevelnext/-/loglevelnext-1.0.3.tgz", - "integrity": "sha512-OCxd/b78TijTB4b6zVqLbMrxhebyvdZKwqpL0VHUZ0pYhavXaPD4l6Xrr4n5xqTYWiqtb0i7ikSoJY/myQ/Org==" + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/loglevelnext/-/loglevelnext-1.0.4.tgz", + "integrity": "sha512-V3N6LAJAiGwa/zjtvmgs2tyeiCJ23bGNhxXN8R+v7k6TNlSlTz40mIyZYdmO762eBnEFymn0Mhha+WuAhnwMBg==" }, "lolex": { "version": "1.6.0", @@ -8240,6 +8234,7 @@ "integrity": "sha1-IyFYM/HaE/1gbMuAh7RIUty4If0=", "dev": true, "requires": { + "JSONStream": "1.3.2", "browser-resolve": "1.11.2", "cached-path-relative": "1.0.1", "concat-stream": "1.5.2", @@ -8247,10 +8242,9 @@ "detective": "4.7.1", "duplexer2": "0.1.4", "inherits": "2.0.3", - "JSONStream": "1.3.2", "parents": "1.0.1", "readable-stream": "2.3.6", - "resolve": "1.6.0", + "resolve": "1.7.0", "stream-combiner2": "1.1.1", "subarg": "1.0.0", "through2": "2.0.3", @@ -8398,6 +8392,32 @@ "is-extendable": "1.0.1" } }, + "is-accessor-descriptor": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-1.0.0.tgz", + "integrity": "sha512-m5hnHTkcVsPfqx3AKlyttIPb7J+XykHvJP2B9bZDjlhLIoEq4XoK64Vg7boZlVWYK6LUY94dYPEE7Lh0ZkZKcQ==", + "requires": { + "kind-of": "6.0.2" + } + }, + "is-data-descriptor": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-1.0.0.tgz", + "integrity": "sha512-jbRXy1FmtAoCjQkVmIVYwuuqDFUbaOeDjmed1tOGPrsMhtJA4rD9tkgA0F1qJ3gRFRXcHYVkdeaP50Q5rE/jLQ==", + "requires": { + "kind-of": "6.0.2" + } + }, + "is-descriptor": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-1.0.2.tgz", + "integrity": "sha512-2eis5WqQGV7peooDyLmNEPUrps9+SXX5c9pL3xEB+4e9HnGuDa7mB7kHxHw4CbqS9k1T2hOH3miL8n8WtiYVtg==", + "requires": { + "is-accessor-descriptor": "1.0.0", + "is-data-descriptor": "1.0.0", + "kind-of": "6.0.2" + } + }, "is-extendable": { "version": "1.0.1", "resolved": "https://registry.npmjs.org/is-extendable/-/is-extendable-1.0.1.tgz", @@ -8744,39 +8764,6 @@ "requires": { "is-descriptor": "0.1.6" } - }, - "is-accessor-descriptor": { - "version": "0.1.6", - "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-0.1.6.tgz", - "integrity": "sha1-qeEss66Nh2cn7u84Q/igiXtcmNY=", - "requires": { - "kind-of": "3.2.2" - } - }, - "is-data-descriptor": { - "version": "0.1.4", - "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-0.1.4.tgz", - "integrity": "sha1-C17mSDiOLIYCgueT8YVv7D8wG1Y=", - "requires": { - "kind-of": "3.2.2" - } - }, - "is-descriptor": { - "version": "0.1.6", - "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-0.1.6.tgz", - "integrity": "sha512-avDYr0SB3DwO9zsMov0gKCESFYqCnE4hq/4z3TdUlukEy5t9C0YRq7HLrsN52NAcqXKaepeCD0n+B0arnVG3Hg==", - "requires": { - "is-accessor-descriptor": "0.1.6", - "is-data-descriptor": "0.1.4", - "kind-of": "5.1.0" - }, - "dependencies": { - "kind-of": { - "version": "5.1.0", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-5.1.0.tgz", - "integrity": "sha512-NGEErnH6F2vUuXDh+OlbcKW7/wOcfdRHaZ7VWtqCztfHri/++YKmP51OdWeGPuqCOba6kk2OTe5d02VmTB80Pw==" - } - } } } }, @@ -10116,7 +10103,7 @@ "jstransformer": "1.0.0", "pug-error": "1.3.2", "pug-walk": "1.1.7", - "resolve": "1.6.0", + "resolve": "1.7.0", "uglify-js": "2.8.29" } }, @@ -10177,7 +10164,7 @@ "requires": { "loader-utils": "1.1.0", "pug-walk": "1.1.7", - "resolve": "1.6.0" + "resolve": "1.7.0" } }, "pug-parser": { @@ -10357,6 +10344,11 @@ "statuses": "1.4.0" } }, + "iconv-lite": { + "version": "0.4.19", + "resolved": "https://registry.npmjs.org/iconv-lite/-/iconv-lite-0.4.19.tgz", + "integrity": "sha512-oTZqweIP51xaGPI4uPa56/Pri/480R+mo7SeU+YETByQNhDG55ycFyNLIgta9vXhILrxXDmF7ZGhqZIcuN0gJQ==" + }, "setprototypeof": { "version": "1.0.3", "resolved": "https://registry.npmjs.org/setprototypeof/-/setprototypeof-1.0.3.tgz", @@ -10490,7 +10482,7 @@ "resolved": "https://registry.npmjs.org/rechoir/-/rechoir-0.6.2.tgz", "integrity": "sha1-hSBLVNuoLVdC4oyWdW70OvUOM4Q=", "requires": { - "resolve": "1.6.0" + "resolve": "1.7.0" } }, "redent": { @@ -10753,7 +10745,7 @@ "integrity": "sha1-Tf5b9r6LjNw3/Pk+BLZVd3InEN4=", "requires": { "aws-sign2": "0.6.0", - "aws4": "1.6.0", + "aws4": "1.7.0", "caseless": "0.11.0", "combined-stream": "1.0.6", "extend": "3.0.1", @@ -10799,9 +10791,9 @@ "integrity": "sha1-kl0mAdOaxIXgkc8NpcbmlNw9yv8=" }, "resolve": { - "version": "1.6.0", - "resolved": "https://registry.npmjs.org/resolve/-/resolve-1.6.0.tgz", - "integrity": "sha512-mw7JQNu5ExIkcw4LPih0owX/TZXjD/ZUF/ZQ/pDnkw3ZKhDcZZw5klmBlj6gVMwjQ3Pz5Jgu7F3d0jcDVuEWdw==", + "version": "1.7.0", + "resolved": "https://registry.npmjs.org/resolve/-/resolve-1.7.0.tgz", + "integrity": "sha512-QdgZ5bjR1WAlpLaO5yHepFvC+o3rCr6wpfE2tpJNMkXdulf2jKomQBdNRQITF3ZKHNlT71syG98yQP03gasgnA==", "requires": { "path-parse": "1.0.5" } @@ -10952,7 +10944,7 @@ "acorn": "5.5.3", "estree-walker": "0.5.1", "magic-string": "0.22.5", - "resolve": "1.6.0", + "resolve": "1.7.0", "rollup-pluginutils": "2.0.1" } }, @@ -10963,7 +10955,7 @@ "requires": { "builtin-modules": "2.0.0", "is-module": "1.0.0", - "resolve": "1.6.0" + "resolve": "1.7.0" } }, "rollup-plugin-progress": { @@ -11091,6 +11083,11 @@ "ret": "0.1.15" } }, + "safer-buffer": { + "version": "2.1.2", + "resolved": "https://registry.npmjs.org/safer-buffer/-/safer-buffer-2.1.2.tgz", + "integrity": "sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg==" + }, "samsam": { "version": "1.3.0", "resolved": "https://registry.npmjs.org/samsam/-/samsam-1.3.0.tgz", @@ -11490,57 +11487,6 @@ "requires": { "is-descriptor": "0.1.6" } - }, - "is-accessor-descriptor": { - "version": "0.1.6", - "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-0.1.6.tgz", - "integrity": "sha1-qeEss66Nh2cn7u84Q/igiXtcmNY=", - "requires": { - "kind-of": "3.2.2" - }, - "dependencies": { - "kind-of": { - "version": "3.2.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", - "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", - "requires": { - "is-buffer": "1.1.6" - } - } - } - }, - "is-data-descriptor": { - "version": "0.1.4", - "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-0.1.4.tgz", - "integrity": "sha1-C17mSDiOLIYCgueT8YVv7D8wG1Y=", - "requires": { - "kind-of": "3.2.2" - }, - "dependencies": { - "kind-of": { - "version": "3.2.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", - "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", - "requires": { - "is-buffer": "1.1.6" - } - } - } - }, - "is-descriptor": { - "version": "0.1.6", - "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-0.1.6.tgz", - "integrity": "sha512-avDYr0SB3DwO9zsMov0gKCESFYqCnE4hq/4z3TdUlukEy5t9C0YRq7HLrsN52NAcqXKaepeCD0n+B0arnVG3Hg==", - "requires": { - "is-accessor-descriptor": "0.1.6", - "is-data-descriptor": "0.1.4", - "kind-of": "5.1.0" - } - }, - "kind-of": { - "version": "5.1.0", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-5.1.0.tgz", - "integrity": "sha512-NGEErnH6F2vUuXDh+OlbcKW7/wOcfdRHaZ7VWtqCztfHri/++YKmP51OdWeGPuqCOba6kk2OTe5d02VmTB80Pw==" } } }, @@ -11808,7 +11754,7 @@ "faye-websocket": "0.11.1", "inherits": "2.0.3", "json3": "3.3.2", - "url-parse": "1.2.0" + "url-parse": "1.3.0" }, "dependencies": { "faye-websocket": { @@ -11998,57 +11944,6 @@ "requires": { "is-descriptor": "0.1.6" } - }, - "is-accessor-descriptor": { - "version": "0.1.6", - "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-0.1.6.tgz", - "integrity": "sha1-qeEss66Nh2cn7u84Q/igiXtcmNY=", - "requires": { - "kind-of": "3.2.2" - }, - "dependencies": { - "kind-of": { - "version": "3.2.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", - "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", - "requires": { - "is-buffer": "1.1.6" - } - } - } - }, - "is-data-descriptor": { - "version": "0.1.4", - "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-0.1.4.tgz", - "integrity": "sha1-C17mSDiOLIYCgueT8YVv7D8wG1Y=", - "requires": { - "kind-of": "3.2.2" - }, - "dependencies": { - "kind-of": { - "version": "3.2.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", - "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", - "requires": { - "is-buffer": "1.1.6" - } - } - } - }, - "is-descriptor": { - "version": "0.1.6", - "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-0.1.6.tgz", - "integrity": "sha512-avDYr0SB3DwO9zsMov0gKCESFYqCnE4hq/4z3TdUlukEy5t9C0YRq7HLrsN52NAcqXKaepeCD0n+B0arnVG3Hg==", - "requires": { - "is-accessor-descriptor": "0.1.6", - "is-data-descriptor": "0.1.4", - "kind-of": "5.1.0" - } - }, - "kind-of": { - "version": "5.1.0", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-5.1.0.tgz", - "integrity": "sha512-NGEErnH6F2vUuXDh+OlbcKW7/wOcfdRHaZ7VWtqCztfHri/++YKmP51OdWeGPuqCOba6kk2OTe5d02VmTB80Pw==" } } }, @@ -12156,14 +12051,6 @@ "resolved": "https://registry.npmjs.org/strict-uri-encode/-/strict-uri-encode-1.1.0.tgz", "integrity": "sha1-J5siXfHVgrH1TmWt3UNS4Y+qBxM=" }, - "string_decoder": { - "version": "1.1.1", - "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-1.1.1.tgz", - "integrity": "sha512-n/ShnvDi6FHbbVfviro+WojiFzv+s8MPMHBczVePfUpDJLwoLT0ht1l4YwBCbi8pJAveEEdnkHyPyTP/mzRfwg==", - "requires": { - "safe-buffer": "5.1.1" - } - }, "string-template": { "version": "0.2.1", "resolved": "https://registry.npmjs.org/string-template/-/string-template-0.2.1.tgz", @@ -12193,6 +12080,14 @@ } } }, + "string_decoder": { + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/string_decoder/-/string_decoder-1.1.1.tgz", + "integrity": "sha512-n/ShnvDi6FHbbVfviro+WojiFzv+s8MPMHBczVePfUpDJLwoLT0ht1l4YwBCbi8pJAveEEdnkHyPyTP/mzRfwg==", + "requires": { + "safe-buffer": "5.1.1" + } + }, "stringstream": { "version": "0.0.5", "resolved": "https://registry.npmjs.org/stringstream/-/stringstream-0.0.5.tgz", @@ -12313,29 +12208,20 @@ "integrity": "sha1-qJS3XUvE9s1nnvMkSp/Y9Gri1Cg=" }, "braces": { - "version": "2.3.1", - "resolved": "https://registry.npmjs.org/braces/-/braces-2.3.1.tgz", - "integrity": "sha512-SO5lYHA3vO6gz66erVvedSCkp7AKWdv6VcQ2N4ysXfPxdAlxAMMAdwegGGcv1Bqwm7naF1hNdk5d6AAIEHV2nQ==", + "version": "2.3.2", + "resolved": "https://registry.npmjs.org/braces/-/braces-2.3.2.tgz", + "integrity": "sha512-aNdbnj9P8PjdXU4ybaWLK2IF3jc/EoDYbC7AazW6to3TRsfXxscC9UXOB5iDiEQrkyIbWp2SLQda4+QAa7nc3w==", "requires": { "arr-flatten": "1.1.0", "array-unique": "0.3.2", - "define-property": "1.0.0", "extend-shallow": "2.0.1", "fill-range": "4.0.0", "isobject": "3.0.1", - "kind-of": "6.0.2", "repeat-element": "1.1.2", "snapdragon": "0.8.2", "snapdragon-node": "2.1.1", "split-string": "3.1.0", "to-regex": "3.0.2" - }, - "dependencies": { - "kind-of": { - "version": "6.0.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-6.0.2.tgz", - "integrity": "sha512-s5kLOcnH0XqDO+FvuaLX8DDjZ18CGFk7VygH40QoKPUQhW4e2rvM0rwUq0t8IQDOwYSeLK01U90OjzBTme2QqA==" - } } }, "clone": { @@ -12393,52 +12279,6 @@ "to-regex-range": "2.1.1" } }, - "is-accessor-descriptor": { - "version": "0.1.6", - "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-0.1.6.tgz", - "integrity": "sha1-qeEss66Nh2cn7u84Q/igiXtcmNY=", - "requires": { - "kind-of": "3.2.2" - }, - "dependencies": { - "kind-of": { - "version": "3.2.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", - "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", - "requires": { - "is-buffer": "1.1.6" - } - } - } - }, - "is-data-descriptor": { - "version": "0.1.4", - "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-0.1.4.tgz", - "integrity": "sha1-C17mSDiOLIYCgueT8YVv7D8wG1Y=", - "requires": { - "kind-of": "3.2.2" - }, - "dependencies": { - "kind-of": { - "version": "3.2.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", - "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", - "requires": { - "is-buffer": "1.1.6" - } - } - } - }, - "is-descriptor": { - "version": "0.1.6", - "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-0.1.6.tgz", - "integrity": "sha512-avDYr0SB3DwO9zsMov0gKCESFYqCnE4hq/4z3TdUlukEy5t9C0YRq7HLrsN52NAcqXKaepeCD0n+B0arnVG3Hg==", - "requires": { - "is-accessor-descriptor": "0.1.6", - "is-data-descriptor": "0.1.4", - "kind-of": "5.1.0" - } - }, "is-number": { "version": "3.0.0", "resolved": "https://registry.npmjs.org/is-number/-/is-number-3.0.0.tgz", @@ -12474,7 +12314,7 @@ "requires": { "arr-diff": "4.0.0", "array-unique": "0.3.2", - "braces": "2.3.1", + "braces": "2.3.2", "define-property": "1.0.0", "extend-shallow": "2.0.1", "extglob": "2.0.4", @@ -12749,6 +12589,32 @@ "is-extendable": "1.0.1" } }, + "is-accessor-descriptor": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-1.0.0.tgz", + "integrity": "sha512-m5hnHTkcVsPfqx3AKlyttIPb7J+XykHvJP2B9bZDjlhLIoEq4XoK64Vg7boZlVWYK6LUY94dYPEE7Lh0ZkZKcQ==", + "requires": { + "kind-of": "6.0.2" + } + }, + "is-data-descriptor": { + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-1.0.0.tgz", + "integrity": "sha512-jbRXy1FmtAoCjQkVmIVYwuuqDFUbaOeDjmed1tOGPrsMhtJA4rD9tkgA0F1qJ3gRFRXcHYVkdeaP50Q5rE/jLQ==", + "requires": { + "kind-of": "6.0.2" + } + }, + "is-descriptor": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-1.0.2.tgz", + "integrity": "sha512-2eis5WqQGV7peooDyLmNEPUrps9+SXX5c9pL3xEB+4e9HnGuDa7mB7kHxHw4CbqS9k1T2hOH3miL8n8WtiYVtg==", + "requires": { + "is-accessor-descriptor": "1.0.0", + "is-data-descriptor": "1.0.0", + "kind-of": "6.0.2" + } + }, "is-extendable": { "version": "1.0.1", "resolved": "https://registry.npmjs.org/is-extendable/-/is-extendable-1.0.1.tgz", @@ -12761,6 +12627,11 @@ "version": "3.0.1", "resolved": "https://registry.npmjs.org/isobject/-/isobject-3.0.1.tgz", "integrity": "sha1-TkMekrEalzFjaqH5yNHMvP2reN8=" + }, + "kind-of": { + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-6.0.2.tgz", + "integrity": "sha512-s5kLOcnH0XqDO+FvuaLX8DDjZ18CGFk7VygH40QoKPUQhW4e2rvM0rwUq0t8IQDOwYSeLK01U90OjzBTme2QqA==" } } }, @@ -13121,9 +12992,9 @@ "integrity": "sha1-TTNA6AfTdzvamZH4MFrNzCpmXSo=" }, "url-parse": { - "version": "1.2.0", - "resolved": "https://registry.npmjs.org/url-parse/-/url-parse-1.2.0.tgz", - "integrity": "sha512-DT1XbYAfmQP65M/mE6OALxmXzZ/z1+e5zk2TcSKe/KiYbNGZxgtttzC0mR/sjopbpOXcbniq7eIKmocJnUWlEw==", + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/url-parse/-/url-parse-1.3.0.tgz", + "integrity": "sha512-zPvPA3T7P6M+0iNsgX+iAcAz4GshKrowtQBHHc/28tVsBc8jK7VRCNX+2GEcoE6zDB6XqXhcyiUWPVZY6C70Cg==", "requires": { "querystringify": "1.0.0", "requires-port": "1.0.0" @@ -13393,17 +13264,15 @@ "integrity": "sha1-qJS3XUvE9s1nnvMkSp/Y9Gri1Cg=" }, "braces": { - "version": "2.3.1", - "resolved": "https://registry.npmjs.org/braces/-/braces-2.3.1.tgz", - "integrity": "sha512-SO5lYHA3vO6gz66erVvedSCkp7AKWdv6VcQ2N4ysXfPxdAlxAMMAdwegGGcv1Bqwm7naF1hNdk5d6AAIEHV2nQ==", + "version": "2.3.2", + "resolved": "https://registry.npmjs.org/braces/-/braces-2.3.2.tgz", + "integrity": "sha512-aNdbnj9P8PjdXU4ybaWLK2IF3jc/EoDYbC7AazW6to3TRsfXxscC9UXOB5iDiEQrkyIbWp2SLQda4+QAa7nc3w==", "requires": { "arr-flatten": "1.1.0", "array-unique": "0.3.2", - "define-property": "1.0.0", "extend-shallow": "2.0.1", "fill-range": "4.0.0", "isobject": "3.0.1", - "kind-of": "6.0.2", "repeat-element": "1.1.2", "snapdragon": "0.8.2", "snapdragon-node": "2.1.1", @@ -13411,14 +13280,6 @@ "to-regex": "3.0.2" }, "dependencies": { - "define-property": { - "version": "1.0.0", - "resolved": "https://registry.npmjs.org/define-property/-/define-property-1.0.0.tgz", - "integrity": "sha1-dp66rz9KY6rTr56NMEybvnm/sOY=", - "requires": { - "is-descriptor": "1.0.2" - } - }, "extend-shallow": { "version": "2.0.1", "resolved": "https://registry.npmjs.org/extend-shallow/-/extend-shallow-2.0.1.tgz", @@ -13468,6 +13329,42 @@ "is-extendable": "0.1.1" } }, + "is-accessor-descriptor": { + "version": "0.1.6", + "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-0.1.6.tgz", + "integrity": "sha1-qeEss66Nh2cn7u84Q/igiXtcmNY=", + "requires": { + "kind-of": "3.2.2" + }, + "dependencies": { + "kind-of": { + "version": "3.2.2", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", + "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", + "requires": { + "is-buffer": "1.1.6" + } + } + } + }, + "is-data-descriptor": { + "version": "0.1.4", + "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-0.1.4.tgz", + "integrity": "sha1-C17mSDiOLIYCgueT8YVv7D8wG1Y=", + "requires": { + "kind-of": "3.2.2" + }, + "dependencies": { + "kind-of": { + "version": "3.2.2", + "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", + "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", + "requires": { + "is-buffer": "1.1.6" + } + } + } + }, "is-descriptor": { "version": "0.1.6", "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-0.1.6.tgz", @@ -13559,39 +13456,29 @@ } }, "is-accessor-descriptor": { - "version": "0.1.6", - "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-0.1.6.tgz", - "integrity": "sha1-qeEss66Nh2cn7u84Q/igiXtcmNY=", + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-accessor-descriptor/-/is-accessor-descriptor-1.0.0.tgz", + "integrity": "sha512-m5hnHTkcVsPfqx3AKlyttIPb7J+XykHvJP2B9bZDjlhLIoEq4XoK64Vg7boZlVWYK6LUY94dYPEE7Lh0ZkZKcQ==", "requires": { - "kind-of": "3.2.2" - }, - "dependencies": { - "kind-of": { - "version": "3.2.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", - "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", - "requires": { - "is-buffer": "1.1.6" - } - } + "kind-of": "6.0.2" } }, "is-data-descriptor": { - "version": "0.1.4", - "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-0.1.4.tgz", - "integrity": "sha1-C17mSDiOLIYCgueT8YVv7D8wG1Y=", + "version": "1.0.0", + "resolved": "https://registry.npmjs.org/is-data-descriptor/-/is-data-descriptor-1.0.0.tgz", + "integrity": "sha512-jbRXy1FmtAoCjQkVmIVYwuuqDFUbaOeDjmed1tOGPrsMhtJA4rD9tkgA0F1qJ3gRFRXcHYVkdeaP50Q5rE/jLQ==", "requires": { - "kind-of": "3.2.2" - }, - "dependencies": { - "kind-of": { - "version": "3.2.2", - "resolved": "https://registry.npmjs.org/kind-of/-/kind-of-3.2.2.tgz", - "integrity": "sha1-MeohpzS6ubuw8yRm2JOupR5KPGQ=", - "requires": { - "is-buffer": "1.1.6" - } - } + "kind-of": "6.0.2" + } + }, + "is-descriptor": { + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/is-descriptor/-/is-descriptor-1.0.2.tgz", + "integrity": "sha512-2eis5WqQGV7peooDyLmNEPUrps9+SXX5c9pL3xEB+4e9HnGuDa7mB7kHxHw4CbqS9k1T2hOH3miL8n8WtiYVtg==", + "requires": { + "is-accessor-descriptor": "1.0.0", + "is-data-descriptor": "1.0.0", + "kind-of": "6.0.2" } }, "is-number": { @@ -13629,7 +13516,7 @@ "requires": { "arr-diff": "4.0.0", "array-unique": "0.3.2", - "braces": "2.3.1", + "braces": "2.3.2", "define-property": "2.0.2", "extend-shallow": "3.0.2", "extglob": "2.0.4", @@ -13695,7 +13582,7 @@ "integrity": "sha512-y7ANxCWmMW8xLOaiopiRDlyjQ9ajKRENBH+2wjntIbk3A6ZR1+BLQttkmSHMY7Arl+AAZFwJ10grg2T6f1WI8A==", "requires": { "ast-types": "0.10.1", - "core-js": "2.5.4", + "core-js": "2.5.5", "esprima": "4.0.0", "private": "0.1.8", "source-map": "0.6.1" @@ -13905,7 +13792,7 @@ "path-is-absolute": "1.0.1", "range-parser": "1.2.0", "url-join": "4.0.0", - "webpack-log": "1.1.2" + "webpack-log": "1.2.0" }, "dependencies": { "mime": { @@ -13946,7 +13833,7 @@ "strip-ansi": "3.0.1", "supports-color": "5.3.0", "webpack-dev-middleware": "3.0.1", - "webpack-log": "1.1.2", + "webpack-log": "1.2.0", "yargs": "9.0.1" }, "dependencies": { @@ -14006,13 +13893,13 @@ } }, "webpack-log": { - "version": "1.1.2", - "resolved": "https://registry.npmjs.org/webpack-log/-/webpack-log-1.1.2.tgz", - "integrity": "sha512-B53SD4N4BHpZdUwZcj4st2QT7gVfqZtqHDruC1N+K2sciq0Rt/3F1Dx6RlylVkcrToMLTaiaeT48k9Lq4iDVDA==", + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/webpack-log/-/webpack-log-1.2.0.tgz", + "integrity": "sha512-U9AnICnu50HXtiqiDxuli5gLB5PGBo7VvcHx36jRZHwK4vzOYLbImqT4lwWwoMHdQWwEKw736fCHEekokTEKHA==", "requires": { "chalk": "2.3.2", "log-symbols": "2.2.0", - "loglevelnext": "1.0.3", + "loglevelnext": "1.0.4", "uuid": "3.2.1" }, "dependencies": { From 8d3755b9c58eef12c5fc9cabfc0b1c05f6db716e Mon Sep 17 00:00:00 2001 From: Semyon Boikov Date: Tue, 10 Apr 2018 11:37:39 +0300 Subject: [PATCH 0009/1463] IGNITE-7222 Added ZooKeeper discovery SPI --- .../JdbcAbstractDmlStatementSelfTest.java | 6 +- .../CommunicationFailureContext.java | 62 + .../CommunicationFailureResolver.java | 28 + .../DefaultCommunicationFailureResolver.java | 305 ++ .../configuration/IgniteConfiguration.java | 22 + .../org/apache/ignite/internal/GridTopic.java | 3 + .../apache/ignite/internal/IgniteKernal.java | 5 +- .../apache/ignite/internal/IgnitionEx.java | 3 + .../internal/managers/GridManagerAdapter.java | 8 + .../managers/communication/GridIoManager.java | 4 +- .../communication/GridIoMessageFactory.java | 12 + .../discovery/CustomMessageWrapper.java | 5 + .../managers/discovery/DiscoCache.java | 8 + .../discovery/DiscoveryCustomMessage.java | 10 +- .../DiscoveryMessageResultsCollector.java | 222 + .../discovery/GridDiscoveryManager.java | 128 +- .../managers/discovery/IgniteClusterNode.java | 69 + .../discovery/IgniteDiscoverySpi.java | 67 + .../IgniteDiscoverySpiInternalListener.java | 42 + .../authentication/UserAcceptedMessage.java | 5 + .../authentication/UserProposedMessage.java | 5 + .../cache/CacheAffinityChangeMessage.java | 5 + .../cache/CacheAffinitySharedManager.java | 26 +- .../CacheStatisticsModeChangeMessage.java | 5 + .../ClientCacheChangeDiscoveryMessage.java | 5 + ...lientCacheChangeDummyDiscoveryMessage.java | 5 + .../cache/DynamicCacheChangeBatch.java | 5 + .../processors/cache/GridCacheAdapter.java | 3 +- .../GridCachePartitionExchangeManager.java | 16 +- .../processors/cache/GridCacheProcessor.java | 4 +- .../processors/cache/GridCacheUtils.java | 6 +- .../cache/WalStateFinishMessage.java | 5 + .../cache/WalStateProposeMessage.java | 5 + .../cache/binary/BinaryMetadataTransport.java | 24 +- .../binary/MetadataUpdateAcceptedMessage.java | 5 + .../binary/MetadataUpdateProposedMessage.java | 5 + .../dht/GridClientPartitionTopology.java | 39 + .../distributed/dht/GridDhtCacheAdapter.java | 4 + .../dht/GridDhtPartitionTopology.java | 6 + .../dht/GridDhtPartitionTopologyImpl.java | 39 + .../GridDhtPartitionsExchangeFuture.java | 26 +- .../ChangeGlobalStateFinishMessage.java | 5 + .../cluster/ChangeGlobalStateMessage.java | 5 + .../cluster/ClusterMetricsUpdateMessage.java | 158 + .../cluster/ClusterNodeMetrics.java | 62 + .../processors/cluster/ClusterProcessor.java | 249 +- .../continuous/AbstractContinuousMessage.java | 5 + .../continuous/ContinuousRoutineInfo.java | 100 + .../ContinuousRoutineStartResultMessage.java | 206 + ...ContinuousRoutinesCommonDiscoveryData.java | 45 + .../continuous/ContinuousRoutinesInfo.java | 132 + ...nuousRoutinesJoiningNodeDiscoveryData.java | 45 + .../continuous/GridContinuousProcessor.java | 862 ++- .../continuous/StartRequestDataV2.java | 164 + .../StartRoutineDiscoveryMessageV2.java | 77 + .../StopRoutineAckDiscoveryMessage.java | 5 + .../datastreamer/DataStreamerImpl.java | 27 +- .../marshaller/MappingAcceptedMessage.java | 5 + .../marshaller/MappingProposedMessage.java | 5 + .../message/SchemaFinishDiscoveryMessage.java | 5 + .../SchemaProposeDiscoveryMessage.java | 5 + .../internal/util/nio/GridNioServer.java | 18 +- .../apache/ignite/spi/IgniteSpiAdapter.java | 10 + .../apache/ignite/spi/IgniteSpiContext.java | 11 + .../tcp/TcpCommunicationSpi.java | 190 +- .../tcp/internal/ConnectionKey.java | 117 + ...TcpCommunicationConnectionCheckFuture.java | 519 ++ ...ommunicationNodeConnectionCheckFuture.java | 30 + .../discovery/DiscoverySpiCustomMessage.java | 15 +- ...scoverySpiMutableCustomMessageSupport.java | 40 + .../spi/discovery/tcp/TcpDiscoverySpi.java | 55 +- .../tcp/internal/TcpDiscoveryNode.java | 32 +- .../resources/META-INF/classnames.properties | 2 + ...ctionExcludeNeighborsAbstractSelfTest.java | 8 +- .../failure/FailureHandlerTriggeredTest.java | 4 + .../internal/ClusterGroupHostsSelfTest.java | 3 + .../ignite/internal/ClusterGroupSelfTest.java | 2 + .../ClusterNodeMetricsUpdateTest.java | 173 + .../internal/DiscoverySpiTestListener.java | 162 + .../internal/GridDiscoverySelfTest.java | 14 +- .../GridJobMasterLeaveAwareSelfTest.java | 2 + .../internal/GridJobStealingSelfTest.java | 2 + .../internal/GridSameVmStartupSelfTest.java | 19 +- .../apache/ignite/internal/GridSelfTest.java | 2 + .../IgniteClientReconnectAbstractTest.java | 53 +- ...IgniteClientReconnectApiExceptionTest.java | 21 +- .../IgniteClientReconnectAtomicsTest.java | 30 +- .../IgniteClientReconnectCacheTest.java | 49 +- .../IgniteClientReconnectCollectionsTest.java | 14 +- .../IgniteClientReconnectComputeTest.java | 6 +- ...lientReconnectContinuousProcessorTest.java | 13 +- ...niteClientReconnectDiscoveryStateTest.java | 22 +- ...teClientReconnectFailoverAbstractTest.java | 12 +- .../IgniteClientReconnectServicesTest.java | 8 +- .../IgniteClientReconnectStopTest.java | 12 +- .../IgniteClientReconnectStreamerTest.java | 4 +- .../internal/IgniteClientRejoinTest.java | 3 + ...ridDiscoveryManagerAliveCacheSelfTest.java | 16 +- ...GridAffinityProcessorAbstractSelfTest.java | 4 +- .../CacheMetricsForClusterGroupSelfTest.java | 12 +- .../cache/GridCacheAbstractSelfTest.java | 2 + .../IgniteCacheNearLockValueSelfTest.java | 4 +- .../IgniteCacheP2pUnmarshallingErrorTest.java | 11 + .../IgniteClusterActivateDeactivateTest.java | 65 + .../IgniteDaemonNodeMarshallerCacheTest.java | 3 +- .../binary/BinaryMetadataUpdatesFlowTest.java | 12 +- ...NodeBinaryObjectMetadataMultinodeTest.java | 2 +- .../GridCacheQueueClientDisconnectTest.java | 10 + ...gniteClientDataStructuresAbstractTest.java | 3 +- .../CacheLateAffinityAssignmentTest.java | 127 +- .../GridCacheNodeFailureAbstractTest.java | 5 +- .../IgniteCache150ClientsTest.java | 2 + .../IgniteCacheManyClientsTest.java | 44 +- .../IgniteOptimisticTxSuspendResumeTest.java | 2 + ...dCacheDhtPreloadMultiThreadedSelfTest.java | 4 + .../dht/GridCacheDhtPreloadSelfTest.java | 2 + .../dht/TxRecoveryStoreEnabledTest.java | 15 +- ...tionedExplicitLockNodeFailureSelfTest.java | 3 +- .../ClientReconnectContinuousQueryTest.java | 19 +- ...emoteFilterMissingInClassPathSelfTest.java | 23 +- ...cheContinuousQueryClientReconnectTest.java | 3 + .../CacheVersionedEntryAbstractTest.java | 33 +- .../continuous/GridEventConsumeSelfTest.java | 34 +- .../ClosureServiceClientsNodesTest.java | 19 +- .../internal/util/GridTestClockTimer.java | 9 + .../GridMarshallerMappingConsistencyTest.java | 4 + .../messaging/GridMessagingSelfTest.java | 126 +- .../GridTcpCommunicationSpiAbstractTest.java | 71 + .../FilterDataForClientNodeDiscoveryTest.java | 5 + .../testframework/GridSpiTestContext.java | 10 + .../config/GridTestProperties.java | 9 + .../junits/GridAbstractTest.java | 129 +- .../junits/multijvm/IgniteNodeRunner.java | 2 + .../IgniteComputeGridTestSuite.java | 2 + ...teCacheDistributedQueryCancelSelfTest.java | 2 +- .../DynamicIndexAbstractBasicSelfTest.java | 5 +- .../GridJtaTransactionManagerSelfTest.java | 21 +- ...ridPartitionedCacheJtaFactorySelfTest.java | 19 +- .../org/apache/ignite/spark/IgniteRDD.scala | 9 +- .../ignite/internal/GridFactorySelfTest.java | 3 +- .../p2p/GridP2PUserVersionChangeSelfTest.java | 5 +- modules/yardstick/pom-standalone.xml | 6 + modules/yardstick/pom.xml | 6 + modules/zookeeper/pom.xml | 40 + .../discovery/zk/ZookeeperDiscoverySpi.java | 557 ++ .../zk/internal/ZkAbstractCallabck.java | 83 + .../internal/ZkAbstractChildrenCallback.java | 61 + .../zk/internal/ZkAbstractWatcher.java | 55 + .../zk/internal/ZkAliveNodeData.java | 40 + .../zk/internal/ZkBulkJoinContext.java | 50 + .../discovery/zk/internal/ZkClusterNodes.java | 103 + .../ZkCommunicationErrorNodeState.java | 46 + .../ZkCommunicationErrorProcessFuture.java | 411 ++ ...ommunicationErrorResolveFinishMessage.java | 69 + .../ZkCommunicationErrorResolveResult.java | 45 + ...CommunicationErrorResolveStartMessage.java | 61 + .../ZkCommunicationFailureContext.java | 188 + .../internal/ZkDiscoveryCustomEventData.java | 89 + .../zk/internal/ZkDiscoveryEventData.java | 165 + .../zk/internal/ZkDiscoveryEventsData.java | 121 + .../ZkDiscoveryNodeFailEventData.java | 55 + .../ZkDiscoveryNodeJoinEventData.java | 60 + .../ZkDistributedCollectDataFuture.java | 250 + .../zk/internal/ZkForceNodeFailMessage.java | 65 + .../discovery/zk/internal/ZkIgnitePaths.java | 307 ++ .../internal/ZkInternalJoinErrorMessage.java | 44 + .../zk/internal/ZkInternalMessage.java | 27 + .../zk/internal/ZkJoinEventDataForJoined.java | 83 + .../zk/internal/ZkJoinedNodeEvtData.java | 79 + .../zk/internal/ZkJoiningNodeData.java | 87 + .../zk/internal/ZkNoServersMessage.java | 50 + .../zk/internal/ZkNodeValidateResult.java | 43 + .../spi/discovery/zk/internal/ZkRunnable.java | 51 + .../discovery/zk/internal/ZkRuntimeState.java | 135 + .../zk/internal/ZkTimeoutObject.java | 54 + .../zk/internal/ZookeeperClient.java | 1219 +++++ .../ZookeeperClientFailedException.java | 40 + .../zk/internal/ZookeeperClusterNode.java | 362 ++ .../zk/internal/ZookeeperDiscoveryImpl.java | 4464 +++++++++++++++ .../java/org/apache/ZookeeperNodeStart.java | 46 + ...cheEntryListenerWithZkDiscoAtomicTest.java | 32 + ...ookeeperDiscoverySpiAbstractTestSuite.java | 118 + .../zk/ZookeeperDiscoverySpiTestSuite1.java | 44 + .../zk/ZookeeperDiscoverySpiTestSuite2.java | 94 + ...okeeperDiscoverySuitePreprocessorTest.java | 101 + .../zk/internal/ZookeeperClientTest.java | 495 ++ ...eeperDiscoverySpiSaslAuthAbstractTest.java | 247 + ...okeeperDiscoverySpiSaslFailedAuthTest.java | 44 + ...perDiscoverySpiSaslSuccessfulAuthTest.java | 48 + .../internal/ZookeeperDiscoverySpiTest.java | 4847 +++++++++++++++++ .../zookeeper/ZkTestClientCnxnSocketNIO.java | 137 + 191 files changed, 21158 insertions(+), 777 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureContext.java create mode 100644 modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureResolver.java create mode 100644 modules/core/src/main/java/org/apache/ignite/configuration/DefaultCommunicationFailureResolver.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageResultsCollector.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteClusterNode.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpi.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpiInternalListener.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterMetricsUpdateMessage.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterNodeMetrics.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineInfo.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineStartResultMessage.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesCommonDiscoveryData.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesInfo.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesJoiningNodeDiscoveryData.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestDataV2.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessageV2.java create mode 100644 modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/ConnectionKey.java create mode 100644 modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationConnectionCheckFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationNodeConnectionCheckFuture.java create mode 100644 modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiMutableCustomMessageSupport.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsUpdateTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/DiscoverySpiTestListener.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractCallabck.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractChildrenCallback.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractWatcher.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAliveNodeData.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkBulkJoinContext.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkClusterNodes.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorNodeState.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorProcessFuture.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveFinishMessage.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveResult.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveStartMessage.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationFailureContext.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryCustomEventData.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventData.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventsData.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeFailEventData.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeJoinEventData.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDistributedCollectDataFuture.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkForceNodeFailMessage.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkIgnitePaths.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalJoinErrorMessage.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalMessage.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinEventDataForJoined.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinedNodeEvtData.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoiningNodeData.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNoServersMessage.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNodeValidateResult.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRunnable.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRuntimeState.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkTimeoutObject.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClient.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientFailedException.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java create mode 100644 modules/zookeeper/src/test/java/org/apache/ZookeeperNodeStart.java create mode 100644 modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/IgniteCacheEntryListenerWithZkDiscoAtomicTest.java create mode 100644 modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiAbstractTestSuite.java create mode 100644 modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite1.java create mode 100644 modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite2.java create mode 100644 modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySuitePreprocessorTest.java create mode 100644 modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java create mode 100644 modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslAuthAbstractTest.java create mode 100644 modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslFailedAuthTest.java create mode 100644 modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslSuccessfulAuthTest.java create mode 100644 modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiTest.java create mode 100644 modules/zookeeper/src/test/java/org/apache/zookeeper/ZkTestClientCnxnSocketNIO.java diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java index f4c0ca3464ed6..0a055a9c8679f 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java @@ -138,8 +138,10 @@ protected String getCfgUrl() { @Override protected void afterTest() throws Exception { ((IgniteEx)ignite(0)).context().cache().dynamicDestroyCache(DEFAULT_CACHE_NAME, true, true, false); - conn.close(); - assertTrue(conn.isClosed()); + if (conn != null) { + conn.close(); + assertTrue(conn.isClosed()); + } cleanUpWorkingDir(); } diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureContext.java b/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureContext.java new file mode 100644 index 0000000000000..a32d38c65be60 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureContext.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.configuration; + +import java.util.List; +import java.util.Map; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.spi.communication.CommunicationSpi; + +/** + * Communication Failure Context. + */ +public interface CommunicationFailureContext { + /** + * @return Current topology snapshot. + */ + public List topologySnapshot(); + + /** + * @param node1 First node. + * @param node2 Second node. + * @return {@code True} if {@link CommunicationSpi} is able to establish connection from first node to second node. + */ + public boolean connectionAvailable(ClusterNode node1, ClusterNode node2); + + /** + * @return Currently started caches. + */ + public Map> startedCaches(); + + /** + * @param cacheName Cache name. + * @return Cache partitions affinity assignment. + */ + public List> cacheAffinity(String cacheName); + + /** + * @param cacheName Cache name. + * @return Cache partitions owners. + */ + public List> cachePartitionOwners(String cacheName); + + /** + * @param node Node to kill. + */ + public void killNode(ClusterNode node); +} diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureResolver.java b/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureResolver.java new file mode 100644 index 0000000000000..a4d92f33c73f0 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/configuration/CommunicationFailureResolver.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.configuration; + +/** + * Communication Failure Resolver. + */ +public interface CommunicationFailureResolver { + /** + * @param ctx Context. + */ + public void resolve(CommunicationFailureContext ctx); +} diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/DefaultCommunicationFailureResolver.java b/modules/core/src/main/java/org/apache/ignite/configuration/DefaultCommunicationFailureResolver.java new file mode 100644 index 0000000000000..a4c6da9e9986b --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/configuration/DefaultCommunicationFailureResolver.java @@ -0,0 +1,305 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.configuration; + +import java.util.BitSet; +import java.util.List; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.resources.LoggerResource; + +/** + * Default Communication Failure Resolver. + */ +public class DefaultCommunicationFailureResolver implements CommunicationFailureResolver { + /** */ + @LoggerResource + private IgniteLogger log; + + /** {@inheritDoc} */ + @Override public void resolve(CommunicationFailureContext ctx) { + ClusterGraph graph = new ClusterGraph(log, ctx); + + ClusterSearch cluster = graph.findLargestIndependentCluster(); + + List nodes = ctx.topologySnapshot(); + + assert nodes.size() > 0; + assert cluster != null; + + if (graph.checkFullyConnected(cluster.nodesBitSet)) { + assert cluster.nodeCnt <= nodes.size(); + + if (cluster.nodeCnt < nodes.size()) { + if (log.isInfoEnabled()) { + log.info("Communication problem resolver found fully connected independent cluster [" + + "clusterSrvCnt=" + cluster.srvCnt + + ", clusterTotalNodes=" + cluster.nodeCnt + + ", totalAliveNodes=" + nodes.size() + "]"); + } + + for (int i = 0; i < nodes.size(); i++) { + if (!cluster.nodesBitSet.get(i)) + ctx.killNode(nodes.get(i)); + } + } + else + U.warn(log, "All alive nodes are fully connected, this should be resolved automatically."); + } + else { + if (log.isInfoEnabled()) { + log.info("Communication problem resolver failed to find fully connected independent cluster."); + } + } + } + + /** + * @param cluster Cluster nodes mask. + * @param nodes Nodes. + * @param limit IDs limit. + * @return Cluster node IDs string. + */ + private static String clusterNodeIds(BitSet cluster, List nodes, int limit) { + int startIdx = 0; + + StringBuilder builder = new StringBuilder(); + + int cnt = 0; + + for (;;) { + int idx = cluster.nextSetBit(startIdx); + + if (idx == -1) + break; + + startIdx = idx + 1; + + if (builder.length() == 0) { + builder.append('['); + } + else + builder.append(", "); + + builder.append(nodes.get(idx).id()); + + if (cnt++ > limit) + builder.append(", ..."); + } + + builder.append(']'); + + return builder.toString(); + } + + /** + * + */ + private static class ClusterSearch { + /** */ + int srvCnt; + + /** */ + int nodeCnt; + + /** */ + final BitSet nodesBitSet; + + /** + * @param nodes Total nodes. + */ + ClusterSearch(int nodes) { + nodesBitSet = new BitSet(nodes); + } + } + + /** + * + */ + private static class ClusterGraph { + /** */ + private final static int WORD_IDX_SHIFT = 6; + + /** */ + private final IgniteLogger log; + + /** */ + private final int nodeCnt; + + /** */ + private final long[] visitBitSet; + + /** */ + private final CommunicationFailureContext ctx; + + /** */ + private final List nodes; + + /** + * @param log Logger. + * @param ctx Context. + */ + ClusterGraph(IgniteLogger log, CommunicationFailureContext ctx) { + this.log = log; + this.ctx = ctx; + + nodes = ctx.topologySnapshot(); + + nodeCnt = nodes.size(); + + assert nodeCnt > 0; + + visitBitSet = initBitSet(nodeCnt); + } + + /** + * @param bitIndex Bit index. + * @return Word index containing bit with given index. + */ + private static int wordIndex(int bitIndex) { + return bitIndex >> WORD_IDX_SHIFT; + } + + /** + * @param bitCnt Number of bits. + * @return Bit set words. + */ + static long[] initBitSet(int bitCnt) { + return new long[wordIndex(bitCnt - 1) + 1]; + } + + /** + * @return Cluster nodes bit set. + */ + ClusterSearch findLargestIndependentCluster() { + ClusterSearch maxCluster = null; + + for (int i = 0; i < nodeCnt; i++) { + if (getBit(visitBitSet, i)) + continue; + + ClusterSearch cluster = new ClusterSearch(nodeCnt); + + search(cluster, i); + + if (log.isInfoEnabled()) { + log.info("Communication problem resolver found cluster [srvCnt=" + cluster.srvCnt + + ", totalNodeCnt=" + cluster.nodeCnt + + ", nodeIds=" + clusterNodeIds(cluster.nodesBitSet, nodes, 1000) + "]"); + } + + if (maxCluster == null || cluster.srvCnt > maxCluster.srvCnt) + maxCluster = cluster; + } + + return maxCluster; + } + + /** + * @param cluster Cluster nodes bit set. + * @return {@code True} if all cluster nodes are able to connect to each other. + */ + boolean checkFullyConnected(BitSet cluster) { + int startIdx = 0; + + int clusterNodes = cluster.cardinality(); + + for (;;) { + int idx = cluster.nextSetBit(startIdx); + + if (idx == -1) + break; + + ClusterNode node1 = nodes.get(idx); + + for (int i = 0; i < clusterNodes; i++) { + if (!cluster.get(i) || i == idx) + continue; + + ClusterNode node2 = nodes.get(i); + + if (cluster.get(i) && !ctx.connectionAvailable(node1, node2)) + return false; + } + + startIdx = idx + 1; + } + + return true; + } + + /** + * @param cluster Current cluster bit set. + * @param idx Node index. + */ + void search(ClusterSearch cluster, int idx) { + assert !getBit(visitBitSet, idx); + + setBit(visitBitSet, idx); + + cluster.nodesBitSet.set(idx); + cluster.nodeCnt++; + + ClusterNode node1 = nodes.get(idx); + + if (!CU.clientNode(node1)) + cluster.srvCnt++; + + for (int i = 0; i < nodeCnt; i++) { + if (i == idx || getBit(visitBitSet, i)) + continue; + + ClusterNode node2 = nodes.get(i); + + boolean connected = ctx.connectionAvailable(node1, node2) || + ctx.connectionAvailable(node2, node1); + + if (connected) + search(cluster, i); + } + } + + /** + * @param words Bit set words. + * @param bitIndex Bit index. + */ + static void setBit(long words[], int bitIndex) { + int wordIndex = wordIndex(bitIndex); + + words[wordIndex] |= (1L << bitIndex); + } + + /** + * @param words Bit set words. + * @param bitIndex Bit index. + * @return Bit value. + */ + static boolean getBit(long[] words, int bitIndex) { + int wordIndex = wordIndex(bitIndex); + + return (words[wordIndex] & (1L << bitIndex)) != 0; + } + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(DefaultCommunicationFailureResolver.class, this); + } +} 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 add388045d1e4..cc3ea1006bef1 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java @@ -493,6 +493,9 @@ public class IgniteConfiguration { /** Failure handler. */ private FailureHandler failureHnd; + /** Communication failure resolver */ + private CommunicationFailureResolver commFailureRslvr; + /** * Creates valid grid configuration with all default values. */ @@ -520,6 +523,8 @@ public IgniteConfiguration(IgniteConfiguration cfg) { loadBalancingSpi = cfg.getLoadBalancingSpi(); indexingSpi = cfg.getIndexingSpi(); + commFailureRslvr = cfg.getCommunicationFailureResolver(); + /* * Order alphabetically for maintenance purposes. */ @@ -606,6 +611,23 @@ public IgniteConfiguration(IgniteConfiguration cfg) { authEnabled = cfg.isAuthenticationEnabled(); } + /** + * @return Communication failure resovler. + */ + public CommunicationFailureResolver getCommunicationFailureResolver() { + return commFailureRslvr; + } + + /** + * @param commFailureRslvr Communication failure resovler. + * @return {@code this} instance. + */ + public IgniteConfiguration setCommunicationFailureResolver(CommunicationFailureResolver commFailureRslvr) { + this.commFailureRslvr = commFailureRslvr; + + return this; + } + /** * Gets optional grid name. Returns {@code null} if non-default grid name was not * provided. 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 4932e671376dc..1227e8cf4abd4 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 @@ -120,6 +120,9 @@ public enum GridTopic { /** */ TOPIC_WAL, + /** */ + TOPIC_METRICS, + /** */ TOPIC_AUTH; 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 8bc46fd1db2f6..0b102e5b665f7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -1298,7 +1298,7 @@ private long checkPoolStarvation( ackStart(rtBean); if (!isDaemon()) - ctx.discovery().ackTopology(localNode().order()); + ctx.discovery().ackTopology(ctx.discovery().localJoin().joinTopologyVersion().topologyVersion()); } /** @@ -2623,6 +2623,9 @@ private Iterable lifecycleAwares(IgniteConfiguration cfg) { objs.add(cfg.getGridLogger()); objs.add(cfg.getMBeanServer()); + if (cfg.getCommunicationFailureResolver() != null) + objs.add(cfg.getCommunicationFailureResolver()); + return objs; } 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 8073faab8c8fa..3abc7111c43a9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java @@ -75,6 +75,7 @@ import org.apache.ignite.failure.FailureType; import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.managers.communication.GridIoPolicy; +import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor; import org.apache.ignite.internal.processors.igfs.IgfsThreadFactory; import org.apache.ignite.internal.processors.igfs.IgfsUtils; @@ -2243,6 +2244,8 @@ private IgniteConfiguration initializeConfiguration(IgniteConfiguration cfg) initializeDefaultSpi(myCfg); + GridDiscoveryManager.initCommunicationErrorResolveConfiguration(myCfg); + initializeDefaultCacheConfiguration(myCfg); ExecutorConfiguration[] execCfgs = myCfg.getExecutorConfiguration(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java index 74f5a102d2f19..b0756cfe80801 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java @@ -618,6 +618,14 @@ protected final String stopInfo() { return ctx.nodeAttributes(); } + @Override public boolean communicationFailureResolveSupported() { + return ctx.discovery().communicationErrorResolveSupported(); + } + + @Override public void resolveCommunicationFailure(ClusterNode node, Exception err) { + ctx.discovery().resolveCommunicationError(node, err); + } + /** * @param e Exception to handle. * @return GridSpiException Converted exception. 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 d5cdd2dd7fc53..8d9a70034aeb3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -298,9 +298,9 @@ public void resetMetrics() { @Override public MessageReader reader(UUID rmtNodeId, MessageFactory msgFactory) throws IgniteCheckedException { - assert rmtNodeId != null; - return new DirectMessageReader(msgFactory, U.directProtocolVersion(ctx, rmtNodeId)); + return new DirectMessageReader(msgFactory, + rmtNodeId != null ? U.directProtocolVersion(ctx, rmtNodeId) : GridIoManager.DIRECT_PROTO_VER); } }; } 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 a0fc2f8088287..5616fd035d445 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java @@ -123,6 +123,8 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheRawVersionedEntry; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionEx; +import org.apache.ignite.internal.processors.cluster.ClusterMetricsUpdateMessage; +import org.apache.ignite.internal.processors.continuous.ContinuousRoutineStartResultMessage; import org.apache.ignite.internal.processors.continuous.GridContinuousMessage; import org.apache.ignite.internal.processors.datastreamer.DataStreamerEntry; import org.apache.ignite.internal.processors.datastreamer.DataStreamerRequest; @@ -909,6 +911,16 @@ public GridIoMessageFactory(MessageFactory[] ext) { break; + case 133: + msg = new ClusterMetricsUpdateMessage(); + + break; + + case 134: + msg = new ContinuousRoutineStartResultMessage(); + + break; + // [-3..119] [124..129] [-23..-27] [-36..-55]- this // [120..123] - DR // [-4..-22, -30..-35] - SQL diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomMessageWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomMessageWrapper.java index 426888631729d..4b6b7a2cfabf7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomMessageWrapper.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomMessageWrapper.java @@ -49,6 +49,11 @@ public class CustomMessageWrapper implements DiscoverySpiCustomMessage { return delegate.isMutable(); } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return delegate.stopProcess(); + } + /** * @return Delegate. */ 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 c21698f4b71ed..fef44fad30bb7 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 @@ -311,6 +311,14 @@ public boolean baselineNode(ClusterNode node) { return null; } + /** + * @param nodeId Node ID. + * @return {@code True} if node is in alives list. + */ + public boolean alive(UUID nodeId) { + return alives.contains(nodeId); + } + /** * Gets all nodes that have cache with given name. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java index c708c6247b2c2..6ed2096faf386 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java @@ -20,6 +20,7 @@ import java.io.Serializable; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddFinishedMessage; import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddedMessage; import org.jetbrains.annotations.Nullable; @@ -87,10 +88,17 @@ public interface DiscoveryCustomMessage extends Serializable { @Nullable public DiscoveryCustomMessage ackMessage(); /** - * @return {@code true} if message can be modified during listener notification. Changes will be send to next nodes. + * @return {@code True} if message can be modified during listener notification. Changes will be sent to next nodes. */ public boolean isMutable(); + /** + * See {@link DiscoverySpiCustomMessage#stopProcess()}. + * + * @return {@code True} if message should not be sent to others nodes after it was processed on coordinator. + */ + public boolean stopProcess(); + /** * Creates new discovery cache if message caused topology version change. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageResultsCollector.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageResultsCollector.java new file mode 100644 index 0000000000000..43be952ea9137 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageResultsCollector.java @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.managers.discovery; + +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public abstract class DiscoveryMessageResultsCollector { + /** */ + private final Map> rcvd = new HashMap<>(); + + /** */ + private int leftMsgs; + + /** */ + protected DiscoCache discoCache; + + /** */ + protected final GridKernalContext ctx; + + /** + * @param ctx Context. + */ + protected DiscoveryMessageResultsCollector(GridKernalContext ctx) { + this.ctx = ctx; + } + + /** + * @param rcvd Received messages. + * @return Result. + */ + protected abstract R createResult(Map> rcvd); + + /** + * @param r Result. + */ + protected abstract void onResultsCollected(R r); + + /** + * @param discoCache Discovery state when discovery message was received. + * @param node Node. + * @return {@code True} if need wait for result from given node. + */ + protected abstract boolean waitForNode(DiscoCache discoCache, ClusterNode node); + + /** + * @param discoCache Discovery state. + */ + public final void init(DiscoCache discoCache) { + assert discoCache != null; + + R res = null; + + synchronized (this) { + assert this.discoCache == null; + assert leftMsgs == 0 : leftMsgs; + + this.discoCache = discoCache; + + for (ClusterNode node : discoCache.allNodes()) { + if (ctx.discovery().alive(node) && waitForNode(discoCache, node) && !rcvd.containsKey(node.id())) { + rcvd.put(node.id(), new NodeMessage<>((M)null)); + + leftMsgs++; + } + } + + if (leftMsgs == 0) + res = createResult(rcvd); + } + + if (res != null) + onResultsCollected(res); + } + + /** + * @param nodeId Node ID. + * @param msg Message. + */ + public final void onMessage(UUID nodeId, M msg) { + R res = null; + + synchronized (this) { + if (allReceived()) + return; + + NodeMessage expMsg = rcvd.get(nodeId); + + if (expMsg == null) + rcvd.put(nodeId, new NodeMessage<>(msg)); + else if (expMsg.set(msg)) { + assert leftMsgs > 0; + + leftMsgs--; + + if (allReceived()) + res = createResult(rcvd); + } + } + + if (res != null) + onResultsCollected(res); + } + + /** + * @param nodeId Failed node ID. + */ + public final void onNodeFail(UUID nodeId) { + R res = null; + + synchronized (this) { + if (allReceived()) + return; + + NodeMessage expMsg = rcvd.get(nodeId); + + if (expMsg != null && expMsg.onNodeFailed()) { + assert leftMsgs > 0 : leftMsgs; + + leftMsgs--; + + if (allReceived()) + res = createResult(rcvd); + } + } + + if (res != null) + onResultsCollected(res); + } + + /** + * @return {@code True} if expected messages are initialized and all message are received. + */ + private boolean allReceived() { + return discoCache != null && leftMsgs == 0; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(DiscoveryMessageResultsCollector.class, this); + } + + /** + * + */ + protected static class NodeMessage { + /** */ + boolean nodeFailed; + + /** */ + M msg; + + /** + * @param msg Message. + */ + NodeMessage(M msg) { + this.msg = msg; + } + + /** + * @return Message or {@code null} if node failed. + */ + @Nullable public M message() { + return msg; + } + + /** + * @return {@code True} if node result was not set before. + */ + boolean onNodeFailed() { + if (nodeFailed || msg != null) + return false; + + nodeFailed = true; + + return true; + } + + /** + * @param msg Received message. + * @return {@code True} if node result was not set before. + */ + boolean set(M msg) { + assert msg != null; + + if (this.msg != null) + return false; + + this.msg = msg; + + return !nodeFailed; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(NodeMessage.class, this); + } + } +} 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 2e814d45aeeae..4c5690e919d34 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java @@ -54,8 +54,11 @@ import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.CommunicationFailureResolver; import org.apache.ignite.configuration.DataRegionConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.DefaultCommunicationFailureResolver; +import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.events.Event; import org.apache.ignite.failure.FailureContext; @@ -112,6 +115,8 @@ import org.apache.ignite.plugin.security.SecurityCredentials; import org.apache.ignite.plugin.segmentation.SegmentationPolicy; import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.communication.CommunicationSpi; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; import org.apache.ignite.spi.discovery.DiscoveryDataBag; import org.apache.ignite.spi.discovery.DiscoveryDataBag.JoiningNodeDiscoveryData; import org.apache.ignite.spi.discovery.DiscoveryMetricsProvider; @@ -120,10 +125,10 @@ import org.apache.ignite.spi.discovery.DiscoverySpiDataExchange; import org.apache.ignite.spi.discovery.DiscoverySpiHistorySupport; import org.apache.ignite.spi.discovery.DiscoverySpiListener; +import org.apache.ignite.spi.discovery.DiscoverySpiMutableCustomMessageSupport; import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator; import org.apache.ignite.spi.discovery.DiscoverySpiOrderSupport; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; -import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; import org.apache.ignite.thread.IgniteThread; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -478,7 +483,7 @@ private void updateClientNodes(UUID leftNodeId) { /** {@inheritDoc} */ @Override protected void onKernalStart0() throws IgniteCheckedException { - if (Boolean.TRUE.equals(ctx.config().isClientMode()) && !getSpi().isClientMode()) + if ((getSpi() instanceof TcpDiscoverySpi) && Boolean.TRUE.equals(ctx.config().isClientMode()) && !getSpi().isClientMode()) ctx.performance().add("Enable client mode for TcpDiscoverySpi " + "(set TcpDiscoverySpi.forceServerMode to false)"); } @@ -551,6 +556,9 @@ private void updateClientNodes(UUID leftNodeId) { }); } + if (ctx.config().getCommunicationFailureResolver() != null) + ctx.resource().injectGeneric(ctx.config().getCommunicationFailureResolver()); + spi.setListener(new DiscoverySpiListener() { private long gridStartTime; @@ -559,8 +567,8 @@ private void updateClientNodes(UUID leftNodeId) { for (IgniteInClosure lsnr : locNodeInitLsnrs) lsnr.apply(locNode); - if (locNode instanceof TcpDiscoveryNode) { - final TcpDiscoveryNode node = (TcpDiscoveryNode)locNode; + if (locNode instanceof IgniteClusterNode) { + final IgniteClusterNode node = (IgniteClusterNode)locNode; if (consistentId != null) node.setConsistentId(consistentId); @@ -1052,7 +1060,7 @@ private GridLocalMetrics createMetrics() { /** * @return Metrics provider. */ - private DiscoveryMetricsProvider createMetricsProvider() { + public DiscoveryMetricsProvider createMetricsProvider() { return new DiscoveryMetricsProvider() { /** */ private final long startTime = U.currentTimeMillis(); @@ -1679,13 +1687,15 @@ public boolean pingNode(UUID nodeId) throws IgniteClientDisconnectedCheckedExcep return getSpi().pingNode(nodeId); } catch (IgniteException e) { - if (e.hasCause(IgniteClientDisconnectedCheckedException.class)) { + if (e.hasCause(IgniteClientDisconnectedCheckedException.class, IgniteClientDisconnectedException.class)) { IgniteFuture reconnectFut = ctx.cluster().clientReconnectFuture(); throw new IgniteClientDisconnectedCheckedException(reconnectFut, e.getMessage()); } - throw e; + LT.warn(log, "Ping failed with error [node=" + nodeId + ", err=" + e + ']'); + + return true; } finally { busyLock.leaveBusy(); @@ -2025,7 +2035,16 @@ private DiscoCache resolveDiscoCache(int grpId, AffinityTopologyVersion topVer) Map> snapshots = topHist; - return snapshots.get(topVer); + Collection nodes = snapshots.get(topVer); + + if (nodes == null) { + DiscoCache cache = discoCacheHist.get(new AffinityTopologyVersion(topVer, 0)); + + if (cache != null) + nodes = cache.allNodes(); + } + + return nodes; } /** @@ -2157,6 +2176,19 @@ public void clientCacheStartEvent(UUID reqId, } } + /** + * @param discoCache + * @param node + */ + public void metricsUpdateEvent(DiscoCache discoCache, ClusterNode node) { + discoWrk.addEvent(EVT_NODE_METRICS_UPDATED, + discoCache.version(), + node, + discoCache, + discoCache.nodeMap.values(), + null); + } + /** * Gets first grid node start time, see {@link DiscoverySpi#getGridStartTime()}. * @@ -2211,8 +2243,9 @@ public void failNode(UUID nodeId, @Nullable String warning) { public boolean reconnectSupported() { DiscoverySpi spi = getSpi(); - return ctx.discovery().localNode().isClient() && (spi instanceof TcpDiscoverySpi) && - !(((TcpDiscoverySpi) spi).isClientReconnectDisabled()); + return ctx.discovery().localNode().isClient() && + (spi instanceof IgniteDiscoverySpi) && + ((IgniteDiscoverySpi)spi).clientReconnectSupported(); } /** @@ -2225,7 +2258,7 @@ public void reconnect() { DiscoverySpi discoverySpi = getSpi(); - ((TcpDiscoverySpi)discoverySpi).reconnect(); + ((IgniteDiscoverySpi)discoverySpi).clientReconnect(); } /** @@ -2379,6 +2412,76 @@ private void addToMap(Map> cacheMap, String cacheName cacheNodes.add(rich); } + /** + * @param cfg Configuration. + * @throws IgniteCheckedException If configuration is not valid. + */ + public static void initCommunicationErrorResolveConfiguration(IgniteConfiguration cfg) throws IgniteCheckedException { + CommunicationFailureResolver rslvr = cfg.getCommunicationFailureResolver(); + CommunicationSpi commSpi = cfg.getCommunicationSpi(); + DiscoverySpi discoverySpi = cfg.getDiscoverySpi(); + + if (rslvr != null) { + if (!supportsCommunicationErrorResolve(commSpi)) + throw new IgniteCheckedException("CommunicationFailureResolver is configured, but CommunicationSpi does not support communication" + + "problem resolve: " + commSpi.getClass().getName()); + + if (!supportsCommunicationErrorResolve(discoverySpi)) + throw new IgniteCheckedException("CommunicationFailureResolver is configured, but DiscoverySpi does not support communication" + + "problem resolve: " + discoverySpi.getClass().getName()); + } + else { + if (supportsCommunicationErrorResolve(commSpi) && supportsCommunicationErrorResolve(discoverySpi)) + cfg.setCommunicationFailureResolver(new DefaultCommunicationFailureResolver()); + } + } + + /** + * @param spi Discovery SPI. + * @return {@code True} if SPI supports communication error resolve. + */ + private static boolean supportsCommunicationErrorResolve(DiscoverySpi spi) { + return spi instanceof IgniteDiscoverySpi && ((IgniteDiscoverySpi)spi).supportsCommunicationFailureResolve(); + } + + /** + * @param spi Discovery SPI. + * @return {@code True} if SPI supports communication error resolve. + */ + private static boolean supportsCommunicationErrorResolve(CommunicationSpi spi) { + return spi instanceof TcpCommunicationSpi; + } + + /** + * @return {@code True} if communication error resolve is supported. + */ + public boolean communicationErrorResolveSupported() { + return ctx.config().getCommunicationFailureResolver() != null; + } + + /** + * @return {@code True} if configured {@link DiscoverySpi} supports mutable custom messages. + */ + public boolean mutableCustomMessages() { + DiscoverySpiMutableCustomMessageSupport ann = U.getAnnotation(ctx.config().getDiscoverySpi().getClass(), + DiscoverySpiMutableCustomMessageSupport.class); + + return ann != null && ann.value(); + } + + /** + * @param node Problem node. + * @param err Error. + */ + public void resolveCommunicationError(ClusterNode node, Exception err) { + DiscoverySpi spi = getSpi(); + + if (!supportsCommunicationErrorResolve(spi) || !supportsCommunicationErrorResolve(ctx.config().getCommunicationSpi())) + throw new UnsupportedOperationException(); + + ((IgniteDiscoverySpi)spi).resolveCommunicationFailure(node, err); + } + /** Worker for network segment checks. */ private class SegmentCheckWorker extends GridWorker { /** */ @@ -2587,6 +2690,9 @@ private void body0() throws InterruptedException { AffinityTopologyVersion topVer = evt.get2(); + if (type == EVT_NODE_METRICS_UPDATED && topVer.compareTo(discoCache.version()) < 0) + return; + ClusterNode node = evt.get3(); boolean isDaemon = node.isDaemon(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteClusterNode.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteClusterNode.java new file mode 100644 index 0000000000000..cbc706afbe3b3 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteClusterNode.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.managers.discovery; + +import java.io.Serializable; +import java.util.Map; +import org.apache.ignite.cache.CacheMetrics; +import org.apache.ignite.cluster.ClusterMetrics; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.IgniteConfiguration; + +/** + * + */ +public interface IgniteClusterNode extends ClusterNode { + /** + * Sets consistent globally unique node ID which survives node restarts. + * + * @param consistentId Consistent globally unique node ID. + */ + public void setConsistentId(Serializable consistentId); + + /** + * Sets node metrics. + * + * @param metrics Node metrics. + */ + public void setMetrics(ClusterMetrics metrics); + + /** + * Gets collections of cache metrics for this node. Note that node cache metrics are constantly updated + * and provide up to date information about caches. + *

    + * Cache metrics are updated with some delay which is directly related to metrics update + * frequency. For example, by default the update will happen every {@code 2} seconds. + * + * @return Runtime metrics snapshots for this node. + */ + public Map cacheMetrics(); + + /** + * Sets node cache metrics. + * + * @param cacheMetrics Cache metrics. + */ + public void setCacheMetrics(Map cacheMetrics); + + /** + * Whether this node is cache client (see {@link IgniteConfiguration#isClientMode()}). + * + * @return {@code True if client}. + */ + public boolean isCacheClient(); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpi.java new file mode 100644 index 0000000000000..9aa5d140bbab5 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpi.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.managers.discovery; + +import java.util.UUID; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.spi.discovery.DiscoverySpi; + +/** + * + */ +public interface IgniteDiscoverySpi extends DiscoverySpi { + /** + * @param nodeId Node ID. + * @return {@code True} if node joining or already joined topology. + */ + public boolean knownNode(UUID nodeId); + + /** + * + * @return {@code True} if SPI supports client reconnect. + */ + public boolean clientReconnectSupported(); + + /** + * + */ + public void clientReconnect(); + + /** + * For TESTING only. + */ + public void simulateNodeFailure(); + + /** + * For TESTING only. + * + * @param lsnr Listener. + */ + public void setInternalListener(IgniteDiscoverySpiInternalListener lsnr); + + /** + * @return {@code True} if supports communication error resolve. + */ + public boolean supportsCommunicationFailureResolve(); + + /** + * @param node Problem node. + * @param err Connection error. + */ + public void resolveCommunicationFailure(ClusterNode node, Exception err); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpiInternalListener.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpiInternalListener.java new file mode 100644 index 0000000000000..24405f8101f0a --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoverySpiInternalListener.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.managers.discovery; + +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.spi.discovery.DiscoverySpi; +import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; + +/** + * For TESTING only. + */ +public interface IgniteDiscoverySpiInternalListener { + /** + * @param locNode Local node. + * @param log Log. + */ + public void beforeJoin(ClusterNode locNode, IgniteLogger log); + + /** + * @param spi SPI instance. + * @param log Logger. + * @param msg Custom message. + * @return {@code False} to cancel event send. + */ + public boolean beforeSendCustomEvent(DiscoverySpi spi, IgniteLogger log, DiscoverySpiCustomMessage msg); +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserAcceptedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserAcceptedMessage.java index ef87a444cb374..2e2aed9dcf0b1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserAcceptedMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserAcceptedMessage.java @@ -71,6 +71,11 @@ public class UserAcceptedMessage implements DiscoveryCustomMessage { return false; } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer, DiscoCache discoCache) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserProposedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserProposedMessage.java index 1a0be8ecaa2c6..19f9e82cf18cb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserProposedMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/UserProposedMessage.java @@ -71,6 +71,11 @@ public class UserProposedMessage implements DiscoveryServerOnlyCustomMessage { return false; } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer, DiscoCache discoCache) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinityChangeMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinityChangeMessage.java index fe1014cf6e010..937a8892bdfae 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinityChangeMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinityChangeMessage.java @@ -155,6 +155,11 @@ public AffinityTopologyVersion topologyVersion() { return false; } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer, DiscoCache discoCache) { 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 6691b132f876f..92b8d3ea87019 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java @@ -1310,20 +1310,17 @@ public void applyAffinityFromFullMessage(final GridDhtPartitionsExchangeFuture f * @param fut Current exchange future. * @param msg Message finish message. * @param resTopVer Result topology version. - * @throws IgniteCheckedException If failed. */ public void onLocalJoin(final GridDhtPartitionsExchangeFuture fut, GridDhtPartitionsFullMessage msg, - final AffinityTopologyVersion resTopVer) - throws IgniteCheckedException { + final AffinityTopologyVersion resTopVer) { final Set affReq = fut.context().groupsAffinityRequestOnJoin(); final Map nodesByOrder = new HashMap<>(); final Map joinedNodeAff = msg.joinedNodeAffinity(); - assert !F.isEmpty(joinedNodeAff) : msg; - assert joinedNodeAff.size() >= affReq.size(); + assert F.isEmpty(affReq) || (!F.isEmpty(joinedNodeAff) && joinedNodeAff.size() >= affReq.size()) : msg; forAllCacheGroups(false, new IgniteInClosureX() { @Override public void applyx(GridAffinityAssignmentCache aff) throws IgniteCheckedException { @@ -1333,7 +1330,7 @@ public void onLocalJoin(final GridDhtPartitionsExchangeFuture fut, assert grp != null; - if (affReq.contains(aff.groupId())) { + if (affReq != null && affReq.contains(aff.groupId())) { assert AffinityTopologyVersion.NONE.equals(aff.lastVersion()); CacheGroupAffinityMessage affMsg = joinedNodeAff.get(aff.groupId()); @@ -2281,6 +2278,23 @@ public Map cacheGroups() { return caches.registeredGrps; } + /** + * @return All registered cache groups. + */ + public Map caches() { + return caches.registeredCaches; + } + + /** + * @param grpId Cache group ID + * @return Cache affinity cache. + */ + @Nullable public GridAffinityAssignmentCache groupAffinity(int grpId) { + CacheGroupHolder grpHolder = grpHolders.get(grpId); + + return grpHolder != null ? grpHolder.affinity() : null; + } + /** * */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java index 40bcfaf12de16..e33256fbed341 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java @@ -100,6 +100,11 @@ public CacheStatisticsModeChangeMessage(UUID reqId, Collection caches, b return false; } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer, DiscoCache discoCache) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDiscoveryMessage.java index e35d80e5c1c6f..ae76c950421db 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDiscoveryMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDiscoveryMessage.java @@ -172,6 +172,11 @@ public void updateTimeoutObject(ClientCacheUpdateTimeout updateTimeoutObj) { return false; } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer, DiscoCache discoCache) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDummyDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDummyDiscoveryMessage.java index 6ed3ecc505f25..4ce0c87bf9898 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDummyDiscoveryMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClientCacheChangeDummyDiscoveryMessage.java @@ -104,6 +104,11 @@ Set cachesToClose() { throw new UnsupportedOperationException(); } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer, DiscoCache discoCache) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java index 83459a5c03589..d85e29b673fe2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java @@ -76,6 +76,11 @@ public DynamicCacheChangeBatch(Collection reqs) { return false; } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer, DiscoCache discoCache) { 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 55357ffb1834f..c2d0f427fc917 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java @@ -82,6 +82,7 @@ import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException; import org.apache.ignite.internal.cluster.IgniteClusterEx; +import org.apache.ignite.internal.managers.discovery.IgniteClusterNode; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.affinity.GridCacheAffinityImpl; import org.apache.ignite.internal.processors.cache.distributed.IgniteExternalizableExpiryPolicy; @@ -3234,7 +3235,7 @@ protected IgniteInternalFuture removeAsync0(final K key, @Nullable fina List metrics = new ArrayList<>(grp.nodes().size()); for (ClusterNode node : grp.nodes()) { - Map nodeCacheMetrics = ((TcpDiscoveryNode)node).cacheMetrics(); + Map nodeCacheMetrics = ((IgniteClusterNode)node).cacheMetrics(); if (nodeCacheMetrics != null) { CacheMetrics e = nodeCacheMetrics.get(context().cacheId()); 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 a30a24ad687ea..77ffce3bdaf06 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java @@ -427,7 +427,7 @@ private void onDiscoveryEvent(DiscoveryEvent evt, DiscoCache cache) { if (evt.type() != EVT_DISCOVERY_CUSTOM_EVT) { assert evt.type() != EVT_NODE_JOINED || n.isLocal() || n.order() > loc.order() : "Node joined with smaller-than-local " + - "order [newOrder=" + n.order() + ", locOrder=" + loc.order() + ']'; + "order [newOrder=" + n.order() + ", locOrder=" + loc.order() + ", evt=" + evt + ']'; exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt); @@ -570,12 +570,6 @@ public void onKernalStart(boolean active, boolean reconnect) throws IgniteChecke for (ClusterNode n : cctx.discovery().remoteNodes()) cctx.versions().onReceived(n.id(), n.metrics().getLastDataVersion()); - ClusterNode loc = cctx.localNode(); - - long startTime = loc.metrics().getStartTime(); - - assert startTime > 0; - DiscoveryLocalJoinData locJoin = cctx.discovery().localJoin(); GridDhtPartitionsExchangeFuture fut = null; @@ -756,6 +750,14 @@ public Object interruptLock() { return interruptLock; } + /** + * @param grpId Cache group ID. + * @return Topology. + */ + @Nullable public GridDhtPartitionTopology clientTopologyIfExists(int grpId) { + return clientTops.get(grpId); + } + /** * @param grpId Cache group ID. * @param discoCache Discovery data cache. 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 a3f7c9430ecd2..7edac7374fbe9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -3363,7 +3363,7 @@ private IgniteNodeValidationResult validateRestartingCaches(ClusterNode node) { * @return Validation result or {@code null} in case of success. */ @Nullable private IgniteNodeValidationResult validateHashIdResolvers(ClusterNode node) { - if (!node.isClient()) { + if (!CU.clientNode(node)) { for (DynamicCacheDescriptor desc : cacheDescriptors().values()) { CacheConfiguration cfg = desc.cacheConfiguration(); @@ -3372,7 +3372,7 @@ private IgniteNodeValidationResult validateRestartingCaches(ClusterNode node) { Object nodeHashObj = aff.resolveNodeHash(node); - for (ClusterNode topNode : ctx.discovery().allNodes()) { + for (ClusterNode topNode : ctx.discovery().aliveServerNodes()) { Object topNodeHashObj = aff.resolveNodeHash(topNode); if (nodeHashObj.hashCode() == topNodeHashObj.hashCode()) { 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 09a96d368528c..6d4c1f21cb273 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -65,6 +65,7 @@ import org.apache.ignite.internal.cluster.ClusterGroupEmptyCheckedException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException; +import org.apache.ignite.internal.managers.discovery.IgniteClusterNode; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockCancelledException; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter; @@ -97,7 +98,6 @@ import org.apache.ignite.lang.IgniteReducer; import org.apache.ignite.lifecycle.LifecycleAware; import org.apache.ignite.plugin.CachePluginConfiguration; -import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; import org.apache.ignite.transactions.Transaction; import org.apache.ignite.transactions.TransactionConcurrency; import org.apache.ignite.transactions.TransactionIsolation; @@ -1347,8 +1347,8 @@ public static List cachePluginConfigurat * @return {@code True} if given node is client node (has flag {@link IgniteConfiguration#isClientMode()} set). */ public static boolean clientNode(ClusterNode node) { - if (node instanceof TcpDiscoveryNode) - return ((TcpDiscoveryNode)node).isCacheClient(); + if (node instanceof IgniteClusterNode) + return ((IgniteClusterNode)node).isCacheClient(); else return clientNodeDirect(node); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateFinishMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateFinishMessage.java index 57f25d0e80ca1..4afa403bfbe73 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateFinishMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateFinishMessage.java @@ -66,6 +66,11 @@ public boolean changed() { return errMsg; } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(WalStateFinishMessage.class, this, "super", super.toString()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateProposeMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateProposeMessage.java index 747fd6af0a3af..b9d96fc35a039 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateProposeMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateProposeMessage.java @@ -97,6 +97,11 @@ public void affinityNode(boolean affNode) { this.affNode = affNode; } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(WalStateProposeMessage.class, this, "super", super.toString()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataTransport.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataTransport.java index 9402a3270180c..38450dfec5c78 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataTransport.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataTransport.java @@ -155,22 +155,30 @@ void addBinaryMetadataUpdateListener(BinaryMetadataUpdatedListener lsnr) { * @param metadata Metadata proposed for update. * @return Future to wait for update result on. */ - GridFutureAdapter requestMetadataUpdate(BinaryMetadata metadata) throws IgniteCheckedException { + GridFutureAdapter requestMetadataUpdate(BinaryMetadata metadata) { MetadataUpdateResultFuture resFut = new MetadataUpdateResultFuture(); if (log.isDebugEnabled()) log.debug("Requesting metadata update for " + metadata.typeId() + "; caller thread is blocked on future " + resFut); - synchronized (this) { - unlabeledFutures.add(resFut); + try { + synchronized (this) { + unlabeledFutures.add(resFut); - if (!stopping) - discoMgr.sendCustomEvent(new MetadataUpdateProposedMessage(metadata, ctx.localNodeId())); - else - resFut.onDone(MetadataUpdateResult.createUpdateDisabledResult()); + if (!stopping) + discoMgr.sendCustomEvent(new MetadataUpdateProposedMessage(metadata, ctx.localNodeId())); + else + resFut.onDone(MetadataUpdateResult.createUpdateDisabledResult()); + } + } + catch (Exception e) { + resFut.onDone(MetadataUpdateResult.createUpdateDisabledResult(), e); } + if (ctx.clientDisconnected()) + onDisconnected(); + return resFut; } @@ -237,6 +245,8 @@ private void cancelFutures(MetadataUpdateResult res) { for (MetadataUpdateResultFuture fut : unlabeledFutures) fut.onDone(res); + unlabeledFutures.clear(); + for (MetadataUpdateResultFuture fut : syncMap.values()) fut.onDone(res); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateAcceptedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateAcceptedMessage.java index 0416746a04b68..df64613b71063 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateAcceptedMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateAcceptedMessage.java @@ -70,6 +70,11 @@ public class MetadataUpdateAcceptedMessage implements DiscoveryCustomMessage { return true; } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer, DiscoCache discoCache) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateProposedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateProposedMessage.java index f9bd66078cba2..84e32e1b3d7cc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateProposedMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/MetadataUpdateProposedMessage.java @@ -133,6 +133,11 @@ public MetadataUpdateProposedMessage(BinaryMetadata metadata, UUID origNodeId) { return true; } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer, DiscoCache discoCache) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java index 9b3c1ec9bafff..5bbbb3102c663 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 @@ -597,6 +597,45 @@ private List nodes(int p, AffinityTopologyVersion topVer, GridDhtPa return owners(p, AffinityTopologyVersion.NONE); } + /** {@inheritDoc} */ + @Override public List> allOwners() { + lock.readLock().lock(); + + try { + int parts = partitions(); + + List> res = new ArrayList<>(parts); + + for (int i = 0; i < parts; i++) + res.add(new ArrayList<>()); + + List allNodes = discoCache.cacheGroupAffinityNodes(grpId); + + for (int i = 0; i < allNodes.size(); i++) { + ClusterNode node = allNodes.get(i); + + GridDhtPartitionMap nodeParts = node2part.get(node.id()); + + if (nodeParts != null) { + for (Map.Entry e : nodeParts.map().entrySet()) { + if (e.getValue() == OWNING) { + int part = e.getKey(); + + List owners = res.get(part); + + owners.add(node); + } + } + } + } + + return res; + } + finally { + lock.readLock().unlock(); + } + } + /** {@inheritDoc} */ @Override public List moving(int p) { return nodes(p, AffinityTopologyVersion.NONE, MOVING, null); 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 ba55543233210..ea99f5d0d8263 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java @@ -970,6 +970,10 @@ else if (req.needVersion()) try { ctx.io().send(nodeId, res, ctx.ioPolicy()); } + catch (ClusterTopologyCheckedException e) { + if (log.isDebugEnabled()) + log.debug("Failed to send get response to node, node failed: " + nodeId); + } catch (IgniteCheckedException e) { U.error(log, "Failed to send get response to node (is node still alive?) [nodeId=" + nodeId + ",req=" + req + ", res=" + res + ']', e); 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 13564c2af2666..7f900cb67f533 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 @@ -236,6 +236,12 @@ public boolean initPartitionsWhenAffinityReady(AffinityTopologyVersion affVer, G */ public List owners(int p); + /** + * @return List indexed by partition number, each list element is collection of all nodes who + * owns corresponding partition. + */ + public List> allOwners(); + /** * @param p Partition ID. * @param topVer Topology version. 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 528f0a6e51c7a..538c57ec9ba97 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 @@ -1216,6 +1216,45 @@ private List nodes( return owners(p, AffinityTopologyVersion.NONE); } + /** {@inheritDoc} */ + @Override public List> allOwners() { + lock.readLock().lock(); + + try { + int parts = partitions(); + + List> res = new ArrayList<>(parts); + + for (int i = 0; i < parts; i++) + res.add(new ArrayList<>()); + + List allNodes = discoCache.cacheGroupAffinityNodes(grp.groupId()); + + for (int i = 0; i < allNodes.size(); i++) { + ClusterNode node = allNodes.get(i); + + GridDhtPartitionMap nodeParts = node2part.get(node.id()); + + if (nodeParts != null) { + for (Map.Entry e : nodeParts.map().entrySet()) { + if (e.getValue() == OWNING) { + int part = e.getKey(); + + List owners = res.get(part); + + owners.add(node); + } + } + } + } + + return res; + } + finally { + lock.readLock().unlock(); + } + } + /** {@inheritDoc} */ @Override public List moving(int p) { if (!grp.rebalanceEnabled()) 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 8da91a8915ae3..cbb49851e1f81 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java @@ -1506,12 +1506,16 @@ private void sendPartitions(ClusterNode oldestNode) { } catch (ClusterTopologyCheckedException ignore) { if (log.isDebugEnabled()) - log.debug("Oldest node left during partition exchange [nodeId=" + oldestNode.id() + + log.debug("Coordinator left during partition exchange [nodeId=" + oldestNode.id() + ", exchId=" + exchId + ']'); } catch (IgniteCheckedException e) { - U.error(log, "Failed to send local partitions to oldest node (will retry after timeout) [oldestNodeId=" + - oldestNode.id() + ", exchId=" + exchId + ']', e); + if (reconnectOnError(e)) + onDone(new IgniteNeedReconnectException(cctx.localNode(), e)); + else { + U.error(log, "Failed to send local partitions to coordinator [crd=" + oldestNode.id() + + ", exchId=" + exchId + ']', e); + } } } @@ -3369,9 +3373,13 @@ public void onNodeLeft(final ClusterNode node) { } if (allReceived) { - awaitSingleMapUpdates(); + cctx.kernalContext().getSystemExecutorService().submit(new Runnable() { + @Override public void run() { + awaitSingleMapUpdates(); - onAllReceived(null); + onAllReceived(null); + } + }); } } else { @@ -3399,7 +3407,13 @@ public void onNodeLeft(final ClusterNode node) { ", newCrd=" + crd0.id() + ']'); } - sendPartitions(crd0); + final ClusterNode newCrd = crd0; + + cctx.kernalContext().getSystemExecutorService().submit(new Runnable() { + @Override public void run() { + sendPartitions(newCrd); + } + }); } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java index d7dfa166b5d14..bbbd9996f358b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateFinishMessage.java @@ -93,6 +93,11 @@ public boolean success() { return false; } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer, DiscoCache discoCache) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java index 50fc0223101e8..81855fcbf44a8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java @@ -130,6 +130,11 @@ void exchangeActions(ExchangeActions exchangeActions) { return false; } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer, DiscoCache discoCache) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterMetricsUpdateMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterMetricsUpdateMessage.java new file mode 100644 index 0000000000000..5c3044b6808cc --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterMetricsUpdateMessage.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cluster; + +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.internal.GridDirectMap; +import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public class ClusterMetricsUpdateMessage implements Message { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private byte[] nodeMetrics; + + /** */ + @GridDirectMap(keyType = UUID.class, valueType = byte[].class) + private Map allNodesMetrics; + + /** + * Required by {@link GridIoMessageFactory}. + */ + public ClusterMetricsUpdateMessage() { + // No-op. + } + + /** + * @param nodeMetrics Node metrics. + */ + ClusterMetricsUpdateMessage(byte[] nodeMetrics) { + this.nodeMetrics = nodeMetrics; + } + + /** + * @param allNodesMetrics All nodes metrcis. + */ + ClusterMetricsUpdateMessage(Map allNodesMetrics) { + this.allNodesMetrics = allNodesMetrics; + } + + /** + * @return Node metrics. + */ + @Nullable byte[] nodeMetrics() { + return nodeMetrics; + } + + /** + * @return All nodes metrics. + */ + @Nullable Map allNodesMetrics() { + return allNodesMetrics; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeMap("allNodesMetrics", allNodesMetrics, MessageCollectionItemType.UUID, MessageCollectionItemType.BYTE_ARR)) + return false; + + writer.incrementState(); + + case 1: + if (!writer.writeByteArray("nodeMetrics", nodeMetrics)) + return false; + + writer.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + switch (reader.state()) { + case 0: + allNodesMetrics = reader.readMap("allNodesMetrics", MessageCollectionItemType.UUID, MessageCollectionItemType.BYTE_ARR, false); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 1: + nodeMetrics = reader.readByteArray("nodeMetrics"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(ClusterMetricsUpdateMessage.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 133; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 2; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ClusterMetricsUpdateMessage.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterNodeMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterNodeMetrics.java new file mode 100644 index 0000000000000..22a385ff03a49 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterNodeMetrics.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cluster; + +import java.io.Serializable; +import java.util.Collections; +import java.util.Map; +import org.apache.ignite.cache.CacheMetrics; +import org.apache.ignite.cluster.ClusterMetrics; +import org.apache.ignite.internal.ClusterMetricsSnapshot; + +/** + * + */ +class ClusterNodeMetrics implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private final byte[] metrics; + + /** */ + private final Map cacheMetrics; + + /** + * @param metrics Metrics. + * @param cacheMetrics Cache metrics. + */ + ClusterNodeMetrics(ClusterMetrics metrics, Map cacheMetrics) { + this.metrics = ClusterMetricsSnapshot.serialize(metrics); + this.cacheMetrics = cacheMetrics; + } + + /** + * @return Metrics. + */ + byte[] metrics() { + return metrics; + } + + /** + * @return Cache metrics. + */ + Map cacheMetrics() { + return cacheMetrics != null ? cacheMetrics : Collections.emptyMap(); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java index 5f2c66ce7fe59..8796302a0b466 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java @@ -33,6 +33,8 @@ import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.events.Event; +import org.apache.ignite.internal.ClusterMetricsSnapshot; +import org.apache.ignite.internal.GridDirectMap; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteDiagnosticInfo; import org.apache.ignite.internal.IgniteDiagnosticMessage; @@ -42,21 +44,29 @@ import org.apache.ignite.internal.cluster.IgniteClusterImpl; import org.apache.ignite.internal.managers.communication.GridIoPolicy; import org.apache.ignite.internal.managers.communication.GridMessageListener; +import org.apache.ignite.internal.managers.discovery.DiscoCache; +import org.apache.ignite.internal.managers.discovery.IgniteClusterNode; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; import org.apache.ignite.internal.processors.GridProcessorAdapter; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.util.GridTimerTask; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.CI1; +import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.apache.ignite.spi.discovery.DiscoveryDataBag; import org.apache.ignite.spi.discovery.DiscoveryDataBag.GridDiscoveryData; +import org.apache.ignite.spi.discovery.DiscoveryMetricsProvider; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.IgniteSystemProperties.IGNITE_DIAGNOSTIC_ENABLED; @@ -66,6 +76,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CLUSTER_PROC; import static org.apache.ignite.internal.GridTopic.TOPIC_INTERNAL_DIAGNOSTIC; +import static org.apache.ignite.internal.GridTopic.TOPIC_METRICS; import static org.apache.ignite.internal.IgniteVersionUtils.VER_STR; /** @@ -102,6 +113,18 @@ public class ClusterProcessor extends GridProcessorAdapter { /** */ private final AtomicLong diagFutId = new AtomicLong(); + /** */ + private final Map allNodesMetrics = new ConcurrentHashMap<>(); + + /** */ + private final JdkMarshaller marsh = new JdkMarshaller(); + + /** */ + private DiscoveryMetricsProvider metricsProvider; + + /** */ + private boolean sndMetrics; + /** * @param ctx Kernal context. */ @@ -111,6 +134,8 @@ public ClusterProcessor(GridKernalContext ctx) { notifyEnabled.set(IgniteSystemProperties.getBoolean(IGNITE_UPDATE_NOTIFIER, true)); cluster = new IgniteClusterImpl(ctx); + + sndMetrics = !(ctx.config().getDiscoverySpi() instanceof TcpDiscoverySpi); } /** @@ -120,33 +145,31 @@ public boolean diagnosticEnabled() { return getBoolean(IGNITE_DIAGNOSTIC_ENABLED, true); } - /** */ - private final JdkMarshaller marsh = new JdkMarshaller(); - /** * @throws IgniteCheckedException If failed. */ public void initDiagnosticListeners() throws IgniteCheckedException { ctx.event().addLocalEventListener(new GridLocalEventListener() { - @Override public void onEvent(Event evt) { - assert evt instanceof DiscoveryEvent; - assert evt.type() == EVT_NODE_FAILED || evt.type() == EVT_NODE_LEFT; + @Override public void onEvent(Event evt) { + assert evt instanceof DiscoveryEvent; + assert evt.type() == EVT_NODE_FAILED || evt.type() == EVT_NODE_LEFT; - DiscoveryEvent discoEvt = (DiscoveryEvent)evt; + DiscoveryEvent discoEvt = (DiscoveryEvent)evt; - UUID nodeId = discoEvt.eventNode().id(); + UUID nodeId = discoEvt.eventNode().id(); - ConcurrentHashMap futs = diagnosticFutMap.get(); + ConcurrentHashMap futs = diagnosticFutMap.get(); - if (futs != null) { - for (InternalDiagnosticFuture fut : futs.values()) { - if (fut.nodeId.equals(nodeId)) - fut.onDone(new IgniteDiagnosticInfo("Target node failed: " + nodeId)); - } + if (futs != null) { + for (InternalDiagnosticFuture fut : futs.values()) { + if (fut.nodeId.equals(nodeId)) + fut.onDone(new IgniteDiagnosticInfo("Target node failed: " + nodeId)); } } - }, - EVT_NODE_FAILED, EVT_NODE_LEFT); + + allNodesMetrics.remove(nodeId); + } + }, EVT_NODE_FAILED, EVT_NODE_LEFT); ctx.io().addMessageListener(TOPIC_INTERNAL_DIAGNOSTIC, new GridMessageListener() { @Override public void onMessage(UUID nodeId, Object msg, byte plc) { @@ -233,6 +256,17 @@ public void initDiagnosticListeners() throws IgniteCheckedException { U.warn(diagnosticLog, "Received unexpected message: " + msg); } }); + + if (sndMetrics) { + ctx.io().addMessageListener(TOPIC_METRICS, new GridMessageListener() { + @Override public void onMessage(UUID nodeId, Object msg, byte plc) { + if (msg instanceof ClusterMetricsUpdateMessage) + processMetricsUpdateMessage(nodeId, (ClusterMetricsUpdateMessage)msg); + else + U.warn(log, "Received unexpected message for TOPIC_METRICS: " + msg); + } + }); + } } /** @@ -296,7 +330,6 @@ private Serializable getDiscoveryData() { } } - /** * @param vals collection to seek through. */ @@ -334,6 +367,14 @@ private Boolean findLastFlag(Collection vals) { log.debug("Failed to create GridUpdateNotifier: " + e); } } + + if (sndMetrics) { + metricsProvider = ctx.discovery().createMetricsProvider(); + + long updateFreq = ctx.config().getMetricsUpdateFrequency(); + + ctx.timeout().addTimeoutObject(new MetricsUpdateTimeoutObject(updateFreq)); + } } /** {@inheritDoc} */ @@ -351,6 +392,133 @@ private Boolean findLastFlag(Collection vals) { ctx.io().removeMessageListener(TOPIC_INTERNAL_DIAGNOSTIC); } + /** + * @param sndNodeId Sender node ID. + * @param msg Message. + */ + private void processMetricsUpdateMessage(UUID sndNodeId, ClusterMetricsUpdateMessage msg) { + byte[] nodeMetrics = msg.nodeMetrics(); + + if (nodeMetrics != null) { + assert msg.allNodesMetrics() == null; + + allNodesMetrics.put(sndNodeId, nodeMetrics); + + updateNodeMetrics(ctx.discovery().discoCache(), sndNodeId, nodeMetrics); + } + else { + Map allNodesMetrics = msg.allNodesMetrics(); + + assert allNodesMetrics != null; + + DiscoCache discoCache = ctx.discovery().discoCache(); + + for (Map.Entry e : allNodesMetrics.entrySet()) { + if (!ctx.localNodeId().equals(e.getKey())) + updateNodeMetrics(discoCache, e.getKey(), e.getValue()); + } + } + } + + /** + * @param discoCache Discovery data cache. + * @param nodeId Node ID. + * @param metricsBytes Marshalled metrics. + */ + private void updateNodeMetrics(DiscoCache discoCache, UUID nodeId, byte[] metricsBytes) { + ClusterNode node = discoCache.node(nodeId); + + if (node == null || !discoCache.alive(nodeId)) + return; + + try { + ClusterNodeMetrics metrics = U.unmarshalZip(ctx.config().getMarshaller(), metricsBytes, null); + + assert node instanceof IgniteClusterNode : node; + + IgniteClusterNode node0 = (IgniteClusterNode)node; + + node0.setMetrics(ClusterMetricsSnapshot.deserialize(metrics.metrics(), 0)); + node0.setCacheMetrics(metrics.cacheMetrics()); + + ctx.discovery().metricsUpdateEvent(discoCache, node0); + } + catch (IgniteCheckedException e) { + U.warn(log, "Failed to unmarshal node metrics: " + e); + } + } + + /** + * + */ + private void updateMetrics() { + if (ctx.isStopping() || ctx.clientDisconnected()) + return; + + ClusterNode oldest = ctx.discovery().oldestAliveServerNode(AffinityTopologyVersion.NONE); + + if (oldest == null) + return; + + if (ctx.localNodeId().equals(oldest.id())) { + IgniteClusterNode locNode = (IgniteClusterNode)ctx.discovery().localNode(); + + locNode.setMetrics(metricsProvider.metrics()); + locNode.setCacheMetrics(metricsProvider.cacheMetrics()); + + ClusterNodeMetrics metrics = new ClusterNodeMetrics(locNode.metrics(), locNode.cacheMetrics()); + + try { + byte[] metricsBytes = U.zip(U.marshal(ctx.config().getMarshaller(), metrics)); + + allNodesMetrics.put(ctx.localNodeId(), metricsBytes); + } + catch (IgniteCheckedException e) { + U.warn(log, "Failed to marshal local node metrics: " + e, e); + } + + ctx.discovery().metricsUpdateEvent(ctx.discovery().discoCache(), locNode); + + Collection allNodes = ctx.discovery().allNodes(); + + ClusterMetricsUpdateMessage msg = new ClusterMetricsUpdateMessage(new HashMap<>(allNodesMetrics)); + + for (ClusterNode node : allNodes) { + if (ctx.localNodeId().equals(node.id()) || !ctx.discovery().alive(node.id())) + continue; + + try { + ctx.io().sendToGridTopic(node, TOPIC_METRICS, msg, GridIoPolicy.SYSTEM_POOL); + } + catch (ClusterTopologyCheckedException e) { + if (log.isDebugEnabled()) + log.debug("Failed to send metrics update, node failed: " + e); + } + catch (IgniteCheckedException e) { + U.warn(log, "Failed to send metrics update: " + e, e); + } + } + } + else { + ClusterNodeMetrics metrics = new ClusterNodeMetrics(metricsProvider.metrics(), metricsProvider.cacheMetrics()); + + try { + byte[] metricsBytes = U.zip(U.marshal(ctx.config().getMarshaller(), metrics)); + + ClusterMetricsUpdateMessage msg = new ClusterMetricsUpdateMessage(metricsBytes); + + ctx.io().sendToGridTopic(oldest, TOPIC_METRICS, msg, GridIoPolicy.SYSTEM_POOL); + } + catch (ClusterTopologyCheckedException e) { + if (log.isDebugEnabled()) + log.debug("Failed to send metrics update to oldest, node failed: " + e); + } + catch (IgniteCheckedException e) { + LT.warn(log, e, "Failed to send metrics update to oldest: " + e, false, false); + } + } + } + /** * Disables update notifier. */ @@ -571,4 +739,51 @@ public void onResponse(IgniteDiagnosticInfo res) { return S.toString(InternalDiagnosticFuture.class, this); } } + + /** + * + */ + private class MetricsUpdateTimeoutObject implements GridTimeoutObject, Runnable { + /** */ + private final IgniteUuid id = IgniteUuid.randomUuid(); + + /** */ + private long endTime; + + /** */ + private final long timeout; + + /** + * @param timeout Timeout. + */ + MetricsUpdateTimeoutObject(long timeout) { + this.timeout = timeout; + + endTime = U.currentTimeMillis() + timeout; + } + + /** {@inheritDoc} */ + @Override public IgniteUuid timeoutId() { + return id; + } + + /** {@inheritDoc} */ + @Override public long endTime() { + return endTime; + } + + /** {@inheritDoc} */ + @Override public void run() { + updateMetrics(); + + endTime = U.currentTimeMillis() + timeout; + + ctx.timeout().addTimeoutObject(this); + } + + /** {@inheritDoc} */ + @Override public void onTimeout() { + ctx.getSystemExecutorService().execute(this); + } + } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java index e9754d12cd966..928c619e128a2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java @@ -62,6 +62,11 @@ public UUID routineId() { return false; } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer, DiscoCache discoCache) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineInfo.java new file mode 100644 index 0000000000000..fc0f1810e481d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineInfo.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.continuous; + +import java.io.Serializable; +import java.util.UUID; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * + */ +class ContinuousRoutineInfo implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + UUID srcNodeId; + + /** */ + final UUID routineId; + + /** */ + final byte[] hnd; + + /** */ + final byte[] nodeFilter; + + /** */ + final int bufSize; + + /** */ + final long interval; + + /** */ + final boolean autoUnsubscribe; + + /** */ + transient boolean disconnected; + + /** + * @param srcNodeId Source node ID. + * @param routineId Routine ID. + * @param hnd Marshalled handler. + * @param nodeFilter Marshalled node filter. + * @param bufSize Handler buffer size. + * @param interval Time interval. + * @param autoUnsubscribe Auto unsubscribe flag. + */ + ContinuousRoutineInfo( + UUID srcNodeId, + UUID routineId, + byte[] hnd, + byte[] nodeFilter, + int bufSize, + long interval, + boolean autoUnsubscribe) + { + this.srcNodeId = srcNodeId; + this.routineId = routineId; + this.hnd = hnd; + this.nodeFilter = nodeFilter; + this.bufSize = bufSize; + this.interval = interval; + this.autoUnsubscribe = autoUnsubscribe; + } + + /** + * @param srcNodeId Source node ID. + */ + void sourceNodeId(UUID srcNodeId) { + this.srcNodeId = srcNodeId; + } + + /** + * + */ + void onDisconnected() { + disconnected = true; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ContinuousRoutineInfo.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineStartResultMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineStartResultMessage.java new file mode 100644 index 0000000000000..581ac603eea4c --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutineStartResultMessage.java @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.continuous; + +import java.nio.ByteBuffer; +import java.util.UUID; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMap; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public class ContinuousRoutineStartResultMessage implements Message { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private static final int ERROR_FLAG = 0x01; + + /** */ + private UUID routineId; + + /** */ + private byte[] errBytes; + + /** */ + private byte[] cntrsMapBytes; + + /** */ + private int flags; + + /** + * + */ + public ContinuousRoutineStartResultMessage() { + // No-op. + } + + /** + * @param routineId Routine ID. + * @param cntrsMapBytes Marshalled {@link CachePartitionPartialCountersMap}. + * @param errBytes Error bytes. + * @param err {@code True} if failed to start routine. + */ + ContinuousRoutineStartResultMessage(UUID routineId, byte[] cntrsMapBytes, byte[] errBytes, boolean err) { + this.routineId = routineId; + this.cntrsMapBytes = cntrsMapBytes; + this.errBytes = errBytes; + + if (err) + flags |= ERROR_FLAG; + } + + /** + * @return Marshalled {@link CachePartitionPartialCountersMap}. + */ + @Nullable byte[] countersMapBytes() { + return cntrsMapBytes; + } + + /** + * @return {@code True} if failed to start routine. + */ + boolean error() { + return (flags & ERROR_FLAG) != 0; + } + + /** + * @return Routine ID. + */ + UUID routineId() { + return routineId; + } + + /** + * @return Error bytes. + */ + @Nullable byte[] errorBytes() { + return errBytes; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeByteArray("cntrsMapBytes", cntrsMapBytes)) + return false; + + writer.incrementState(); + + case 1: + if (!writer.writeByteArray("errBytes", errBytes)) + return false; + + writer.incrementState(); + + case 2: + if (!writer.writeInt("flags", flags)) + return false; + + writer.incrementState(); + + case 3: + if (!writer.writeUuid("routineId", routineId)) + return false; + + writer.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + switch (reader.state()) { + case 0: + cntrsMapBytes = reader.readByteArray("cntrsMapBytes"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 1: + errBytes = reader.readByteArray("errBytes"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 2: + flags = reader.readInt("flags"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 3: + routineId = reader.readUuid("routineId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(ContinuousRoutineStartResultMessage.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 134; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 4; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ContinuousRoutineStartResultMessage.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesCommonDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesCommonDiscoveryData.java new file mode 100644 index 0000000000000..d29de89b9b111 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesCommonDiscoveryData.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.continuous; + +import java.io.Serializable; +import java.util.List; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * + */ +public class ContinuousRoutinesCommonDiscoveryData implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + final List startedRoutines; + + /** + * @param startedRoutines Routines started in cluster. + */ + ContinuousRoutinesCommonDiscoveryData(List startedRoutines) { + this.startedRoutines = startedRoutines; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ContinuousRoutinesCommonDiscoveryData.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesInfo.java new file mode 100644 index 0000000000000..ad24ff1805387 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesInfo.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.continuous; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.spi.discovery.DiscoveryDataBag; + +import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CONTINUOUS_PROC; + +/** + * + */ +class ContinuousRoutinesInfo { + /** */ + private final Map startedRoutines = new HashMap<>(); + + /** + * @param dataBag Discovery data bag. + */ + void collectGridNodeData(DiscoveryDataBag dataBag) { + synchronized (startedRoutines) { + if (!dataBag.commonDataCollectedFor(CONTINUOUS_PROC.ordinal())) + dataBag.addGridCommonData(CONTINUOUS_PROC.ordinal(), + new ContinuousRoutinesCommonDiscoveryData(new ArrayList<>(startedRoutines.values()))); + } + } + + /** + * @param dataBag Discovery data bag. + */ + void collectJoiningNodeData(DiscoveryDataBag dataBag) { + synchronized (startedRoutines) { + for (ContinuousRoutineInfo info : startedRoutines.values()) { + if (info.disconnected) + info.sourceNodeId(dataBag.joiningNodeId()); + } + + dataBag.addJoiningNodeData(CONTINUOUS_PROC.ordinal(), + new ContinuousRoutinesJoiningNodeDiscoveryData(new ArrayList<>(startedRoutines.values()))); + } + } + + /** + * @param info Routine info. + */ + void addRoutineInfo(ContinuousRoutineInfo info) { + synchronized (startedRoutines) { + startedRoutines.put(info.routineId, info); + } + } + + /** + * @param routineId Routine ID. + * @return {@code True} if routine exists. + */ + boolean routineExists(UUID routineId) { + synchronized (startedRoutines) { + return startedRoutines.containsKey(routineId); + } + } + + /** + * @param routineId Routine ID. + */ + void removeRoutine(UUID routineId) { + synchronized (startedRoutines) { + startedRoutines.remove(routineId); + } + } + + /** + * @param locRoutines Routines IDs which can survive reconnect. + */ + void onClientDisconnected(Collection locRoutines) { + synchronized (startedRoutines) { + for (Iterator> it = startedRoutines.entrySet().iterator(); it.hasNext();) { + Map.Entry e = it.next(); + + ContinuousRoutineInfo info = e.getValue(); + + if (!locRoutines.contains(info.routineId)) + it.remove(); + else + info.onDisconnected(); + } + } + } + + /** + * Removes all routines with autoUnsubscribe=false started by given node. + * + * @param nodeId Node ID. + */ + void onNodeFail(UUID nodeId) { + synchronized (startedRoutines) { + for (Iterator> it = startedRoutines.entrySet().iterator(); it.hasNext();) { + Map.Entry e = it.next(); + + ContinuousRoutineInfo info = e.getValue(); + + if (info.autoUnsubscribe && info.srcNodeId.equals(nodeId)) + it.remove(); + } + } + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ContinuousRoutinesInfo.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesJoiningNodeDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesJoiningNodeDiscoveryData.java new file mode 100644 index 0000000000000..9be6ef8e07eab --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/ContinuousRoutinesJoiningNodeDiscoveryData.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.continuous; + +import java.io.Serializable; +import java.util.List; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * + */ +public class ContinuousRoutinesJoiningNodeDiscoveryData implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + final List startedRoutines; + + /** + * @param startedRoutines Routines registered on nodes, to be started in cluster. + */ + ContinuousRoutinesJoiningNodeDiscoveryData(List startedRoutines) { + this.startedRoutines = startedRoutines; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ContinuousRoutinesJoiningNodeDiscoveryData.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java index 01a5a71a75f9c..cebe4b177e300 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java @@ -56,6 +56,8 @@ import org.apache.ignite.internal.managers.deployment.GridDeploymentInfo; import org.apache.ignite.internal.managers.deployment.GridDeploymentInfoBean; import org.apache.ignite.internal.managers.discovery.CustomEventListener; +import org.apache.ignite.internal.managers.discovery.DiscoCache; +import org.apache.ignite.internal.managers.discovery.DiscoveryMessageResultsCollector; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; import org.apache.ignite.internal.managers.eventstorage.HighPriorityListener; import org.apache.ignite.internal.processors.GridProcessorAdapter; @@ -63,6 +65,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheProcessor; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMap; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryHandler; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -147,6 +150,12 @@ public class GridContinuousProcessor extends GridProcessorAdapter { /** Query sequence number for message topic. */ private final AtomicLong seq = new AtomicLong(); + /** */ + private ContinuousRoutinesInfo routinesInfo; + + /** */ + private int discoProtoVer; + /** * @param ctx Kernal context. */ @@ -156,6 +165,11 @@ public GridContinuousProcessor(GridKernalContext ctx) { /** {@inheritDoc} */ @Override public void start() throws IgniteCheckedException { + discoProtoVer = ctx.discovery().mutableCustomMessages() ? 1 : 2; + + if (discoProtoVer == 2) + routinesInfo = new ContinuousRoutinesInfo(); + if (ctx.config().isDaemon()) return; @@ -177,6 +191,8 @@ public GridContinuousProcessor(GridKernalContext ctx) { @Override public void onCustomEvent(AffinityTopologyVersion topVer, ClusterNode snd, StartRoutineDiscoveryMessage msg) { + assert discoProtoVer == 1 : discoProtoVer; + if (ctx.isStopping()) return; @@ -184,6 +200,20 @@ public GridContinuousProcessor(GridKernalContext ctx) { } }); + ctx.discovery().setCustomEventListener(StartRoutineDiscoveryMessageV2.class, + new CustomEventListener() { + @Override public void onCustomEvent(AffinityTopologyVersion topVer, + ClusterNode snd, + StartRoutineDiscoveryMessageV2 msg) { + assert discoProtoVer == 2 : discoProtoVer; + + if (ctx.isStopping()) + return; + + processStartRequestV2(topVer, snd, msg); + } + }); + ctx.discovery().setCustomEventListener(StartRoutineAckDiscoveryMessage.class, new CustomEventListener() { @Override public void onCustomEvent(AffinityTopologyVersion topVer, @@ -201,6 +231,9 @@ public GridContinuousProcessor(GridKernalContext ctx) { @Override public void onCustomEvent(AffinityTopologyVersion topVer, ClusterNode snd, StopRoutineDiscoveryMessage msg) { + if (discoProtoVer == 2) + routinesInfo.removeRoutine(msg.routineId); + if (ctx.isStopping()) return; @@ -222,32 +255,36 @@ public GridContinuousProcessor(GridKernalContext ctx) { ctx.io().addMessageListener(TOPIC_CONTINUOUS, new GridMessageListener() { @Override public void onMessage(UUID nodeId, Object obj, byte plc) { - GridContinuousMessage msg = (GridContinuousMessage)obj; + if (obj instanceof ContinuousRoutineStartResultMessage) + processRoutineStartResultMessage(nodeId, (ContinuousRoutineStartResultMessage)obj); + else { + GridContinuousMessage msg = (GridContinuousMessage)obj; - if (msg.data() == null && msg.dataBytes() != null) { - try { - msg.data(U.unmarshal(marsh, msg.dataBytes(), U.resolveClassLoader(ctx.config()))); - } - catch (IgniteCheckedException e) { - U.error(log, "Failed to process message (ignoring): " + msg, e); + if (msg.data() == null && msg.dataBytes() != null) { + try { + msg.data(U.unmarshal(marsh, msg.dataBytes(), U.resolveClassLoader(ctx.config()))); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to process message (ignoring): " + msg, e); - return; + return; + } } - } - switch (msg.type()) { - case MSG_EVT_NOTIFICATION: - processNotification(nodeId, msg); + switch (msg.type()) { + case MSG_EVT_NOTIFICATION: + processNotification(nodeId, msg); - break; + break; - case MSG_EVT_ACK: - processMessageAck(msg); + case MSG_EVT_ACK: + processMessageAck(msg); - break; + break; - default: - assert false : "Unexpected message received: " + msg.type(); + default: + assert false : "Unexpected message received: " + msg.type(); + } } } }); @@ -341,6 +378,15 @@ public void unlockStopping() { /** {@inheritDoc} */ @Override public void collectJoiningNodeData(DiscoveryDataBag dataBag) { + if (ctx.isDaemon()) + return; + + if (discoProtoVer == 2) { + routinesInfo.collectJoiningNodeData(dataBag); + + return; + } + Serializable data = getDiscoveryData(dataBag.joiningNodeId()); if (data != null) @@ -349,6 +395,15 @@ public void unlockStopping() { /** {@inheritDoc} */ @Override public void collectGridNodeData(DiscoveryDataBag dataBag) { + if (ctx.isDaemon()) + return; + + if (discoProtoVer == 2) { + routinesInfo.collectGridNodeData(dataBag); + + return; + } + Serializable data = getDiscoveryData(dataBag.joiningNodeId()); if (data != null) @@ -393,6 +448,7 @@ private Serializable getDiscoveryData(UUID joiningNodeId) { return data; } + return null; } @@ -430,22 +486,118 @@ private Map copyLocalInfos(Map l @Override public void onJoiningNodeDataReceived(JoiningNodeDiscoveryData data) { if (log.isDebugEnabled()) { log.debug("onJoiningNodeDataReceived [joining=" + data.joiningNodeId() + - ", loc=" + ctx.localNodeId() + - ", data=" + data.joiningNodeData() + - ']'); + ", loc=" + ctx.localNodeId() + + ", data=" + data.joiningNodeData() + + ']'); } - if (data.hasJoiningNodeData()) - onDiscoDataReceived((DiscoveryData) data.joiningNodeData()); + if (discoProtoVer == 2) { + if (data.hasJoiningNodeData()) { + ContinuousRoutinesJoiningNodeDiscoveryData nodeData = (ContinuousRoutinesJoiningNodeDiscoveryData) + data.joiningNodeData(); + + for (ContinuousRoutineInfo routineInfo : nodeData.startedRoutines) { + routinesInfo.addRoutineInfo(routineInfo); + + startDiscoveryDataRoutine(routineInfo); + } + } + } + else { + if (data.hasJoiningNodeData()) + onDiscoDataReceived((DiscoveryData) data.joiningNodeData()); + } } /** {@inheritDoc} */ @Override public void onGridDataReceived(GridDiscoveryData data) { - Map nodeSpecData = data.nodeSpecificData(); + if (discoProtoVer == 2) { + if (ctx.isDaemon()) + return; + + if (data.commonData() != null) { + ContinuousRoutinesCommonDiscoveryData commonData = + (ContinuousRoutinesCommonDiscoveryData)data.commonData(); + + for (ContinuousRoutineInfo routineInfo : commonData.startedRoutines) { + if (routinesInfo.routineExists(routineInfo.routineId)) + continue; + + routinesInfo.addRoutineInfo(routineInfo); + + startDiscoveryDataRoutine(routineInfo); + } + } + } + else { + Map nodeSpecData = data.nodeSpecificData(); + + if (nodeSpecData != null) { + for (Map.Entry e : nodeSpecData.entrySet()) + onDiscoDataReceived((DiscoveryData) e.getValue()); + } + } + } + + /** + * @param routineInfo Routine info. + */ + private void startDiscoveryDataRoutine(ContinuousRoutineInfo routineInfo) { + IgnitePredicate nodeFilter = null; + + try { + if (routineInfo.nodeFilter != null) { + nodeFilter = U.unmarshal(marsh, routineInfo.nodeFilter, U.resolveClassLoader(ctx.config())); + + ctx.resource().injectGeneric(nodeFilter); + } + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to unmarshal continuous routine filter, ignore routine [" + + "routineId=" + routineInfo.routineId + + ", srcNodeId=" + routineInfo.srcNodeId + ']', e); + + return; + } + + if (nodeFilter == null || nodeFilter.apply(ctx.discovery().localNode())) { + GridContinuousHandler hnd; - if (nodeSpecData != null) { - for (Map.Entry e : nodeSpecData.entrySet()) - onDiscoDataReceived((DiscoveryData) e.getValue()); + try { + hnd = U.unmarshal(marsh, routineInfo.hnd, U.resolveClassLoader(ctx.config())); + + if (ctx.config().isPeerClassLoadingEnabled()) + hnd.p2pUnmarshal(routineInfo.srcNodeId, ctx); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to unmarshal continuous routine handler, ignore routine [" + + "routineId=" + routineInfo.routineId + + ", srcNodeId=" + routineInfo.srcNodeId + ']', e); + + return; + } + + try { + registerHandler(routineInfo.srcNodeId, + routineInfo.routineId, + hnd, + routineInfo.bufSize, + routineInfo.interval, + routineInfo.autoUnsubscribe, + false); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to register continuous routine handler, ignore routine [" + + "routineId=" + routineInfo.routineId + + ", srcNodeId=" + routineInfo.srcNodeId + ']', e); + } + } + else { + if (log.isDebugEnabled()) { + log.debug("Do not register continuous routine, rejected by node filter [" + + "routineId=" + routineInfo.routineId + + ", srcNodeId=" + routineInfo.srcNodeId + ']'); + } } } @@ -564,13 +716,14 @@ public void onCacheStop(GridCacheContext ctx) { * @param rmtFilter Remote filter. * @param prjPred Projection predicate. * @return Routine ID. + * @throws IgniteCheckedException If failed. */ @SuppressWarnings("unchecked") public UUID registerStaticRoutine( String cacheName, CacheEntryUpdatedListener locLsnr, CacheEntryEventSerializableFilter rmtFilter, - @Nullable IgnitePredicate prjPred) { + @Nullable IgnitePredicate prjPred) throws IgniteCheckedException { String topicPrefix = "CONTINUOUS_QUERY_STATIC" + "_" + cacheName; CacheContinuousQueryHandler hnd = new CacheContinuousQueryHandler( @@ -589,6 +742,17 @@ public UUID registerStaticRoutine( LocalRoutineInfo routineInfo = new LocalRoutineInfo(prjPred, hnd, 1, 0, true); + if (discoProtoVer == 2) { + routinesInfo.addRoutineInfo(createRoutineInfo( + ctx.localNodeId(), + routineId, + hnd, + prjPred, + routineInfo.bufSize, + routineInfo.interval, + routineInfo.autoUnsubscribe)); + } + locInfos.put(routineId, routineInfo); registerMessageListener(hnd); @@ -596,6 +760,40 @@ public UUID registerStaticRoutine( return routineId; } + /** + * @param srcNodeId Source node ID. + * @param routineId Routine ID. + * @param hnd Handler. + * @param nodeFilter Node filter. + * @param bufSize Handler buffer size. + * @param interval Time interval. + * @param autoUnsubscribe Auto unsubscribe flag. + * @return Routine info instance. + * @throws IgniteCheckedException If failed. + */ + private ContinuousRoutineInfo createRoutineInfo( + UUID srcNodeId, + UUID routineId, + GridContinuousHandler hnd, + @Nullable IgnitePredicate nodeFilter, + int bufSize, + long interval, + boolean autoUnsubscribe) + throws IgniteCheckedException { + byte[] hndBytes = marsh.marshal(hnd); + + byte[] filterBytes = nodeFilter != null ? marsh.marshal(nodeFilter) : null; + + return new ContinuousRoutineInfo( + srcNodeId, + routineId, + hndBytes, + filterBytes, + bufSize, + interval, + autoUnsubscribe); + } + /** * @param hnd Handler. * @param bufSize Buffer size. @@ -638,30 +836,10 @@ public IgniteInternalFuture startRoutine(GridContinuousHandler hnd, // Whether local node is included in routine. boolean locIncluded = prjPred == null || prjPred.apply(ctx.discovery().localNode()); - StartRequestData reqData = new StartRequestData(prjPred, hnd.clone(), bufSize, interval, autoUnsubscribe); + AbstractContinuousMessage msg; try { - if (ctx.config().isPeerClassLoadingEnabled()) { - // Handle peer deployment for projection predicate. - if (prjPred != null && !U.isGrid(prjPred.getClass())) { - Class cls = U.detectClass(prjPred); - - String clsName = cls.getName(); - - GridDeployment dep = ctx.deploy().deploy(cls, U.detectClassLoader(cls)); - - if (dep == null) - throw new IgniteDeploymentCheckedException("Failed to deploy projection predicate: " + prjPred); - - reqData.className(clsName); - reqData.deploymentInfo(new GridDeploymentInfoBean(dep)); - - reqData.p2pMarshal(marsh); - } - - // Handle peer deployment for other handler-specific objects. - reqData.handler().p2pMarshal(ctx); - } + msg = createStartMessage(routineId, hnd, bufSize, interval, autoUnsubscribe, prjPred); } catch (IgniteCheckedException e) { return new GridFinishedFuture<>(e); @@ -674,20 +852,26 @@ public IgniteInternalFuture startRoutine(GridContinuousHandler hnd, return new GridFinishedFuture<>(new NodeStoppingException("Failed to start continuous query (node is stopping)")); try { - StartFuture fut = new StartFuture(ctx, routineId); + StartFuture fut = new StartFuture(routineId); startFuts.put(routineId, fut); try { - if (locIncluded || hnd.isQuery()) - registerHandler(ctx.localNodeId(), routineId, hnd, bufSize, interval, autoUnsubscribe, true); + if (locIncluded || hnd.isQuery()) { + registerHandler(ctx.localNodeId(), + routineId, + hnd, + bufSize, + interval, + autoUnsubscribe, + true); + } - ctx.discovery().sendCustomEvent(new StartRoutineDiscoveryMessage(routineId, reqData, - reqData.handler().keepBinary())); - } - catch (IgniteCheckedException e) { - startFuts.remove(routineId); - locInfos.remove(routineId); + ctx.discovery().sendCustomEvent(msg); + } + catch (IgniteCheckedException e) { + startFuts.remove(routineId); + locInfos.remove(routineId); unregisterHandler(routineId, hnd, true); @@ -706,6 +890,92 @@ public IgniteInternalFuture startRoutine(GridContinuousHandler hnd, } } + /** + * @param routineId Routine ID. + * @param hnd Handler. + * @param bufSize Buffer size. + * @param interval Interval. + * @param autoUnsubscribe Auto unsubscribe flag. + * @param nodeFilter Node filter. + * @return Routine start message. + * @throws IgniteCheckedException If failed. + */ + private AbstractContinuousMessage createStartMessage(UUID routineId, + GridContinuousHandler hnd, + int bufSize, + long interval, + boolean autoUnsubscribe, + @Nullable IgnitePredicate nodeFilter) + throws IgniteCheckedException + { + hnd = hnd.clone(); + + String clsName = null; + GridDeploymentInfoBean dep = null; + + if (ctx.config().isPeerClassLoadingEnabled()) { + // Handle peer deployment for projection predicate. + if (nodeFilter != null && !U.isGrid(nodeFilter.getClass())) { + Class cls = U.detectClass(nodeFilter); + + clsName = cls.getName(); + + GridDeployment dep0 = ctx.deploy().deploy(cls, U.detectClassLoader(cls)); + + if (dep0 == null) + throw new IgniteDeploymentCheckedException("Failed to deploy projection predicate: " + nodeFilter); + + dep = new GridDeploymentInfoBean(dep0); + } + + // Handle peer deployment for other handler-specific objects. + hnd.p2pMarshal(ctx); + } + + if (discoProtoVer == 1) { + StartRequestData reqData = new StartRequestData( + nodeFilter, + hnd, + bufSize, + interval, + autoUnsubscribe); + + if (clsName != null) { + reqData.className(clsName); + reqData.deploymentInfo(dep); + + reqData.p2pMarshal(marsh); + } + + return new StartRoutineDiscoveryMessage( + routineId, + reqData, + reqData.handler().keepBinary()); + } + else { + assert discoProtoVer == 2 : discoProtoVer; + + byte[] nodeFilterBytes = nodeFilter != null ? U.marshal(marsh, nodeFilter) : null; + byte[] hndBytes = U.marshal(marsh, hnd); + + StartRequestDataV2 reqData = new StartRequestDataV2(nodeFilterBytes, + hndBytes, + bufSize, + interval, + autoUnsubscribe); + + if (clsName != null) { + reqData.className(clsName); + reqData.deploymentInfo(dep); + } + + return new StartRoutineDiscoveryMessageV2( + routineId, + reqData, + hnd.keepBinary()); + } + } + /** * @param hnd Handler. */ @@ -760,29 +1030,38 @@ public IgniteInternalFuture stopRoutine(UUID routineId) { doStop = true; } - if (doStop) { - // Unregister routine locally. - LocalRoutineInfo routine = locInfos.remove(routineId); - - // Finish if routine is not found (wrong ID is provided). - if (routine == null) { - stopFuts.remove(routineId); + if (doStop) { + boolean stop = false; - fut.onDone(); + // Unregister routine locally. + LocalRoutineInfo routine = locInfos.remove(routineId); - return fut; - } + if (routine != null) { + stop = true; // Unregister handler locally. unregisterHandler(routineId, routine.hnd, true); + } - try { - ctx.discovery().sendCustomEvent(new StopRoutineDiscoveryMessage(routineId)); - } - catch (IgniteCheckedException e) { - fut.onDone(e); + if (!stop && discoProtoVer == 2) + stop = routinesInfo.routineExists(routineId); + + // Finish if routine is not found (wrong ID is provided). + if (!stop) { + stopFuts.remove(routineId); + + fut.onDone(); + + return fut; } + try { + ctx.discovery().sendCustomEvent(new StopRoutineDiscoveryMessage(routineId)); + } + catch (IgniteCheckedException e) { + fut.onDone(e); + } + if (ctx.isStopping()) fut.onDone(); } @@ -924,6 +1203,9 @@ public void addNotification(UUID nodeId, clientInfos.clear(); + if (discoProtoVer == 2) + routinesInfo.onClientDisconnected(locInfos.keySet()); + if (log.isDebugEnabled()) { log.debug("after onDisconnected [rmtInfos=" + rmtInfos + ", locInfos=" + locInfos + @@ -996,35 +1278,11 @@ private void processStartAckRequest(AffinityTopologyVersion topVer, StartFuture fut = startFuts.remove(msg.routineId()); if (fut != null) { - if (msg.errs().isEmpty()) { - LocalRoutineInfo routine = locInfos.get(msg.routineId()); - - // Update partition counters. - if (routine != null && routine.handler().isQuery()) { - Map>> cntrsPerNode = msg.updateCountersPerNode(); - Map> cntrs = msg.updateCounters(); - - GridCacheAdapter interCache = - ctx.cache().internalCache(routine.handler().cacheName()); - - GridCacheContext cctx = interCache != null ? interCache.context() : null; - - if (cctx != null && cntrsPerNode != null && !cctx.isLocal() && cctx.affinityNode()) - cntrsPerNode.put(ctx.localNodeId(), - toCountersMap(cctx.topology().localUpdateCounters(false))); - - routine.handler().updateCounters(topVer, cntrsPerNode, cntrs); - } - - fut.onRemoteRegistered(); - } - else { - IgniteCheckedException firstEx = F.first(msg.errs().values()); - - fut.onDone(firstEx); - - stopRoutine(msg.routineId()); - } + fut.onAllRemoteRegistered( + topVer, + msg.errs(), + msg.updateCountersPerNode(), + msg.updateCounters()); } } @@ -1137,6 +1395,199 @@ private void processStartRequest(ClusterNode node, StartRoutineDiscoveryMessage req.addError(ctx.localNodeId(), err); } + /** + * @param sndId Sender node ID. + * @param msg Message. + */ + private void processRoutineStartResultMessage(UUID sndId, ContinuousRoutineStartResultMessage msg) { + StartFuture fut = startFuts.get(msg.routineId()); + + if (fut != null) + fut.onResult(sndId, msg); + } + + /** + * @param topVer Current topology version. + * @param snd Sender. + * @param msg Start request. + */ + private void processStartRequestV2(final AffinityTopologyVersion topVer, + final ClusterNode snd, + final StartRoutineDiscoveryMessageV2 msg) { + StartRequestDataV2 reqData = msg.startRequestData(); + + ContinuousRoutineInfo routineInfo = new ContinuousRoutineInfo(snd.id(), + msg.routineId(), + reqData.handlerBytes(), + reqData.nodeFilterBytes(), + reqData.bufferSize(), + reqData.interval(), + reqData.autoUnsubscribe()); + + routinesInfo.addRoutineInfo(routineInfo); + + final DiscoCache discoCache = ctx.discovery().discoCache(topVer); + + // Should not use marshaller and send messages from discovery thread. + ctx.getSystemExecutorService().execute(new Runnable() { + @Override public void run() { + if (snd.id().equals(ctx.localNodeId())) { + StartFuture fut = startFuts.get(msg.routineId()); + + if (fut != null) + fut.initRemoteNodes(discoCache); + + return; + } + + StartRequestDataV2 reqData = msg.startRequestData(); + + Exception err = null; + + IgnitePredicate nodeFilter = null; + + byte[] cntrs = null; + + if (reqData.nodeFilterBytes() != null) { + try { + if (ctx.config().isPeerClassLoadingEnabled() && reqData.className() != null) { + String clsName = reqData.className(); + GridDeploymentInfo depInfo = reqData.deploymentInfo(); + + GridDeployment dep = ctx.deploy().getGlobalDeployment(depInfo.deployMode(), + clsName, + clsName, + depInfo.userVersion(), + snd.id(), + depInfo.classLoaderId(), + depInfo.participants(), + null); + + if (dep == null) { + throw new IgniteDeploymentCheckedException("Failed to obtain deployment " + + "for class: " + clsName); + } + + nodeFilter = U.unmarshal(marsh, + reqData.nodeFilterBytes(), + U.resolveClassLoader(dep.classLoader(), ctx.config())); + } + else { + nodeFilter = U.unmarshal(marsh, + reqData.nodeFilterBytes(), + U.resolveClassLoader(ctx.config())); + } + + if (nodeFilter != null) + ctx.resource().injectGeneric(nodeFilter); + } + catch (Exception e) { + err = e; + + U.error(log, "Failed to unmarshal continuous routine filter [" + + "routineId=" + msg.routineId + + ", srcNodeId=" + snd.id() + ']', e); + } + } + + boolean register = err == null && + (nodeFilter == null || nodeFilter.apply(ctx.discovery().localNode())); + + if (register) { + try { + GridContinuousHandler hnd = U.unmarshal(marsh, + reqData.handlerBytes(), + U.resolveClassLoader(ctx.config())); + + if (ctx.config().isPeerClassLoadingEnabled()) + hnd.p2pUnmarshal(snd.id(), ctx); + + if (msg.keepBinary()) { + assert hnd instanceof CacheContinuousQueryHandler : hnd; + + ((CacheContinuousQueryHandler)hnd).keepBinary(true); + } + + GridContinuousHandler hnd0 = hnd instanceof GridMessageListenHandler ? + new GridMessageListenHandler((GridMessageListenHandler)hnd) : + hnd; + + registerHandler(snd.id(), + msg.routineId, + hnd0, + reqData.bufferSize(), + reqData.interval(), + reqData.autoUnsubscribe(), + false); + + if (hnd0.isQuery()) { + GridCacheProcessor proc = ctx.cache(); + + if (proc != null) { + GridCacheAdapter cache = ctx.cache().internalCache(hnd0.cacheName()); + + if (cache != null && !cache.isLocal() && cache.context().userCache()) { + CachePartitionPartialCountersMap cntrsMap = + cache.context().topology().localUpdateCounters(false); + + cntrs = U.marshal(marsh, cntrsMap); + } + } + } + } + catch (Exception e) { + err = e; + + U.error(log, "Failed to register continuous routine handler [" + + "routineId=" + msg.routineId + + ", srcNodeId=" + snd.id() + ']', e); + } + } + + sendMessageStartResult(snd, msg.routineId(), cntrs, err); + } + }); + } + + /** + * @param node Target node. + * @param routineId Routine ID. + * @param cntrsMapBytes Marshalled {@link CachePartitionPartialCountersMap}. + * @param err Start error if any. + */ + private void sendMessageStartResult(final ClusterNode node, + final UUID routineId, + byte[] cntrsMapBytes, + final @Nullable Exception err) + { + byte[] errBytes = null; + + if (err != null) { + try { + errBytes = U.marshal(marsh, err); + } + catch (Exception e) { + U.error(log, "Failed to marshal routine start error: " + e, e); + } + } + + ContinuousRoutineStartResultMessage msg = new ContinuousRoutineStartResultMessage(routineId, + cntrsMapBytes, + errBytes, + err != null); + + try { + ctx.io().sendToGridTopic(node, TOPIC_CONTINUOUS, msg, SYSTEM_POOL, null); + } + catch (ClusterTopologyCheckedException e) { + if (log.isDebugEnabled()) + log.debug("Failed to send routine start result, node failed: " + e); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to send routine start result: " + e, e); + } + } + /** * @param msg Message. */ @@ -1455,6 +1906,13 @@ private class DiscoveryListener implements GridLocalEventListener, HighPriorityL UUID nodeId = ((DiscoveryEvent)evt).eventNode().id(); + if (discoProtoVer == 2) { + routinesInfo.onNodeFail(nodeId); + + for (StartFuture fut : startFuts.values()) + fut.onNodeFail(nodeId); + } + clientInfos.remove(nodeId); // Unregister handlers created by left node. @@ -1894,10 +2352,7 @@ public DiscoveryDataItem() { /** * Future for start routine. */ - private static class StartFuture extends GridFutureAdapter { - /** */ - private GridKernalContext ctx; - + private class StartFuture extends GridFutureAdapter { /** Consume ID. */ private UUID routineId; @@ -1907,56 +2362,170 @@ private static class StartFuture extends GridFutureAdapter { /** All remote listeners are registered. */ private volatile boolean rmt; - /** Timeout object. */ - private volatile GridTimeoutObject timeoutObj; + /** */ + private final DiscoveryMessageResultsCollector + resCollect; /** - * @param ctx Kernal context. * @param routineId Consume ID. */ - StartFuture(GridKernalContext ctx, UUID routineId) { - this.ctx = ctx; - + StartFuture(UUID routineId) { this.routineId = routineId; + + resCollect = new DiscoveryMessageResultsCollector(ctx) { + @Override protected RoutineRegisterResults createResult(Map> rcvd) { + Map errs = null; + Map>> cntrsPerNode = null; + + for (Map.Entry> entry : rcvd.entrySet()) { + ContinuousRoutineStartResultMessage msg = entry.getValue().message(); + + if (msg == null) + continue; + + if (msg.error()) { + byte[] errBytes = msg.errorBytes(); + + Exception err = null; + + if (errBytes != null) { + try { + err = U.unmarshal(marsh, errBytes, U.resolveClassLoader(ctx.config())); + } + catch (Exception e) { + U.warn(log, "Failed to unmarhal continuous routine start error: " + e); + } + } + + if (err == null) { + err = new IgniteCheckedException("Failed to start continuous " + + "routine on node: " + entry.getKey()); + } + + if (errs == null) + errs = new HashMap<>(); + + errs.put(entry.getKey(), err); + } + else { + byte[] cntrsMapBytes = msg.countersMapBytes(); + + if (cntrsMapBytes != null) { + try { + CachePartitionPartialCountersMap cntrsMap = U.unmarshal( + marsh, + cntrsMapBytes, + U.resolveClassLoader(ctx.config())); + + if (cntrsPerNode == null) + cntrsPerNode = new HashMap<>(); + + cntrsPerNode.put(entry.getKey(), CachePartitionPartialCountersMap.toCountersMap(cntrsMap)); + } + catch (Exception e) { + U.warn(log, "Failed to unmarhal continuous query update counters: " + e); + } + } + } + } + + return new RoutineRegisterResults(discoCache.version(), errs, cntrsPerNode); + } + + @Override protected void onResultsCollected(RoutineRegisterResults res0) { + onAllRemoteRegistered(res0.topVer, res0.errs, res0.cntrsPerNode, null); + } + + @Override protected boolean waitForNode(DiscoCache discoCache, ClusterNode node) { + return !ctx.localNodeId().equals(node.id()); + } + }; } /** - * Called when local listener is registered. + * @param topVer Topology version. + * @param errs Errors. + * @param cntrsPerNode Update counters. + * @param cntrs Update counters. */ - public void onLocalRegistered() { - loc = true; + private void onAllRemoteRegistered( + AffinityTopologyVersion topVer, + @Nullable Map errs, + Map>> cntrsPerNode, + Map> cntrs) { + try { + if (errs == null || errs.isEmpty()) { + LocalRoutineInfo routine = locInfos.get(routineId); - if (rmt && !isDone()) - onDone(routineId); + // Update partition counters. + if (routine != null && routine.handler().isQuery()) { + GridCacheAdapter interCache = + ctx.cache().internalCache(routine.handler().cacheName()); + + GridCacheContext cctx = interCache != null ? interCache.context() : null; + + if (cctx != null && cntrsPerNode != null && !cctx.isLocal() && cctx.affinityNode()) + cntrsPerNode.put(ctx.localNodeId(), + toCountersMap(cctx.topology().localUpdateCounters(false))); + + routine.handler().updateCounters(topVer, cntrsPerNode, cntrs); + } + + onRemoteRegistered(); + } + else { + Exception firstEx = F.first(errs.values()); + + onDone(firstEx); + + stopRoutine(routineId); + } + } + finally { + startFuts.remove(routineId, this); + } } /** - * Called when all remote listeners are registered. + * @param discoCache Discovery state. */ - public void onRemoteRegistered() { - rmt = true; + void initRemoteNodes(DiscoCache discoCache) { + resCollect.init(discoCache); + } - if (loc && !isDone()) - onDone(routineId); + /** + * @param nodeId Node ID. + * @param msg Message. + */ + void onResult(UUID nodeId, ContinuousRoutineStartResultMessage msg) { + resCollect.onMessage(nodeId, msg); } /** - * @param timeoutObj Timeout object. + * @param nodeId Failed node ID. */ - public void addTimeoutObject(GridTimeoutObject timeoutObj) { - assert timeoutObj != null; + void onNodeFail(UUID nodeId) { + resCollect.onNodeFail(nodeId); + } - this.timeoutObj = timeoutObj; + /** + * Called when local listener is registered. + */ + void onLocalRegistered() { + loc = true; - ctx.timeout().addTimeoutObject(timeoutObj); + if (rmt && !isDone()) + onDone(routineId); } - /** {@inheritDoc} */ - @Override public boolean onDone(@Nullable UUID res, @Nullable Throwable err) { - if (timeoutObj != null) - ctx.timeout().removeTimeoutObject(timeoutObj); + /** + * Called when all remote listeners are registered. + */ + void onRemoteRegistered() { + rmt = true; - return super.onDone(res, err); + if (loc && !isDone()) + onDone(routineId); } /** {@inheritDoc} */ @@ -1965,6 +2534,33 @@ public void addTimeoutObject(GridTimeoutObject timeoutObj) { } } + /** + * + */ + private static class RoutineRegisterResults { + /** */ + private final AffinityTopologyVersion topVer; + + /** */ + private final Map errs; + + /** */ + private final Map>> cntrsPerNode; + + /** + * @param topVer Topology version. + * @param errs Errors. + * @param cntrsPerNode Update counters. + */ + RoutineRegisterResults(AffinityTopologyVersion topVer, + Map errs, + Map>> cntrsPerNode) { + this.topVer = topVer; + this.errs = errs; + this.cntrsPerNode = cntrsPerNode; + } + } + /** * Future for stop routine. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestDataV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestDataV2.java new file mode 100644 index 0000000000000..c001616eab3c1 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestDataV2.java @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.continuous; + +import java.io.Serializable; +import org.apache.ignite.internal.managers.deployment.GridDeploymentInfo; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * Start request data. + */ +class StartRequestDataV2 implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** Serialized node filter. */ + private byte[] nodeFilterBytes; + + /** Deployment class name. */ + private String clsName; + + /** Deployment info. */ + private GridDeploymentInfo depInfo; + + /** Serialized handler. */ + private byte[] hndBytes; + + /** Buffer size. */ + private int bufSize; + + /** Time interval. */ + private long interval; + + /** Automatic unsubscribe flag. */ + private boolean autoUnsubscribe; + + /** + * @param nodeFilterBytes Serialized node filter. + * @param hndBytes Serialized handler. + * @param bufSize Buffer size. + * @param interval Time interval. + * @param autoUnsubscribe Automatic unsubscribe flag. + */ + StartRequestDataV2( + byte[] nodeFilterBytes, + byte[] hndBytes, + int bufSize, + long interval, + boolean autoUnsubscribe) { + assert hndBytes != null; + assert bufSize > 0; + assert interval >= 0; + + this.nodeFilterBytes = nodeFilterBytes; + this.hndBytes = hndBytes; + this.bufSize = bufSize; + this.interval = interval; + this.autoUnsubscribe = autoUnsubscribe; + } + + /** + * @return Serialized node filter. + */ + public byte[] nodeFilterBytes() { + return nodeFilterBytes; + } + + /** + * @return Deployment class name. + */ + public String className() { + return clsName; + } + + /** + * @param clsName New deployment class name. + */ + public void className(String clsName) { + this.clsName = clsName; + } + + /** + * @return Deployment info. + */ + public GridDeploymentInfo deploymentInfo() { + return depInfo; + } + + /** + * @param depInfo New deployment info. + */ + public void deploymentInfo(GridDeploymentInfo depInfo) { + this.depInfo = depInfo; + } + + /** + * @return Handler. + */ + public byte[] handlerBytes() { + return hndBytes; + } + + /** + * @return Buffer size. + */ + public int bufferSize() { + return bufSize; + } + + /** + * @param bufSize New buffer size. + */ + public void bufferSize(int bufSize) { + this.bufSize = bufSize; + } + + /** + * @return Time interval. + */ + public long interval() { + return interval; + } + + /** + * @param interval New time interval. + */ + public void interval(long interval) { + this.interval = interval; + } + + /** + * @return Automatic unsubscribe flag. + */ + public boolean autoUnsubscribe() { + return autoUnsubscribe; + } + + /** + * @param autoUnsubscribe New automatic unsubscribe flag. + */ + public void autoUnsubscribe(boolean autoUnsubscribe) { + this.autoUnsubscribe = autoUnsubscribe; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(StartRequestDataV2.class, this); + } +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessageV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessageV2.java new file mode 100644 index 0000000000000..275765da73050 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessageV2.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.continuous; + +import java.util.UUID; +import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * + */ +public class StartRoutineDiscoveryMessageV2 extends AbstractContinuousMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private static final int KEEP_BINARY_FLAG = 0x01; + + /** */ + private final StartRequestDataV2 startReqData; + + /** Flags. */ + private int flags; + + /** + * @param routineId Routine id. + * @param startReqData Start request data. + * @param keepBinary Keep binary flag. + */ + StartRoutineDiscoveryMessageV2(UUID routineId, StartRequestDataV2 startReqData, boolean keepBinary) { + super(routineId); + + this.startReqData = startReqData; + + if (keepBinary) + flags |= KEEP_BINARY_FLAG; + } + + /** + * @return Start request data. + */ + public StartRequestDataV2 startRequestData() { + return startReqData; + } + + /** + * @return {@code True} if keep binary flag was set on continuous handler. + */ + public boolean keepBinary() { + return (flags & KEEP_BINARY_FLAG) != 0; + } + + /** {@inheritDoc} */ + @Override public DiscoveryCustomMessage ackMessage() { + return null; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(StartRoutineDiscoveryMessageV2.class, this, "routineId", routineId()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineAckDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineAckDiscoveryMessage.java index 79d8b29a24081..dfba0e759f9f7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineAckDiscoveryMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineAckDiscoveryMessage.java @@ -41,6 +41,11 @@ public StopRoutineAckDiscoveryMessage(UUID routineId) { return null; } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(StopRoutineAckDiscoveryMessage.class, this, "routineId", routineId()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java index 4a893f4096877..8cad342a7e696 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java @@ -47,6 +47,7 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteClientDisconnectedException; import org.apache.ignite.IgniteDataStreamer; import org.apache.ignite.IgniteDataStreamerTimeoutException; import org.apache.ignite.IgniteException; @@ -100,6 +101,7 @@ import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.GPC; @@ -1059,6 +1061,9 @@ private void doFlush() throws IgniteCheckedException { return; while (true) { + if (disconnectErr != null) + throw disconnectErr; + Queue> q = null; for (Buffer buf : bufMappings.values()) { @@ -1826,15 +1831,19 @@ private void submit( catch (IgniteCheckedException e) { GridFutureAdapter fut0 = ((GridFutureAdapter)fut); - try { - if (ctx.discovery().alive(node) && ctx.discovery().pingNode(node.id())) - fut0.onDone(e); - else - fut0.onDone(new ClusterTopologyCheckedException("Failed to send request (node has left): " - + node.id())); - } - catch (IgniteClientDisconnectedCheckedException e0) { - fut0.onDone(e0); + if (X.hasCause(e, IgniteClientDisconnectedCheckedException.class, IgniteClientDisconnectedException.class)) + fut0.onDone(e); + else { + try { + if (ctx.discovery().alive(node) && ctx.discovery().pingNode(node.id())) + fut0.onDone(e); + else + fut0.onDone(new ClusterTopologyCheckedException("Failed to send request (node has left): " + + node.id())); + } + catch (IgniteClientDisconnectedCheckedException e0) { + fut0.onDone(e0); + } } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingAcceptedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingAcceptedMessage.java index 7af0559752b3c..80e3f7dafd6ca 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingAcceptedMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingAcceptedMessage.java @@ -62,6 +62,11 @@ public class MappingAcceptedMessage implements DiscoveryCustomMessage { return false; } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer, DiscoCache discoCache) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingProposedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingProposedMessage.java index b4e13fba14bf0..93585858668d0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingProposedMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingProposedMessage.java @@ -97,6 +97,11 @@ private enum ProposalStatus { return true; } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Nullable @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer, DiscoCache discoCache) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaFinishDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaFinishDiscoveryMessage.java index 2245b24667ac7..f802e090dfa39 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaFinishDiscoveryMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaFinishDiscoveryMessage.java @@ -58,6 +58,11 @@ public SchemaFinishDiscoveryMessage(SchemaAbstractOperation op, SchemaOperationE return false; } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Override public boolean exchange() { return false; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaProposeDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaProposeDiscoveryMessage.java index 0e1270b17b623..62b6d6abb9f0c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaProposeDiscoveryMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/message/SchemaProposeDiscoveryMessage.java @@ -59,6 +59,11 @@ public SchemaProposeDiscoveryMessage(SchemaAbstractOperation op) { return true; } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Override public boolean exchange() { return exchange; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java index e0ec8d1abf103..0fcde0e136644 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java @@ -489,6 +489,17 @@ public GridNioFuture close(GridNioSession ses) { return fut; } + /** + * @param ses Session. + */ + public void closeFromWorkerThread(GridNioSession ses) { + assert ses instanceof GridSelectorNioSessionImpl : ses; + + GridSelectorNioSessionImpl ses0 = (GridSelectorNioSessionImpl)ses; + + ((AbstractNioClientWorker)ses0.worker()).close((GridSelectorNioSessionImpl)ses, null); + } + /** * @param ses Session. * @param msg Message. @@ -2170,7 +2181,12 @@ private void dumpStats(StringBuilder sb, dumpSelectorInfo(sb, keys); for (SelectionKey key : keys) { - GridSelectorNioSessionImpl ses = (GridSelectorNioSessionImpl)key.attachment(); + GridNioKeyAttachment attach = (GridNioKeyAttachment)key.attachment(); + + if (!attach.hasSession()) + continue; + + GridSelectorNioSessionImpl ses = attach.session(); boolean sesInfo = p == null || p.apply(ses); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java index 1754cc8cfcb66..e8c27d2f90626 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java @@ -963,5 +963,15 @@ private class GridDummySpiContext implements IgniteSpiContext { @Override public Map nodeAttributes() { return Collections.emptyMap(); } + + /** {@inheritDoc} */ + @Override public boolean communicationFailureResolveSupported() { + return false; + } + + /** {@inheritDoc} */ + @Override public void resolveCommunicationFailure(ClusterNode node, Exception err) { + throw new UnsupportedOperationException(); + } } } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java index 108c4d443c6e0..d4402f41e1928 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java @@ -365,4 +365,15 @@ public interface IgniteSpiContext { * @return Current node attributes. */ public Map nodeAttributes(); + + /** + * @return {@code True} if cluster supports communication error resolving. + */ + public boolean communicationFailureResolveSupported(); + + /** + * @param node Problem node. + * @param err Error. + */ + public void resolveCommunicationFailure(ClusterNode node, Exception err); } 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 c5f366bfb8d26..a3fccbcad5da2 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -32,6 +32,7 @@ import java.nio.channels.spi.AbstractInterruptibleChannel; import java.util.ArrayList; import java.util.Arrays; +import java.util.BitSet; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -65,12 +66,14 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; import org.apache.ignite.internal.managers.eventstorage.HighPriorityListener; import org.apache.ignite.internal.util.GridConcurrentFactory; import org.apache.ignite.internal.util.GridSpinReadWriteLock; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.future.IgniteFutureImpl; import org.apache.ignite.internal.util.ipc.IpcEndpoint; import org.apache.ignite.internal.util.ipc.IpcToNioAdapter; import org.apache.ignite.internal.util.ipc.shmem.IpcOutOfSystemResourcesException; @@ -133,6 +136,9 @@ import org.apache.ignite.spi.IgniteSpiTimeoutObject; import org.apache.ignite.spi.communication.CommunicationListener; import org.apache.ignite.spi.communication.CommunicationSpi; +import org.apache.ignite.spi.communication.tcp.internal.ConnectionKey; +import org.apache.ignite.spi.communication.tcp.internal.TcpCommunicationConnectionCheckFuture; +import org.apache.ignite.spi.communication.tcp.internal.TcpCommunicationNodeConnectionCheckFuture; import org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage; import org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage2; import org.apache.ignite.spi.communication.tcp.messages.NodeIdMessage; @@ -145,6 +151,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.SSL_META; +import static org.apache.ignite.spi.communication.tcp.internal.TcpCommunicationConnectionCheckFuture.SES_FUT_META; import static org.apache.ignite.spi.communication.tcp.messages.RecoveryLastReceivedMessage.ALREADY_CONNECTED; import static org.apache.ignite.spi.communication.tcp.messages.RecoveryLastReceivedMessage.NEED_WAIT; import static org.apache.ignite.spi.communication.tcp.messages.RecoveryLastReceivedMessage.NODE_STOPPING; @@ -309,7 +316,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati private static final IgniteProductVersion VERSION_SINCE_CLIENT_COULD_WAIT_TO_CONNECT = IgniteProductVersion.fromString("2.1.4"); /** Connection index meta for session. */ - private static final int CONN_IDX_META = GridNioSessionMetaKey.nextUniqueKey(); + public static final int CONN_IDX_META = GridNioSessionMetaKey.nextUniqueKey(); /** Message tracker meta for session. */ private static final int TRACKER_META = GridNioSessionMetaKey.nextUniqueKey(); @@ -407,6 +414,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati ConnectionKey connId = ses.meta(CONN_IDX_META); if (connId != null) { + if (connId.dummy()) + return; + UUID id = connId.nodeId(); GridCommunicationClient[] nodeClients = clients.get(id); @@ -480,20 +490,22 @@ private void onFirstMessage(final GridNioSession ses, Message msg) { if (rmtNode == null) { DiscoverySpi discoverySpi = ignite().configuration().getDiscoverySpi(); - assert discoverySpi instanceof TcpDiscoverySpi; - - TcpDiscoverySpi tcpDiscoverySpi = (TcpDiscoverySpi) discoverySpi; + boolean unknownNode = true; - ClusterNode node0 = tcpDiscoverySpi.getNode0(sndId); + if (discoverySpi instanceof TcpDiscoverySpi) { + TcpDiscoverySpi tcpDiscoverySpi = (TcpDiscoverySpi) discoverySpi; - boolean unknownNode = true; + ClusterNode node0 = tcpDiscoverySpi.getNode0(sndId); - if (node0 != null) { - assert node0.isClient() : node0; + if (node0 != null) { + assert node0.isClient() : node0; - if (node0.version().compareTo(VERSION_SINCE_CLIENT_COULD_WAIT_TO_CONNECT) >= 0) - unknownNode = false; + if (node0.version().compareTo(VERSION_SINCE_CLIENT_COULD_WAIT_TO_CONNECT) >= 0) + unknownNode = false; + } } + else if (discoverySpi instanceof IgniteDiscoverySpi) + unknownNode = !((IgniteDiscoverySpi) discoverySpi).knownNode(sndId); if (unknownNode) { U.warn(log, "Close incoming connection, unknown node [nodeId=" + sndId + ", ses=" + ses + ']'); @@ -708,9 +720,9 @@ private void closeStaleConnections(ConnectionKey connKey) { } } else { - metricsLsnr.onMessageReceived(msg, connKey.nodeId()); - if (msg instanceof RecoveryLastReceivedMessage) { + metricsLsnr.onMessageReceived(msg, connKey.nodeId()); + GridNioRecoveryDescriptor recovery = ses.outRecoveryDescriptor(); if (recovery != null) { @@ -723,9 +735,9 @@ private void closeStaleConnections(ConnectionKey connKey) { } recovery.ackReceived(msg0.received()); - - return; } + + return; } else { GridNioRecoveryDescriptor recovery = ses.inRecoveryDescriptor(); @@ -745,8 +757,23 @@ private void closeStaleConnections(ConnectionKey connKey) { recovery.lastAcknowledged(rcvCnt); } } + else if (connKey.dummy()) { + assert msg instanceof NodeIdMessage : msg; + + TcpCommunicationNodeConnectionCheckFuture fut = ses.meta(SES_FUT_META); + + assert fut != null : msg; + + fut.onConnected(U.bytesToUuid(((NodeIdMessage)msg).nodeIdBytes(), 0)); + + nioSrvr.closeFromWorkerThread(ses); + + return; + } } + metricsLsnr.onMessageReceived(msg, connKey.nodeId()); + IgniteRunnable c; if (msgQueueLimit > 0) { @@ -2111,6 +2138,13 @@ private void dumpInfo(StringBuilder sb, UUID dstNodeId) { } } + /** + * @return Bound TCP server port. + */ + public int boundPort() { + return boundTcpPort; + } + /** {@inheritDoc} */ @Override public void spiStart(String igniteInstanceName) throws IgniteSpiException { assert locHost != null; @@ -2568,6 +2602,27 @@ private void checkAttributePresence(ClusterNode node, String attrName) { sendMessage0(node, msg, null); } + /** + * @param nodes Nodes to check connection with. + * @return Result future (each bit in result BitSet contains connection status to corresponding node). + */ + public IgniteFuture checkConnection(List nodes) { + TcpCommunicationConnectionCheckFuture fut = new TcpCommunicationConnectionCheckFuture( + this, + log.getLogger(TcpCommunicationConnectionCheckFuture.class), + nioSrvr, + nodes); + + long timeout = failureDetectionTimeoutEnabled() ? failureDetectionTimeout() : connTimeout; + + if (log.isInfoEnabled()) + log.info("Start check connection process [nodeCnt=" + nodes.size() + ", timeout=" + timeout + ']'); + + fut.init(timeout); + + return new IgniteFutureImpl<>(fut); + } + /** * Sends given message to destination node. Note that characteristics of the * exchange such as durability, guaranteed delivery or error notification is @@ -2988,7 +3043,7 @@ private void checkClientQueueSize(GridNioSession ses, int msgQueueSize) { ConnectionKey id = ses.meta(CONN_IDX_META); if (id != null) { - ClusterNode node = getSpiContext().node(id.nodeId); + ClusterNode node = getSpiContext().node(id.nodeId()); if (node != null && node.isClient()) { String msg = "Client node outbound message queue size exceeded slowClientQueueLimit, " + @@ -3009,9 +3064,20 @@ private void checkClientQueueSize(GridNioSession ses, int msgQueueSize) { /** * @param node Node. * @return Node addresses. + * @throws IgniteCheckedException If failed. + */ + private Collection nodeAddresses(ClusterNode node) throws IgniteCheckedException { + return nodeAddresses(node, filterReachableAddresses); + } + + /** + * @param node Node. + * @param filterReachableAddresses Filter addresses flag. + * @return Node addresses. * @throws IgniteCheckedException If node does not have addresses. */ - private LinkedHashSet nodeAddresses(ClusterNode node) throws IgniteCheckedException { + public Collection nodeAddresses(ClusterNode node, boolean filterReachableAddresses) + throws IgniteCheckedException { Collection rmtAddrs0 = node.attribute(createSpiAttributeName(ATTR_ADDRS)); Collection rmtHostNames0 = node.attribute(createSpiAttributeName(ATTR_HOST_NAMES)); Integer boundPort = node.attribute(createSpiAttributeName(ATTR_PORT)); @@ -3092,7 +3158,7 @@ private LinkedHashSet nodeAddresses(ClusterNode node) throws * @throws IgniteCheckedException If failed. */ protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) throws IgniteCheckedException { - LinkedHashSet addrs = nodeAddresses(node); + Collection addrs = nodeAddresses(node); GridCommunicationClient client = null; IgniteCheckedException errs = null; @@ -3110,6 +3176,9 @@ protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) int lastWaitingTimeout = 1; while (client == null) { // Reconnection on handshake timeout. + if (stopping) + throw new IgniteSpiException("Node is stopping."); + if (addr.getAddress().isLoopbackAddress() && addr.getPort() == boundTcpPort) { if (log.isDebugEnabled()) log.debug("Skipping local address [addr=" + addr + @@ -3350,8 +3419,18 @@ else if (X.hasCause(e, SocketTimeoutException.class)) "operating system firewall is disabled on local and remote hosts) " + "[addrs=" + addrs + ']'); - if (enableForcibleNodeKill) { - if (getSpiContext().node(node.id()) != null + boolean commErrResolve = false; + + IgniteSpiContext ctx = getSpiContext(); + + if (connectionError(errs) && ctx.communicationFailureResolveSupported()) { + commErrResolve = true; + + ctx.resolveCommunicationFailure(node, errs); + } + + if (!commErrResolve && enableForcibleNodeKill) { + if (ctx.node(node.id()) != null && (CU.clientNode(node) || !CU.clientNode(getLocalNode())) && connectionError(errs)) { String msg = "TcpCommunicationSpi failed to establish connection to node, node will be dropped from " + @@ -3362,7 +3441,7 @@ else if (X.hasCause(e, SocketTimeoutException.class)) else U.warn(log, msg); - getSpiContext().failNode(node.id(), "TcpCommunicationSpi failed to establish connection to node [" + + ctx.failNode(node.id(), "TcpCommunicationSpi failed to establish connection to node [" + "rmtNode=" + node + ", errs=" + errs + ", connectErrs=" + Arrays.toString(errs.getSuppressed()) + ']'); @@ -4565,77 +4644,6 @@ private static class DisconnectedSessionInfo { } } - /** - * - */ - private static class ConnectionKey { - /** */ - private final UUID nodeId; - - /** */ - private final int idx; - - /** */ - private final long connCnt; - - /** - * @param nodeId Node ID. - * @param idx Connection index. - * @param connCnt Connection counter (set only for incoming connections). - */ - ConnectionKey(UUID nodeId, int idx, long connCnt) { - this.nodeId = nodeId; - this.idx = idx; - this.connCnt = connCnt; - } - - /** - * @return Connection counter. - */ - long connectCount() { - return connCnt; - } - - /** - * @return Node ID. - */ - UUID nodeId() { - return nodeId; - } - - /** - * @return Connection index. - */ - int connectionIndex() { - return idx; - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object o) { - if (this == o) - return true; - - if (o == null || getClass() != o.getClass()) - return false; - - ConnectionKey key = (ConnectionKey) o; - - return idx == key.idx && nodeId.equals(key.nodeId); - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - int res = nodeId.hashCode(); - res = 31 * res + idx; - return res; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(ConnectionKey.class, this); - } - } - /** * */ diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/ConnectionKey.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/ConnectionKey.java new file mode 100644 index 0000000000000..0559df7892459 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/ConnectionKey.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.communication.tcp.internal; + +import java.util.UUID; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.NotNull; + +/** + * Connection Key. + */ +public class ConnectionKey { + /** */ + private final UUID nodeId; + + /** */ + private final int idx; + + /** */ + private final long connCnt; + + /** */ + private final boolean dummy; + + /** + * Creates ConnectionKey with false value of dummy flag. + * + * @param nodeId Node ID. Should be not null. + * @param idx Connection index. + * @param connCnt Connection counter (set only for incoming connections). + */ + public ConnectionKey(@NotNull UUID nodeId, int idx, long connCnt) { + this(nodeId, idx, connCnt, false); + } + + /** + * @param nodeId Node ID. Should be not null. + * @param idx Connection index. + * @param connCnt Connection counter (set only for incoming connections). + * @param dummy Indicates that session with this ConnectionKey is temporary + * (for now dummy sessions are used only for Communication Failure Resolving process). + */ + public ConnectionKey(@NotNull UUID nodeId, int idx, long connCnt, boolean dummy) { + this.nodeId = nodeId; + this.idx = idx; + this.connCnt = connCnt; + this.dummy = dummy; + } + + /** + * @return Connection counter. + */ + public long connectCount() { + return connCnt; + } + + /** + * @return Node ID. + */ + public UUID nodeId() { + return nodeId; + } + + /** + * @return Connection index. + */ + public int connectionIndex() { + return idx; + } + + /** + * @return {@code True} if this ConnectionKey is dummy and serves temporary session. + */ + public boolean dummy() { + return dummy; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + ConnectionKey key = (ConnectionKey) o; + + return idx == key.idx && nodeId.equals(key.nodeId); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + int res = nodeId.hashCode(); + res = 31 * res + idx; + return res; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ConnectionKey.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationConnectionCheckFuture.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationConnectionCheckFuture.java new file mode 100644 index 0000000000000..c42fa576c768a --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationConnectionCheckFuture.java @@ -0,0 +1,519 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.communication.tcp.internal; + +import java.net.InetSocketAddress; +import java.nio.channels.SocketChannel; +import java.util.BitSet; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.events.DiscoveryEvent; +import org.apache.ignite.events.Event; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; +import org.apache.ignite.internal.util.GridLeanMap; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.nio.GridNioServer; +import org.apache.ignite.internal.util.nio.GridNioSession; +import org.apache.ignite.internal.util.nio.GridNioSessionMetaKey; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.spi.IgniteSpiTimeoutObject; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; +import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; + +/** + * Tcp Communication Connection Check Future. + */ +public class TcpCommunicationConnectionCheckFuture extends GridFutureAdapter implements IgniteSpiTimeoutObject, GridLocalEventListener { + /** Session future. */ + public static final int SES_FUT_META = GridNioSessionMetaKey.nextUniqueKey(); + + /** */ + private static final AtomicIntegerFieldUpdater connFutDoneUpdater = + AtomicIntegerFieldUpdater.newUpdater(SingleAddressConnectFuture.class, "done"); + + /** */ + private static final AtomicIntegerFieldUpdater connResCntUpdater = + AtomicIntegerFieldUpdater.newUpdater(MultipleAddressesConnectFuture.class, "resCnt"); + + /** */ + private final AtomicInteger resCntr = new AtomicInteger(); + + /** */ + private final List nodes; + + /** */ + private volatile ConnectFuture[] futs; + + /** */ + private final GridNioServer nioSrvr; + + /** */ + private final TcpCommunicationSpi spi; + + /** */ + private final IgniteUuid timeoutObjId = IgniteUuid.randomUuid(); + + /** */ + private final BitSet resBitSet; + + /** */ + private long endTime; + + /** */ + private final IgniteLogger log; + + /** + * @param spi SPI instance. + * @param log Logger. + * @param nioSrvr NIO server. + * @param nodes Nodes to check. + */ + public TcpCommunicationConnectionCheckFuture(TcpCommunicationSpi spi, + IgniteLogger log, + GridNioServer nioSrvr, + List nodes) + { + this.spi = spi; + this.log = log; + this.nioSrvr = nioSrvr; + this.nodes = nodes; + + resBitSet = new BitSet(nodes.size()); + } + + /** + * @param timeout Connect timeout. + */ + public void init(long timeout) { + ConnectFuture[] futs = new ConnectFuture[nodes.size()]; + + UUID locId = spi.getSpiContext().localNode().id(); + + for (int i = 0; i < nodes.size(); i++) { + ClusterNode node = nodes.get(i); + + if (!node.id().equals(locId)) { + if (spi.getSpiContext().node(node.id()) == null) { + receivedConnectionStatus(i, false); + + continue; + } + + Collection addrs; + + try { + addrs = spi.nodeAddresses(node, false); + } + catch (Exception e) { + U.error(log, "Failed to get node addresses: " + node, e); + + receivedConnectionStatus(i, false); + + continue; + } + + if (addrs.size() == 1) { + SingleAddressConnectFuture fut = new SingleAddressConnectFuture(i); + + fut.init(addrs.iterator().next(), node.id()); + + futs[i] = fut; + } + else { + MultipleAddressesConnectFuture fut = new MultipleAddressesConnectFuture(i); + + fut.init(addrs, node.id()); + + futs[i] = fut; + } + } + else + receivedConnectionStatus(i, true); + } + + this.futs = futs; + + spi.getSpiContext().addLocalEventListener(this, EVT_NODE_LEFT, EVT_NODE_FAILED); + + if (!isDone()) { + endTime = System.currentTimeMillis() + timeout; + + spi.getSpiContext().addTimeoutObject(this); + } + } + + /** + * @param idx Node index. + * @param res Success flag. + */ + private void receivedConnectionStatus(int idx, boolean res) { + assert resCntr.get() < nodes.size(); + + synchronized (resBitSet) { + resBitSet.set(idx, res); + } + + if (resCntr.incrementAndGet() == nodes.size()) + onDone(resBitSet); + } + + /** + * @param nodeIdx Node index. + * @return Node ID. + */ + private UUID nodeId(int nodeIdx) { + return nodes.get(nodeIdx).id(); + } + + /** {@inheritDoc} */ + @Override public IgniteUuid id() { + return timeoutObjId; + } + + /** {@inheritDoc} */ + @Override public long endTime() { + return endTime; + } + + /** {@inheritDoc} */ + @Override public void onEvent(Event evt) { + if (isDone()) + return; + + assert evt instanceof DiscoveryEvent : evt; + assert evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED ; + + UUID nodeId = ((DiscoveryEvent)evt).eventNode().id(); + + for (int i = 0; i < nodes.size(); i++) { + if (nodes.get(i).id().equals(nodeId)) { + ConnectFuture fut = futs[i]; + + if (fut != null) + fut.onNodeFailed(); + + return; + } + } + } + + /** {@inheritDoc} */ + @Override public void onTimeout() { + if (isDone()) + return; + + ConnectFuture[] futs = this.futs; + + for (int i = 0; i < futs.length; i++) { + ConnectFuture fut = futs[i]; + + if (fut != null) + fut.onTimeout(); + } + } + + /** {@inheritDoc} */ + @Override public boolean onDone(@Nullable BitSet res, @Nullable Throwable err) { + if (super.onDone(res, err)) { + spi.getSpiContext().removeTimeoutObject(this); + + spi.getSpiContext().removeLocalEventListener(this); + + return true; + } + + return false; + } + + /** + * + */ + private interface ConnectFuture { + /** + * + */ + void onTimeout(); + + /** + * + */ + void onNodeFailed(); + } + + /** + * + */ + private class SingleAddressConnectFuture implements TcpCommunicationNodeConnectionCheckFuture, ConnectFuture { + /** */ + final int nodeIdx; + + /** */ + volatile int done; + + /** */ + Map sesMeta; + + /** */ + private SocketChannel ch; + + /** + * @param nodeIdx Node index. + */ + SingleAddressConnectFuture(int nodeIdx) { + this.nodeIdx = nodeIdx; + } + + /** + * @param addr Node address. + * @param rmtNodeId Id of node to open connection check session with. + */ + public void init(InetSocketAddress addr, UUID rmtNodeId) { + boolean connect; + + try { + ch = SocketChannel.open(); + + ch.configureBlocking(false); + + ch.socket().setTcpNoDelay(true); + ch.socket().setKeepAlive(false); + + connect = ch.connect(addr); + } + catch (Exception e) { + finish(false); + + return; + } + + if (!connect) { + sesMeta = new GridLeanMap<>(3); + + // Set dummy key to identify connection-check outgoing connection. + sesMeta.put(TcpCommunicationSpi.CONN_IDX_META, new ConnectionKey(rmtNodeId, -1, -1, true)); + sesMeta.put(SES_FUT_META, this); + + nioSrvr.createSession(ch, sesMeta, true, new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture fut) { + if (fut.error() != null) + finish(false); + } + }); + } + } + + /** + * + */ + @SuppressWarnings("unchecked") + void cancel() { + if (finish(false)) + nioSrvr.cancelConnect(ch, sesMeta); + } + + /** {@inheritDoc} */ + public void onTimeout() { + cancel(); + } + + /** {@inheritDoc} */ + public void onConnected(UUID rmtNodeId) { + finish(nodeId(nodeIdx).equals(rmtNodeId)); + } + + /** {@inheritDoc} */ + @Override public void onNodeFailed() { + cancel(); + } + + /** + * @param res Result. + * @return {@code True} if result was set by this call. + */ + public boolean finish(boolean res) { + if (connFutDoneUpdater.compareAndSet(this, 0, 1)) { + onStatusReceived(res); + + return true; + } + + return false; + } + + /** + * @param res Result. + */ + void onStatusReceived(boolean res) { + receivedConnectionStatus(nodeIdx, res); + } + } + + /** + * + */ + private class MultipleAddressesConnectFuture implements ConnectFuture { + /** */ + volatile int resCnt; + + /** */ + volatile SingleAddressConnectFuture[] futs; + + /** */ + final int nodeIdx; + + /** + * @param nodeIdx Node index. + */ + MultipleAddressesConnectFuture(int nodeIdx) { + this.nodeIdx = nodeIdx; + + } + + /** {@inheritDoc} */ + @Override public void onNodeFailed() { + SingleAddressConnectFuture[] futs = this.futs; + + for (int i = 0; i < futs.length; i++) { + ConnectFuture fut = futs[i]; + + if (fut != null) + fut.onNodeFailed(); + } + } + + /** {@inheritDoc} */ + @Override public void onTimeout() { + SingleAddressConnectFuture[] futs = this.futs; + + for (int i = 0; i < futs.length; i++) { + ConnectFuture fut = futs[i]; + + if (fut != null) + fut.onTimeout(); + } + } + + /** + * @param addrs Node addresses. + * @param rmtNodeId Id of node to open connection check session with. + */ + void init(Collection addrs, UUID rmtNodeId) { + SingleAddressConnectFuture[] futs = new SingleAddressConnectFuture[addrs.size()]; + + for (int i = 0; i < addrs.size(); i++) { + SingleAddressConnectFuture fut = new SingleAddressConnectFuture(nodeIdx) { + @Override void onStatusReceived(boolean res) { + receivedAddressStatus(res); + } + }; + + futs[i] = fut; + } + + this.futs = futs; + + int idx = 0; + + for (InetSocketAddress addr : addrs) { + futs[idx++].init(addr, rmtNodeId); + + if (resCnt == Integer.MAX_VALUE) + return; + } + + // Close race. + if (done()) + cancelFutures(); + } + + /** + * @return {@code True} + */ + private boolean done() { + int resCnt0 = resCnt; + + return resCnt0 == Integer.MAX_VALUE || resCnt0 == futs.length; + } + + /** + * + */ + private void cancelFutures() { + SingleAddressConnectFuture[] futs = this.futs; + + if (futs != null) { + for (int i = 0; i < futs.length; i++) { + SingleAddressConnectFuture fut = futs[i]; + + fut.cancel(); + } + } + } + + /** + * @param res Result. + */ + void receivedAddressStatus(boolean res) { + if (res) { + for (;;) { + int resCnt0 = resCnt; + + if (resCnt0 == Integer.MAX_VALUE) + return; + + if (connResCntUpdater.compareAndSet(this, resCnt0, Integer.MAX_VALUE)) { + receivedConnectionStatus(nodeIdx, true); + + cancelFutures(); // Cancel others connects if they are still in progress. + + return; + } + } + } + else { + for (;;) { + int resCnt0 = resCnt; + + if (resCnt0 == Integer.MAX_VALUE) + return; + + int resCnt1 = resCnt0 + 1; + + if (connResCntUpdater.compareAndSet(this, resCnt0, resCnt1)) { + if (resCnt1 == futs.length) + receivedConnectionStatus(nodeIdx, false); + + return; + } + } + } + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationNodeConnectionCheckFuture.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationNodeConnectionCheckFuture.java new file mode 100644 index 0000000000000..cbf27b5c2af4b --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpCommunicationNodeConnectionCheckFuture.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.communication.tcp.internal; + +import java.util.UUID; + +/** + * Tcp Communication Node Connection Check Future. + */ +public interface TcpCommunicationNodeConnectionCheckFuture { + /** + * @param nodeId Remote node ID. + */ + public void onConnected(UUID nodeId); +} diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiCustomMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiCustomMessage.java index a0f9b7563d1d8..f26ad3353b417 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiCustomMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiCustomMessage.java @@ -30,12 +30,23 @@ */ public interface DiscoverySpiCustomMessage extends Serializable { /** - * Called when message passed the ring. + * Called when custom message has been handled by all nodes. + * + * @return Ack message or {@code null} if ack is not required. */ @Nullable public DiscoverySpiCustomMessage ackMessage(); /** - * @return {@code true} if message can be modified during listener notification. Changes will be send to next nodes. + * @return {@code True} if message can be modified during listener notification. Changes will be send to next nodes. */ public boolean isMutable(); + + /** + * Called on discovery coordinator node after listener is notified. If returns {@code true} + * then message is not passed to others nodes, if after this method {@link #ackMessage()} returns non-null ack + * message, it is sent to all nodes. + * + * @return {@code True} if message should not be sent to all nodes. + */ + public boolean stopProcess(); } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiMutableCustomMessageSupport.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiMutableCustomMessageSupport.java new file mode 100644 index 0000000000000..37aa3235c9909 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiMutableCustomMessageSupport.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Inherited; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * This annotation is for all implementations of {@link DiscoverySpi} that support + * topology mutable {@link DiscoverySpiCustomMessage}s. + */ +@Documented +@Inherited +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE}) +public @interface DiscoverySpiMutableCustomMessageSupport { + /** + * @return Whether or not target SPI supports mutable {@link DiscoverySpiCustomMessage}s. + */ + public boolean value(); +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java index 2d9a31407c21f..f0a5186f0ae08 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java @@ -55,6 +55,8 @@ import org.apache.ignite.configuration.AddressResolver; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpiInternalListener; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.F; @@ -88,6 +90,7 @@ import org.apache.ignite.spi.discovery.DiscoverySpiDataExchange; import org.apache.ignite.spi.discovery.DiscoverySpiHistorySupport; import org.apache.ignite.spi.discovery.DiscoverySpiListener; +import org.apache.ignite.spi.discovery.DiscoverySpiMutableCustomMessageSupport; import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator; import org.apache.ignite.spi.discovery.DiscoverySpiOrderSupport; import org.apache.ignite.spi.discovery.tcp.internal.DiscoveryDataPacket; @@ -103,6 +106,7 @@ import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCheckFailedMessage; import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryDuplicateIdMessage; import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryEnsureDelivery; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryJoinRequestMessage; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.IgniteSystemProperties.IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT; @@ -223,7 +227,8 @@ @IgniteSpiMultipleInstancesSupport(true) @DiscoverySpiOrderSupport(true) @DiscoverySpiHistorySupport(true) -public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi { +@DiscoverySpiMutableCustomMessageSupport(true) +public class TcpDiscoverySpi extends IgniteSpiAdapter implements IgniteDiscoverySpi { /** Node attribute that is mapped to node's external addresses (value is disc.tcp.ext-addrs). */ public static final String ATTR_EXT_ADDRS = "disc.tcp.ext-addrs"; @@ -409,6 +414,9 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi { /** */ protected IgniteSpiContext spiCtx; + /** */ + private IgniteDiscoverySpiInternalListener internalLsnr; + /** * Gets current SPI state. * @@ -473,6 +481,13 @@ public ClusterNode getNode0(UUID id) { /** {@inheritDoc} */ @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException { + IgniteDiscoverySpiInternalListener internalLsnr = this.internalLsnr; + + if (internalLsnr != null) { + if (!internalLsnr.beforeSendCustomEvent(this, log, msg)) + return; + } + impl.sendCustomEvent(msg); } @@ -1559,6 +1574,9 @@ protected void writeToSocket(Socket sock, OutputStream out, TcpDiscoveryAbstractMessage msg, long timeout) throws IOException, IgniteCheckedException { + if (internalLsnr != null && msg instanceof TcpDiscoveryJoinRequestMessage) + internalLsnr.beforeJoin(locNode, log); + assert sock != null; assert msg != null; assert out != null; @@ -2118,15 +2136,31 @@ boolean isSslEnabled() { return ignite().configuration().getSslContextFactory() != null; } - /** - * Force reconnect to cluster. - * - * @throws IgniteSpiException If failed. - */ - public void reconnect() throws IgniteSpiException { + /** {@inheritDoc} */ + public void clientReconnect() throws IgniteSpiException { impl.reconnect(); } + /** {@inheritDoc} */ + @Override public boolean knownNode(UUID nodeId) { + return getNode0(nodeId) != null; + } + + /** {@inheritDoc} */ + @Override public boolean clientReconnectSupported() { + return !clientReconnectDisabled; + } + + /** {@inheritDoc} */ + @Override public boolean supportsCommunicationFailureResolve() { + return false; + } + + /** {@inheritDoc} */ + @Override public void resolveCommunicationFailure(ClusterNode node, Exception err) { + throw new UnsupportedOperationException(); + } + /** * FOR TEST ONLY!!! */ @@ -2148,6 +2182,11 @@ public void addSendMessageListener(IgniteInClosure sndMsgLsnrs.add(lsnr); } + /** {@inheritDoc} */ + @Override public void setInternalListener(IgniteDiscoverySpiInternalListener lsnr) { + this.internalLsnr = lsnr; + } + /** * FOR TEST ONLY!!! */ @@ -2185,7 +2224,7 @@ public void waitForClientMessagePrecessed() { *

    * This method is intended for test purposes only. */ - protected void simulateNodeFailure() { + public void simulateNodeFailure() { impl.simulateNodeFailure(); } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java index 01534f7e43c6e..55fe4e65dc5cf 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java @@ -33,9 +33,9 @@ import org.apache.ignite.cache.CacheMetrics; import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.ClusterMetricsSnapshot; import org.apache.ignite.internal.IgniteNodeAttributes; +import org.apache.ignite.internal.managers.discovery.IgniteClusterNode; import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; @@ -58,7 +58,7 @@ * This class is not intended for public use and has been made * public due to certain limitations of Java technology. */ -public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements ClusterNode, +public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements IgniteClusterNode, Comparable, Externalizable { /** */ private static final long serialVersionUID = 0L; @@ -291,26 +291,14 @@ public Map getAttributes() { return metrics; } - /** - * Sets node metrics. - * - * @param metrics Node metrics. - */ + /** {@inheritDoc} */ public void setMetrics(ClusterMetrics metrics) { assert metrics != null; this.metrics = metrics; } - /** - * Gets collections of cache metrics for this node. Note that node cache metrics are constantly updated - * and provide up to date information about caches. - *

    - * Cache metrics are updated with some delay which is directly related to metrics update - * frequency. For example, by default the update will happen every {@code 2} seconds. - * - * @return Runtime metrics snapshots for this node. - */ + /** {@inheritDoc} */ public Map cacheMetrics() { if (metricsProvider != null) { Map cacheMetrics0 = metricsProvider.cacheMetrics(); @@ -323,11 +311,7 @@ public Map cacheMetrics() { return cacheMetrics; } - /** - * Sets node cache metrics. - * - * @param cacheMetrics Cache metrics. - */ + /** {@inheritDoc} */ public void setCacheMetrics(Map cacheMetrics) { this.cacheMetrics = cacheMetrics != null ? cacheMetrics : Collections.emptyMap(); } @@ -544,11 +528,7 @@ public TcpDiscoveryNode clientReconnectNode(Map nodeAttrs) { return node; } - /** - * Whether this node is cache client (see {@link IgniteConfiguration#isClientMode()}). - * - * @return {@code True if client}. - */ + /** {@inheritDoc} */ public boolean isCacheClient() { if (!cacheCliInit) { cacheCli = CU.clientNodeDirect(this); diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties index f0f143ddc66df..6dc3d85d592b1 100644 --- a/modules/core/src/main/resources/META-INF/classnames.properties +++ b/modules/core/src/main/resources/META-INF/classnames.properties @@ -832,6 +832,7 @@ org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPar org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader$1 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader$2 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments +org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtDemandedPartitionsMap org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtPartitionCountersMap org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtPartitionCountersMap2 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtPartitionHistorySuppliersMap @@ -1129,6 +1130,7 @@ org.apache.ignite.internal.processors.closure.GridClosureProcessor$T8 org.apache.ignite.internal.processors.closure.GridClosureProcessor$T9 org.apache.ignite.internal.processors.closure.GridClosureProcessor$TaskNoReduceAdapter org.apache.ignite.internal.processors.closure.GridPeerDeployAwareTaskAdapter +org.apache.ignite.internal.processors.cluster.ClusterNodeMetrics org.apache.ignite.internal.processors.cluster.BaselineTopology org.apache.ignite.internal.processors.cluster.BaselineTopologyHistory org.apache.ignite.internal.processors.cluster.BaselineTopologyHistoryItem diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityFunctionExcludeNeighborsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityFunctionExcludeNeighborsAbstractSelfTest.java index 900d4f54bd237..eee47c717286b 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityFunctionExcludeNeighborsAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityFunctionExcludeNeighborsAbstractSelfTest.java @@ -32,7 +32,6 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; -import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -124,12 +123,9 @@ public void testAffinityMultiNode() throws Exception { Affinity aff = g.affinity(DEFAULT_CACHE_NAME); - List top = new ArrayList<>(); + List top = new ArrayList<>(g.cluster().nodes()); - for (ClusterNode node : g.cluster().nodes()) - top.add((TcpDiscoveryNode) node); - - Collections.sort(top); + Collections.sort((List)top); assertEquals(grids, top.size()); diff --git a/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlerTriggeredTest.java b/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlerTriggeredTest.java index 4e4d75a662144..5eca7d698af7c 100644 --- a/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlerTriggeredTest.java +++ b/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlerTriggeredTest.java @@ -120,6 +120,10 @@ static class ExchangeWorkerFailureTask extends SchemaExchangeWorkerTask implemen @Override public boolean isMutable() { return false; } + + @Override public boolean stopProcess() { + return false; + } }); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupHostsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupHostsSelfTest.java index 2328c84df6b7d..141f4af7f9194 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupHostsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupHostsSelfTest.java @@ -61,6 +61,9 @@ public class ClusterGroupHostsSelfTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testForHosts() throws Exception { + if (!tcpDiscovery()) + return; + Ignite ignite = grid(); assertEquals(1, ignite.cluster().forHost("h_1").nodes().size()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java index 9df561ac2f099..99006d1f99f91 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java @@ -68,6 +68,8 @@ public class ClusterGroupSelfTest extends ClusterGroupAbstractTest { if (i == 0) ignite = g; } + + waitForTopology(NODES_CNT); } finally { Ignition.setClientMode(false); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsUpdateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsUpdateTest.java new file mode 100644 index 0000000000000..6e6b4a4fd8344 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/ClusterNodeMetricsUpdateTest.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import junit.framework.AssertionFailedError; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCompute; +import org.apache.ignite.Ignition; +import org.apache.ignite.cluster.ClusterGroup; +import org.apache.ignite.cluster.ClusterMetrics; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * + */ +public class ClusterNodeMetricsUpdateTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private boolean client; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER); + + cfg.setMetricsUpdateFrequency(500); + + cfg.setClientMode(client); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testMetrics() throws Exception { + int NODES = 6; + + Ignite srv0 = startGridsMultiThreaded(NODES / 2); + + client = true; + + startGridsMultiThreaded(NODES / 2, NODES / 2); + + Map expJobs = new HashMap<>(); + + for (int i = 0; i < NODES; i++) + expJobs.put(nodeId(i), 0); + + checkMetrics(NODES, expJobs); + + for (int i = 0; i < NODES; i++) { + UUID nodeId = nodeId(i); + + IgniteCompute c = srv0.compute(srv0.cluster().forNodeId(nodeId(i))); + + c.call(new DummyCallable(null)); + + expJobs.put(nodeId, 1); + } + } + + /** + * @param expNodes Expected nodes. + * @param expJobs Expected jobs number per node. + */ + private void checkMetrics0(int expNodes, Map expJobs) { + List nodes = Ignition.allGrids(); + + assertEquals(expNodes, nodes.size()); + assertEquals(expNodes, expJobs.size()); + + int totalJobs = 0; + + for (Integer c : expJobs.values()) + totalJobs += c; + + for (final Ignite ignite : nodes) { + ClusterMetrics m = ignite.cluster().metrics(); + + assertEquals(expNodes, m.getTotalNodes()); + assertEquals(totalJobs, m.getTotalExecutedJobs()); + + for (Map.Entry e : expJobs.entrySet()) { + UUID nodeId = e.getKey(); + + ClusterGroup g = ignite.cluster().forNodeId(nodeId); + + ClusterMetrics nodeM = g.metrics(); + + assertEquals(e.getValue(), (Integer)nodeM.getTotalExecutedJobs()); + } + } + } + + /** + * @param expNodes Expected nodes. + * @param expJobs Expected jobs number per node. + * @throws Exception If failed. + */ + private void checkMetrics(final int expNodes, final Map expJobs) throws Exception { + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + try { + checkMetrics0(expNodes, expJobs); + } + catch (AssertionFailedError e) { + return false; + } + + return true; + } + }, 5000); + + checkMetrics0(expNodes, expJobs); + } + + /** + * + */ + private static class DummyCallable implements IgniteCallable { + /** */ + private byte[] data; + + /** + * @param data Data. + */ + DummyCallable(byte[] data) { + this.data = data; + } + + /** {@inheritDoc} */ + @Override public Object call() throws Exception { + return data; + } + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/DiscoverySpiTestListener.java b/modules/core/src/test/java/org/apache/ignite/internal/DiscoverySpiTestListener.java new file mode 100644 index 0000000000000..46d9edc6854b7 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/DiscoverySpiTestListener.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpiInternalListener; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.spi.discovery.DiscoverySpi; +import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; +import org.apache.ignite.testframework.GridTestUtils; + +/** + * Test callback for discovery SPI. + *

    + * Allows block/delay node join and custom event sending. + */ +public class DiscoverySpiTestListener implements IgniteDiscoverySpiInternalListener { + /** */ + private volatile CountDownLatch joinLatch; + + /** */ + private Set> blockCustomEvtCls; + + /** */ + private final Object mux = new Object(); + + /** */ + private List blockedMsgs = new ArrayList<>(); + + /** */ + private volatile DiscoverySpi spi; + + /** */ + private volatile IgniteLogger log; + + /** + * + */ + public void startBlockJoin() { + joinLatch = new CountDownLatch(1); + } + + /** + * + */ + public void stopBlockJoin() { + joinLatch.countDown(); + } + + /** {@inheritDoc} */ + @Override public void beforeJoin(ClusterNode locNode, IgniteLogger log) { + try { + CountDownLatch writeLatch0 = joinLatch; + + if (writeLatch0 != null) { + log.info("Block join"); + + U.await(writeLatch0); + } + } + catch (Exception e) { + throw new IgniteException(e); + } + } + + /** {@inheritDoc} */ + @Override public boolean beforeSendCustomEvent(DiscoverySpi spi, IgniteLogger log, DiscoverySpiCustomMessage msg) { + this.spi = spi; + this.log = log; + + synchronized (mux) { + if (blockCustomEvtCls != null) { + DiscoveryCustomMessage msg0 = GridTestUtils.getFieldValue(msg, "delegate"); + + if (blockCustomEvtCls.contains(msg0.getClass())) { + log.info("Block custom message: " + msg0); + + blockedMsgs.add(msg); + + mux.notifyAll(); + + return false; + } + } + } + + return true; + } + + /** + * @param blockCustomEvtCls Event class to block. + */ + public void blockCustomEvent(Class cls0, Class ... blockCustomEvtCls) { + synchronized (mux) { + assert blockedMsgs.isEmpty() : blockedMsgs; + + this.blockCustomEvtCls = new HashSet<>(); + + this.blockCustomEvtCls.add(cls0); + + Collections.addAll(this.blockCustomEvtCls, blockCustomEvtCls); + } + } + + /** + * @throws InterruptedException If interrupted. + */ + public void waitCustomEvent() throws InterruptedException { + synchronized (mux) { + while (blockedMsgs.isEmpty()) + mux.wait(); + } + } + + /** + * + */ + public void stopBlockCustomEvents() { + if (spi == null) + return; + + List msgs; + + synchronized (this) { + msgs = new ArrayList<>(blockedMsgs); + + blockCustomEvtCls = null; + + blockedMsgs.clear(); + } + + for (DiscoverySpiCustomMessage msg : msgs) { + log.info("Resend blocked message: " + msg); + + spi.sendCustomEvent(msg); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java index e6b678b313a7c..883d677d645af 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java @@ -49,6 +49,7 @@ import org.jetbrains.annotations.Nullable; import static java.util.concurrent.TimeUnit.MINUTES; +import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; import static org.apache.ignite.events.EventType.EVT_NODE_JOINED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.lang.IgniteProductVersion.fromString; @@ -158,10 +159,10 @@ public void testDiscoveryListener() throws Exception { final AtomicInteger cnt = new AtomicInteger(); - /** Joined nodes counter. */ + // Joined nodes counter. final CountDownLatch joinedCnt = new CountDownLatch(NODES_CNT); - /** Left nodes counter. */ + // Left nodes counter. final CountDownLatch leftCnt = new CountDownLatch(NODES_CNT); IgnitePredicate lsnr = new IgnitePredicate() { @@ -171,7 +172,7 @@ public void testDiscoveryListener() throws Exception { joinedCnt.countDown(); } - else if (EVT_NODE_LEFT == evt.type()) { + else if (EVT_NODE_LEFT == evt.type() || EVT_NODE_FAILED == evt.type()) { int i = cnt.decrementAndGet(); assert i >= 0; @@ -185,7 +186,10 @@ else if (EVT_NODE_LEFT == evt.type()) { } }; - ignite.events().localListen(lsnr, EVT_NODE_LEFT, EVT_NODE_JOINED); + int[] evts = tcpDiscovery() ? new int[]{EVT_NODE_LEFT, EVT_NODE_JOINED} : + new int[]{EVT_NODE_LEFT, EVT_NODE_FAILED, EVT_NODE_JOINED}; + + ignite.events().localListen(lsnr, evts); try { for (int i = 0; i < NODES_CNT; i++) @@ -242,6 +246,8 @@ public void testCacheNodes() throws Exception { for (int i = 0; i < NODES_CNT; i++) stopGrid(i); + waitForTopology(1); + final long topVer = discoMgr.topologyVersion(); assert topVer == topVer0 + NODES_CNT * 2 : "Unexpected topology version: " + topVer; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java index cd6b2c081d2d4..a8be5419314df 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java @@ -259,6 +259,8 @@ public void testCannotSendJobExecuteResponse() throws Exception { // Now we stop master grid. stopGrid(lastGridIdx, true); + waitForTopology(GRID_CNT - 1); + // Release communication SPI wait latches. As master node is stopped, job worker will receive and exception. for (int i = 0; i < lastGridIdx; i++) ((CommunicationSpi)grid(i).configuration().getCommunicationSpi()).releaseWaitLatch(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridJobStealingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridJobStealingSelfTest.java index f3a19aaeccb4f..6824d51bcb17e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridJobStealingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridJobStealingSelfTest.java @@ -187,6 +187,8 @@ public void testProjectionPredicate() throws Exception { public void testProjectionPredicateInternalStealing() throws Exception { final Ignite ignite3 = startGrid(3); + waitForTopology(3); + final UUID node1 = ignite1.cluster().localNode().id(); final UUID node3 = ignite3.cluster().localNode().id(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridSameVmStartupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridSameVmStartupSelfTest.java index 66e9cf43b3297..a04c38e7a7590 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridSameVmStartupSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridSameVmStartupSelfTest.java @@ -20,6 +20,7 @@ import java.util.Collection; import java.util.UUID; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import org.apache.ignite.Ignite; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.events.DiscoveryEvent; @@ -75,8 +76,10 @@ public void testSameVmStartup() throws Exception { ignite2.events().localListen(new IgnitePredicate() { @Override public boolean apply(Event evt) { - assert evt.type() != EVT_NODE_FAILED : - "Node1 did not exit gracefully."; + boolean tcpDiscovery = tcpDiscovery(); + + if (tcpDiscovery) + assert evt.type() != EVT_NODE_FAILED : "Node1 did not exit gracefully."; if (evt instanceof DiscoveryEvent) { // Local node can send METRICS_UPDATED event. @@ -86,8 +89,14 @@ public void testSameVmStartup() throws Exception { ((DiscoveryEvent) evt).eventNode().id() + ", expected=" + grid1LocNodeId + ", type=" + evt.type() + ']'; - if (evt.type() == EVT_NODE_LEFT) - latch.countDown(); + if (tcpDiscovery) { + if (evt.type() == EVT_NODE_LEFT) + latch.countDown(); + } + else { + if (evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED) + latch.countDown(); + } } return true; @@ -96,7 +105,7 @@ public void testSameVmStartup() throws Exception { stopGrid(1); - latch.await(); + assertTrue(latch.await(10, TimeUnit.SECONDS)); Collection top2 = ignite2.cluster().forRemotes().nodes(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java index 7e368cb51717c..f71ffb0ea97d2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java @@ -45,6 +45,8 @@ public class GridSelfTest extends ClusterGroupAbstractTest { for (int i = 0; i < NODES_CNT; i++) startGrid(i); + + waitForTopology(NODES_CNT); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java index fa9cc35313b9a..e68ea1306d0bf 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.io.OutputStream; import java.net.Socket; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -38,6 +39,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.events.Event; import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteFuture; @@ -141,6 +143,14 @@ protected static TestTcpDiscoverySpi spi(Ignite ignite) { return ((TestTcpDiscoverySpi)ignite.configuration().getDiscoverySpi()); } + /** + * @param ignite Node. + * @return Discovery SPI. + */ + protected static IgniteDiscoverySpi spi0(Ignite ignite) { + return ((IgniteDiscoverySpi)ignite.configuration().getDiscoverySpi()); + } + /** * @param ignite Node. * @return Communication SPI. @@ -185,16 +195,28 @@ protected BlockTcpCommunicationSpi commSpi(Ignite ignite) { * @return Server node client connected to. */ protected Ignite clientRouter(Ignite client) { - TcpDiscoveryNode node = (TcpDiscoveryNode)client.cluster().localNode(); + if (tcpDiscovery()) { + TcpDiscoveryNode node = (TcpDiscoveryNode)client.cluster().localNode(); + + assertTrue(node.isClient()); + assertNotNull(node.clientRouterNodeId()); - assertTrue(node.isClient()); - assertNotNull(node.clientRouterNodeId()); + Ignite srv = G.ignite(node.clientRouterNodeId()); - Ignite srv = G.ignite(node.clientRouterNodeId()); + assertNotNull(srv); + + return srv; + } + else { + for (Ignite node : G.allGrids()) { + if (!node.cluster().localNode().isClient()) + return node; + } - assertNotNull(srv); + fail(); - return srv; + return null; + } } /** @@ -251,15 +273,24 @@ protected static void reconnectClientNodes(final IgniteLogger log, List clients, Ignite srv, @Nullable Runnable disconnectedC) throws Exception { - final TestTcpDiscoverySpi srvSpi = spi(srv); + final IgniteDiscoverySpi srvSpi = spi0(srv); final CountDownLatch disconnectLatch = new CountDownLatch(clients.size()); final CountDownLatch reconnectLatch = new CountDownLatch(clients.size()); log.info("Block reconnect."); - for (Ignite client : clients) - spi(client).writeLatch = new CountDownLatch(1); + List blockLsnrs = new ArrayList<>(); + + for (Ignite client : clients) { + DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener(); + + lsnr.startBlockJoin(); + + blockLsnrs.add(lsnr); + + spi0(client).setInternalListener(lsnr); + } IgnitePredicate p = new IgnitePredicate() { @Override public boolean apply(Event evt) { @@ -291,8 +322,8 @@ else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) { log.info("Allow reconnect."); - for (Ignite client : clients) - spi(client).writeLatch.countDown(); + for (DiscoverySpiTestListener blockLsnr : blockLsnrs) + blockLsnr.stopBlockJoin(); waitReconnectEvent(log, reconnectLatch); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectApiExceptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectApiExceptionTest.java index 06bde99de673d..43da2d15a7d0e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectApiExceptionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectApiExceptionTest.java @@ -44,6 +44,7 @@ import org.apache.ignite.configuration.CollectionConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.events.Event; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; import org.apache.ignite.internal.util.typedef.C1; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.U; @@ -51,6 +52,7 @@ import org.apache.ignite.lang.IgniteCallable; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.spi.discovery.DiscoverySpi; import org.apache.ignite.testframework.GridTestUtils; import static java.util.concurrent.TimeUnit.SECONDS; @@ -99,7 +101,7 @@ public void testErrorOnDisconnect() throws Exception { * @throws Exception If failed. */ @SuppressWarnings("unchecked") - public void dataStructureOperationsTest() throws Exception { + private void dataStructureOperationsTest() throws Exception { clientMode = true; final Ignite client = startGrid(serverCount()); @@ -219,7 +221,7 @@ public void dataStructureOperationsTest() throws Exception { * @throws Exception If failed. */ @SuppressWarnings("unchecked") - public void cacheOperationsTest() throws Exception { + private void cacheOperationsTest() throws Exception { clientMode = true; final Ignite client = startGrid(serverCount()); @@ -537,7 +539,7 @@ public void cacheOperationsTest() throws Exception { * @throws Exception If failed. */ @SuppressWarnings("unchecked") - public void igniteOperationsTest() throws Exception { + private void igniteOperationsTest() throws Exception { clientMode = true; final Ignite client = startGrid(serverCount()); @@ -775,11 +777,11 @@ private void doTestIgniteOperationOnDisconnect(Ignite client, final List futs = new ArrayList<>(); @@ -832,7 +837,7 @@ else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) { log.info("Allow reconnect."); - clientSpi.writeLatch.countDown(); + lsnr.stopBlockJoin(); waitReconnectEvent(reconnectLatch); @@ -857,7 +862,7 @@ else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) { } } finally { - clientSpi.writeLatch.countDown(); + lsnr.stopBlockJoin(); for (IgniteInternalFuture fut : futs) fut.cancel(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java index 00daf5feb90a4..d1e3ade29b70a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java @@ -111,7 +111,7 @@ public void testAtomicSeqReconnect() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); IgniteAtomicSequence clientAtomicSeq = client.atomicSequence("atomicSeq", 0, true); @@ -144,7 +144,7 @@ public void testAtomicSeqReconnectRemoved() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); final IgniteAtomicSequence clientAtomicSeq = client.atomicSequence("atomicSeqRmv", 0, true); @@ -192,7 +192,7 @@ public void testAtomicSeqReconnectInProgress() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); BlockTcpCommunicationSpi commSpi = commSpi(srv); @@ -253,7 +253,7 @@ public void testAtomicReferenceReconnect() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); IgniteAtomicReference clientAtomicRef = client.atomicReference("atomicRef", "1st value", true); @@ -294,7 +294,7 @@ public void testAtomicReferenceReconnectRemoved() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); final IgniteAtomicReference clientAtomicRef = client.atomicReference("atomicRefRemoved", "1st value", true); @@ -347,7 +347,7 @@ public void testAtomicReferenceReconnectInProgress() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); final IgniteAtomicReference clientAtomicRef = client.atomicReference("atomicRefInProg", "1st value", true); @@ -414,7 +414,7 @@ public void testAtomicStampedReconnect() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); IgniteAtomicStamped clientAtomicStamped = client.atomicStamped("atomicStamped", 0, 0, true); @@ -455,7 +455,7 @@ public void testAtomicStampedReconnectRemoved() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); final IgniteAtomicStamped clientAtomicStamped = client.atomicStamped("atomicStampedRemoved", 0, 0, true); @@ -506,7 +506,7 @@ public void testAtomicStampedReconnectInProgress() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); final IgniteAtomicStamped clientAtomicStamped = client.atomicStamped("atomicStampedInProgress", 0, 0, true); @@ -574,7 +574,7 @@ public void testAtomicLongReconnect() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); IgniteAtomicLong clientAtomicLong = client.atomicLong("atomicLong", 0, true); @@ -605,7 +605,7 @@ public void testAtomicLongReconnectRemoved() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); final IgniteAtomicLong clientAtomicLong = client.atomicLong("atomicLongRmv", 0, true); @@ -646,7 +646,7 @@ public void testAtomicLongReconnectInProgress() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); BlockTcpCommunicationSpi commSpi = commSpi(srv); @@ -701,7 +701,7 @@ public void testLatchReconnect() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); IgniteCountDownLatch clientLatch = client.countDownLatch("latch1", 3, false, true); @@ -742,7 +742,7 @@ public void testSemaphoreReconnect() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); IgniteSemaphore clientSemaphore = client.semaphore("semaphore1", 3, false, true); @@ -789,7 +789,7 @@ private void testReentrantLockReconnect(final boolean fair) throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); IgniteLock clientLock = client.reentrantLock("lock1", true, fair, true); 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 518e674d6c052..3cb82e07cab14 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java @@ -49,6 +49,7 @@ import org.apache.ignite.events.Event; import org.apache.ignite.internal.managers.communication.GridIoMessage; import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetResponse; @@ -67,6 +68,7 @@ import org.apache.ignite.resources.LoggerResource; import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.discovery.DiscoverySpi; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.transactions.Transaction; @@ -155,11 +157,11 @@ public void testReconnect() throws Exception { IgniteEx client = startGrid(SRV_CNT); - final TestTcpDiscoverySpi clientSpi = spi(client); + final IgniteDiscoverySpi clientSpi = spi0(client); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); - TestTcpDiscoverySpi srvSpi = spi(srv); + DiscoverySpi srvSpi = ignite(0).configuration().getDiscoverySpi(); final IgniteCache cache = client.getOrCreateCache(new CacheConfiguration<>(DEFAULT_CACHE_NAME)); @@ -188,7 +190,11 @@ public void testReconnect() throws Exception { log.info("Block reconnect."); - clientSpi.writeLatch = new CountDownLatch(1); + DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener(); + + clientSpi.setInternalListener(lsnr); + + lsnr.startBlockJoin(); final AtomicReference blockPutRef = new AtomicReference<>(); @@ -254,7 +260,7 @@ else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) { log.info("Allow reconnect."); - clientSpi.writeLatch.countDown(); + lsnr.stopBlockJoin(); assertTrue(reconnectLatch.await(5000, MILLISECONDS)); @@ -319,7 +325,7 @@ public void testReconnectTransactions() throws Exception { IgniteEx client = startGrid(SRV_CNT); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); CacheConfiguration ccfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME); @@ -412,17 +418,21 @@ private void reconnectTransactionInProgress1(IgniteEx client, final TransactionConcurrency txConcurrency, final IgniteCache cache) throws Exception { - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); - final TestTcpDiscoverySpi clientSpi = spi(client); - final TestTcpDiscoverySpi srvSpi = spi(srv); + final IgniteDiscoverySpi clientSpi = spi0(client); + final DiscoverySpi srvSpi = spi0(srv); final CountDownLatch disconnectLatch = new CountDownLatch(1); final CountDownLatch reconnectLatch = new CountDownLatch(1); log.info("Block reconnect."); - clientSpi.writeLatch = new CountDownLatch(1); + DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener(); + + clientSpi.setInternalListener(lsnr); + + lsnr.startBlockJoin(); client.events().localListen(new IgnitePredicate() { @Override public boolean apply(Event evt) { @@ -530,7 +540,7 @@ else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) { assertTrue(putFailed.await(5000, MILLISECONDS)); - clientSpi.writeLatch.countDown(); + lsnr.stopBlockJoin(); waitReconnectEvent(reconnectLatch); @@ -604,9 +614,9 @@ public void testReconnectExchangeInProgress() throws Exception { IgniteEx client = startGrid(SRV_CNT); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); - TestTcpDiscoverySpi srvSpi = spi(srv); + DiscoverySpi srvSpi = spi0(srv); TestCommunicationSpi coordCommSpi = (TestCommunicationSpi)grid(0).configuration().getCommunicationSpi(); @@ -691,7 +701,7 @@ public void testReconnectInitialExchangeInProgress() throws Exception { IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { @Override public Boolean call() throws Exception { try { - Ignition.start(optimize(getConfiguration(getTestIgniteInstanceName(SRV_CNT)))); + startGrid(optimize(getConfiguration(getTestIgniteInstanceName(SRV_CNT)))); // Commented due to IGNITE-4473, because // IgniteClientDisconnectedException won't @@ -722,7 +732,7 @@ public void testReconnectInitialExchangeInProgress() throws Exception { } }); - TestTcpDiscoverySpi srvSpi = spi(srv); + DiscoverySpi srvSpi = spi0(srv); try { if (!joinLatch.await(10_000, MILLISECONDS)) { @@ -1256,30 +1266,35 @@ else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) { * */ static class TestClass1 implements Serializable { + // No-op. } /** * */ static class TestClass2 implements Serializable { + // No-op. } /** * */ static class TestClass3 implements Serializable { + // No-op. } /** * */ static class TestClass4 implements Serializable { + // No-op. } /** * */ static class TestClass5 implements Serializable { + // No-op. } /** @@ -1294,11 +1309,11 @@ private void checkOperationInProgressFails(final IgniteEx client, Class msgToBlock, final IgniteInClosure> c) throws Exception { - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); final UUID id = client.localNode().id(); - TestTcpDiscoverySpi srvSpi = spi(srv); + DiscoverySpi srvSpi = spi0(srv); final IgniteCache cache = client.getOrCreateCache(ccfg); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java index 3f0e33d3046ab..5be59b0537ea2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java @@ -180,7 +180,7 @@ public void testServerReconnect() throws Exception { private void serverNodeReconnect(CollectionConfiguration colCfg) throws Exception { final Ignite client = grid(serverCount()); - final Ignite srv = clientRouter(client); + final Ignite srv = ignite(0); assertNotNull(srv.queue("q", 0, colCfg)); assertNotNull(srv.set("s", colCfg)); @@ -201,7 +201,7 @@ private void setReconnect(CollectionConfiguration colCfg) throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); final String setName = "set-" + colCfg.getAtomicityMode(); @@ -235,7 +235,7 @@ private void setReconnectRemove(CollectionConfiguration colCfg) throws Exception assertTrue(client.cluster().localNode().isClient()); - final Ignite srv = clientRouter(client); + final Ignite srv = ignite(0); final String setName = "set-rm-" + colCfg.getAtomicityMode(); @@ -281,7 +281,7 @@ private void setReconnectInProgress(final CollectionConfiguration colCfg) throws assertTrue(client.cluster().localNode().isClient()); - final Ignite srv = clientRouter(client); + final Ignite srv = ignite(0); final String setName = "set-in-progress-" + colCfg.getAtomicityMode(); @@ -347,7 +347,7 @@ private void queueReconnect(CollectionConfiguration colCfg) throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); final String setName = "queue-" + colCfg.getAtomicityMode(); @@ -379,7 +379,7 @@ private void queueReconnectRemoved(CollectionConfiguration colCfg) throws Except assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); final String setName = "queue-rmv" + colCfg.getAtomicityMode(); @@ -423,7 +423,7 @@ private void queueReconnectInProgress(final CollectionConfiguration colCfg) thro assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); final String setName = "queue-rmv" + colCfg.getAtomicityMode(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectComputeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectComputeTest.java index cce0c7e3489f9..57d31882db6d7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectComputeTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectComputeTest.java @@ -49,7 +49,7 @@ public void testReconnectAffinityCallInProgress() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); IgniteCache cache = client.getOrCreateCache("test-cache"); @@ -103,7 +103,7 @@ public void testReconnectBroadcastInProgress() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); BlockTcpCommunicationSpi commSpi = commSpi(srv); @@ -152,7 +152,7 @@ public void testReconnectApplyInProgress() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); BlockTcpCommunicationSpi commSpi = commSpi(srv); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java index ca0d88974111e..d68fc1cd3cceb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java @@ -28,6 +28,7 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.events.Event; import org.apache.ignite.events.EventType; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; import org.apache.ignite.internal.util.typedef.P2; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.lang.IgniteRunnable; @@ -61,9 +62,9 @@ public void testEventListenerReconnect() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); - TestTcpDiscoverySpi srvSpi = spi(srv); + IgniteDiscoverySpi srvSpi = spi0(srv); EventListener lsnr = new EventListener(); @@ -133,9 +134,9 @@ private void testMessageListenerReconnect(boolean stopFromClient) throws Excepti assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); - TestTcpDiscoverySpi srvSpi = spi(srv); + IgniteDiscoverySpi srvSpi = spi0(srv); final String topic = "testTopic"; @@ -309,9 +310,9 @@ private void continuousQueryReconnect(Ignite client, CacheEventListener lsnr) throws Exception { - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); - TestTcpDiscoverySpi srvSpi = spi(srv); + IgniteDiscoverySpi srvSpi = spi0(srv); final CountDownLatch reconnectLatch = new CountDownLatch(1); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectDiscoveryStateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectDiscoveryStateTest.java index c071ee2ae68e5..6e77742b758f6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectDiscoveryStateTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectDiscoveryStateTest.java @@ -27,6 +27,7 @@ import org.apache.ignite.events.Event; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.spi.discovery.DiscoverySpi; import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED; import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_RECONNECTED; @@ -64,20 +65,23 @@ public void testReconnect() throws Exception { nodeCnt.put(1, 1); nodeCnt.put(2, 2); nodeCnt.put(3, 3); - nodeCnt.put(4, 4); - for (Map.Entry e : nodeCnt.entrySet()) { - Collection nodes = cluster.topology(e.getKey()); + if (tcpDiscovery()) { + nodeCnt.put(4, 4); - assertNotNull("No nodes for topology: " + e.getKey(), nodes); - assertEquals((int)e.getValue(), nodes.size()); + for (Map.Entry e : nodeCnt.entrySet()) { + Collection nodes = cluster.topology(e.getKey()); + + assertNotNull("No nodes for topology: " + e.getKey(), nodes); + assertEquals((int)e.getValue(), nodes.size()); + } } ClusterNode locNode = cluster.localNode(); assertEquals(topVer, locNode.order()); - TestTcpDiscoverySpi srvSpi = spi(clientRouter(client)); + DiscoverySpi srvSpi = ignite(0).configuration().getDiscoverySpi(); final CountDownLatch reconnectLatch = new CountDownLatch(1); @@ -112,7 +116,11 @@ else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) { assertEquals(topVer, locNode.order()); assertEquals(topVer, cluster.topologyVersion()); - nodeCnt.put(5, 3); + if (tcpDiscovery()) + nodeCnt.put(5, 3); + else + nodeCnt.clear(); + nodeCnt.put(6, 4); for (Map.Entry e : nodeCnt.entrySet()) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverAbstractTest.java index 3e98051efb0ee..37292ff12b342 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverAbstractTest.java @@ -29,6 +29,7 @@ import org.apache.ignite.IgniteClientDisconnectedException; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.events.Event; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; @@ -87,9 +88,9 @@ protected final void reconnectFailover(final Callable c) throws Exception assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); - TestTcpDiscoverySpi srvSpi = spi(srv); + IgniteDiscoverySpi srvSpi = spi0(srv); final AtomicBoolean stop = new AtomicBoolean(false); @@ -209,14 +210,17 @@ else if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) { } if (err != null) { - log.error(err); + log.error("Test error: " + err); U.dumpThreads(log); CyclicBarrier barrier0 = barrier; - if (barrier0 != null) + if (barrier0 != null) { + barrier = null; + barrier0.reset(); + } stop.set(true); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectServicesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectServicesTest.java index 3e961e537eccb..1e6dd64f970b2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectServicesTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectServicesTest.java @@ -65,7 +65,7 @@ public void testReconnect() throws Exception { assertEquals((Object)topVer, srvc.test()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); reconnectClientNode(client, srv, null); @@ -88,7 +88,7 @@ public void testServiceRemove() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); IgniteServices clnServices = client.services(); @@ -132,7 +132,7 @@ public void testReconnectInDeploying() throws Exception { final IgniteServices services = client.services(); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); BlockTcpCommunicationSpi commSpi = commSpi(srv); @@ -179,7 +179,7 @@ public void testReconnectInProgress() throws Exception { final IgniteServices services = client.services(); - final Ignite srv = clientRouter(client); + final Ignite srv = ignite(0); services.deployClusterSingleton("testReconnectInProgress", new TestServiceImpl()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStopTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStopTest.java index e863cdfcb8655..b5c3ee86d0d4a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStopTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStopTest.java @@ -23,8 +23,10 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.events.Event; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.spi.discovery.DiscoverySpi; import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED; import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_RECONNECTED; @@ -50,15 +52,19 @@ public void testStopWhenDisconnected() throws Exception { Ignite srv = clientRouter(client); - TestTcpDiscoverySpi srvSpi = spi(srv); + DiscoverySpi srvSpi = spi0(srv); final CountDownLatch disconnectLatch = new CountDownLatch(1); final CountDownLatch reconnectLatch = new CountDownLatch(1); - final TestTcpDiscoverySpi clientSpi = spi(client); + final IgniteDiscoverySpi clientSpi = spi0(client); + + DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener(); + + clientSpi.setInternalListener(lsnr); log.info("Block reconnect."); - clientSpi.writeLatch = new CountDownLatch(1); + lsnr.startBlockJoin(); client.events().localListen(new IgnitePredicate() { @Override public boolean apply(Event evt) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStreamerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStreamerTest.java index 3959feb80804d..36b989093440e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStreamerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStreamerTest.java @@ -71,7 +71,7 @@ public void testStreamerReconnect() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); final IgniteCache srvCache = srv.cache(CACHE_NAME); @@ -135,7 +135,7 @@ public void testStreamerReconnectInProgress() throws Exception { assertTrue(client.cluster().localNode().isClient()); - Ignite srv = clientRouter(client); + Ignite srv = ignite(0); final IgniteCache srvCache = srv.cache(CACHE_NAME); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java index a5d42e9a12cba..8edbb52babae1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java @@ -256,6 +256,9 @@ public void testClientsReconnectDisabled() throws Exception { Ignite srv1 = startGrid("server1"); + if (!tcpDiscovery()) + return; + crd = ((IgniteKernal)srv1).localNode(); Ignite srv2 = startGrid("server2"); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java index a8afa8b928a4f..8fad6404f1936 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java @@ -178,16 +178,20 @@ public void testAlivesClient() throws Exception { * Waits while topology on all nodes became equals to the expected size. * * @param nodesCnt Expected nodes count. - * @throws InterruptedException If interrupted. + * @throws Exception If interrupted. */ @SuppressWarnings("BusyWait") - private void awaitDiscovery(long nodesCnt) throws InterruptedException { - for (Ignite g : alive) { - ((TcpDiscoverySpi)g.configuration().getDiscoverySpi()).waitForClientMessagePrecessed(); + private void awaitDiscovery(int nodesCnt) throws Exception { + if (tcpDiscovery()) { + for (Ignite g : alive) { + ((TcpDiscoverySpi)g.configuration().getDiscoverySpi()).waitForClientMessagePrecessed(); - while (g.cluster().nodes().size() != nodesCnt) - Thread.sleep(10); + while (g.cluster().nodes().size() != nodesCnt) + Thread.sleep(10); + } } + else + waitForTopology(nodesCnt); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java index 1d70246993d36..aa2abaed7cd44 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java @@ -91,12 +91,12 @@ public abstract class GridAffinityProcessorAbstractSelfTest extends GridCommonAb @Override protected void beforeTestsStarted() throws Exception { assert NODES_CNT >= 1; - withCache = false; + withCache = true; for (int i = 0; i < NODES_CNT; i++) startGrid(i); - withCache = true; + withCache = false; for (int i = NODES_CNT; i < 2 * NODES_CNT; i++) startGrid(i); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java index aefbc23d28f28..b8f9d70dc5dde 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java @@ -27,9 +27,9 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.events.Event; import org.apache.ignite.events.EventType; +import org.apache.ignite.internal.managers.discovery.IgniteClusterNode; import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgnitePredicate; -import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import static org.apache.ignite.events.EventType.EVT_NODE_METRICS_UPDATED; @@ -103,7 +103,7 @@ public void testMetricsStatisticsEnabled() throws Exception { Collection nodes = grid(0).cluster().forRemotes().nodes(); for (ClusterNode node : nodes) { - Map metrics = ((TcpDiscoveryNode)node).cacheMetrics(); + Map metrics = ((IgniteClusterNode)node).cacheMetrics(); assertNotNull(metrics); assertFalse(metrics.isEmpty()); } @@ -118,6 +118,8 @@ public void testMetricsStatisticsEnabled() throws Exception { /** * Test cluster group metrics in case of statistics disabled. + * + * @throws Exception If failed. */ public void testMetricsStatisticsDisabled() throws Exception { createCaches(false); @@ -134,7 +136,7 @@ public void testMetricsStatisticsDisabled() throws Exception { Collection nodes = grid(0).cluster().forRemotes().nodes(); for (ClusterNode node : nodes) { - Map metrics = ((TcpDiscoveryNode) node).cacheMetrics(); + Map metrics = ((IgniteClusterNode)node).cacheMetrics(); assertNotNull(metrics); assertTrue(metrics.isEmpty()); } @@ -172,7 +174,9 @@ private void destroyCaches() { } /** - * Wait for {@link EventType#EVT_NODE_METRICS_UPDATED} event will be receieved. + * Wait for {@link EventType#EVT_NODE_METRICS_UPDATED} event will be received. + * + * @throws InterruptedException If interrupted. */ private void awaitMetricsUpdate() throws InterruptedException { final CountDownLatch latch = new CountDownLatch((GRID_CNT + 1) * 2); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java index df93ae4aab2c3..299dbf4a6eb69 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java @@ -115,6 +115,8 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest { protected void initStoreStrategy() throws IgniteCheckedException { if (storeStgy == null) storeStgy = isMultiJvm() ? new H2CacheStoreStrategy() : new MapCacheStoreStrategy(); + else if (isMultiJvm() && !(storeStgy instanceof H2CacheStoreStrategy)) + storeStgy = new H2CacheStoreStrategy(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java index 0069110662ac4..c135f2df9557a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java @@ -45,7 +45,9 @@ public class IgniteCacheNearLockValueSelfTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { - startGridsMultiThreaded(2); + startGrid(1); + + startGrid(0); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java index f3214532410df..55ff31a62ed17 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java @@ -66,6 +66,17 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes return null; } + /** {@inheritDoc} */ + @Override protected void startGrids() throws Exception { + int cnt = gridCount(); + + assert cnt >= 1 : "At least one grid must be started"; + + startGridsMultiThreaded(1, cnt - 1); + + startGrid(0); + } + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java index 2337329c5ce83..838e56deebdd3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java @@ -38,6 +38,7 @@ import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.TestRecordingCommunicationSpi; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage; @@ -1108,6 +1109,70 @@ private void stateChangeFailover2(boolean activate) throws Exception { checkCaches1(10); } + /** + * @throws Exception If failed. + */ + public void testActivateFailover3() throws Exception { + stateChangeFailover3(true); + } + + /** + * @throws Exception If failed. + */ + public void testDeactivateFailover3() throws Exception { + stateChangeFailover3(false); + } + + /** + * @param activate If {@code true} tests activation, otherwise deactivation. + * @throws Exception If failed. + */ + private void stateChangeFailover3(boolean activate) throws Exception { + testReconnectSpi = true; + + startNodesAndBlockStatusChange(4, 0, 0, !activate); + + client = false; + + IgniteInternalFuture startFut1 = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + startGrid(4); + + return null; + } + }, "start-node1"); + + IgniteInternalFuture startFut2 = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + startGrid(5); + + return null; + } + }, "start-node2"); + + U.sleep(1000); + + // Stop all nodes participating in state change and not allow last node to finish exchange. + for (int i = 0; i < 4; i++) + ((IgniteDiscoverySpi)ignite(i).configuration().getDiscoverySpi()).simulateNodeFailure(); + + for (int i = 0; i < 4; i++) + stopGrid(getTestIgniteInstanceName(i), true, false); + + startFut1.get(); + startFut2.get(); + + assertFalse(ignite(4).active()); + assertFalse(ignite(5).active()); + + ignite(4).active(true); + + for (int i = 0; i < 4; i++) + startGrid(i); + + checkCaches1(6); + } + /** * @param exp If {@code true} there should be recorded messages. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java index 566860de75e01..2f9bd533ae916 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java @@ -79,7 +79,7 @@ public void testMarshalOnDaemonNode2() throws Exception { * @param startFirst If {@code true} daemon node is started first. * @throws Exception If failed. */ - public void marshalOnDaemonNode(boolean startFirst) throws Exception { + private void marshalOnDaemonNode(boolean startFirst) throws Exception { int nodeIdx = 0; if (!startFirst) { @@ -92,6 +92,7 @@ public void marshalOnDaemonNode(boolean startFirst) throws Exception { Ignite daemonNode = startGrid(nodeIdx++); + assertTrue(daemonNode.cluster().localNode().isDaemon()); assertEquals("true", daemonNode.cluster().localNode().attribute(ATTR_DAEMON)); daemon = false; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataUpdatesFlowTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataUpdatesFlowTest.java index 3ee51c8e8ddfb..7e8c086c9c2b1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataUpdatesFlowTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataUpdatesFlowTest.java @@ -184,6 +184,7 @@ public class BinaryMetadataUpdatesFlowTest extends GridCommonAbstractTest { * Starts new ignite node and submits computation job to it. * @param idx Index. * @param stopFlag Stop flag. + * @throws Exception If failed. */ private void startComputation(int idx, AtomicBoolean stopFlag) throws Exception { clientMode = false; @@ -199,6 +200,7 @@ private void startComputation(int idx, AtomicBoolean stopFlag) throws Exception * @param idx Index. * @param deafClient Deaf client. * @param observedIds Observed ids. + * @throws Exception If failed. */ private void startListening(int idx, boolean deafClient, Set observedIds) throws Exception { clientMode = true; @@ -269,7 +271,7 @@ private static class CQListener implements CacheEntryUpdatedListener { } /** - * + * @throws Exception If failed. */ public void testFlowNoConflicts() throws Exception { startComputation(0, stopFlag0); @@ -311,11 +313,14 @@ public void testFlowNoConflicts() throws Exception { } /** - * + * @throws Exception If failed. */ public void testFlowNoConflictsWithClients() throws Exception { startComputation(0, stopFlag0); + if (!tcpDiscovery()) + return; + startComputation(1, stopFlag1); startComputation(2, stopFlag2); @@ -617,6 +622,9 @@ private static final class BinaryObjectAdder implements IgniteCallable { while (!updatesQueue.isEmpty()) { BinaryUpdateDescription desc = updatesQueue.poll(); + if (desc == null) + break; + BinaryObjectBuilder builder = ignite.binary().builder(BINARY_TYPE_NAME); BinaryObject bo = newBinaryObject(builder, desc); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java index 313aaf910308d..81614cb9ff7a5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheClientNodeBinaryObjectMetadataMultinodeTest.java @@ -242,7 +242,7 @@ public void testFailoverOnStart() throws Exception { @Override public boolean apply() { Collection metaCol = p0.types(); - return metaCol.size() == 1000; + return metaCol.size() >= 1000; } }, getTestTimeout()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueClientDisconnectTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueClientDisconnectTest.java index ed54377327e2e..dac3ff6730e41 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueClientDisconnectTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueClientDisconnectTest.java @@ -32,6 +32,9 @@ import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +/** + * + */ public class GridCacheQueueClientDisconnectTest extends GridCommonAbstractTest { /** */ private static final String IGNITE_QUEUE_NAME = "ignite-queue-client-reconnect-test"; @@ -66,6 +69,10 @@ public class GridCacheQueueClientDisconnectTest extends GridCommonAbstractTest { return cfg; } + /** + * @param cacheAtomicityMode Atomicity mode. + * @return Configuration. + */ private static CollectionConfiguration collectionConfiguration(CacheAtomicityMode cacheAtomicityMode) { CollectionConfiguration colCfg = new CollectionConfiguration(); @@ -74,6 +81,9 @@ private static CollectionConfiguration collectionConfiguration(CacheAtomicityMod return colCfg; } + /** + * @throws Exception If failed. + */ public void testClientDisconnect() throws Exception { try { Ignite server = startGrid(0); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java index 51764b5be8fe1..d85201a4b2a8b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java @@ -494,7 +494,8 @@ private Ignite clientIgnite() { assertTrue(ignite.configuration().isClientMode()); - assertEquals(clientDiscovery(), ignite.configuration().getDiscoverySpi().isClientMode()); + if (tcpDiscovery()) + assertEquals(clientDiscovery(), ignite.configuration().getDiscoverySpi().isClientMode()); return ignite; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java index 0704dbdfe845d..e4560478daae2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java @@ -52,6 +52,7 @@ import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.events.DiscoveryEvent; +import org.apache.ignite.internal.DiscoverySpiTestListener; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.cluster.NodeOrderComparator; import org.apache.ignite.internal.cluster.NodeOrderLegacyComparator; @@ -60,7 +61,7 @@ import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.internal.TestRecordingCommunicationSpi; import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException; -import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.affinity.GridAffinityFunctionContextImpl; import org.apache.ignite.internal.processors.cache.CacheAffinityChangeMessage; @@ -88,7 +89,6 @@ import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.services.Service; import org.apache.ignite.services.ServiceContext; -import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; @@ -158,7 +158,7 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest { cfg.setCommunicationSpi(commSpi); - TestTcpDiscoverySpi discoSpi = new TestTcpDiscoverySpi(); + TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); discoSpi.setForceServerMode(forceSrvMode); discoSpi.setIpFinder(ipFinder); @@ -674,9 +674,11 @@ public void testNodeLeaveExchangeWaitAffinityMessage() throws Exception { checkAffinity(4, topVer(4, 0), true); - TestTcpDiscoverySpi discoSpi = (TestTcpDiscoverySpi)ignite0.configuration().getDiscoverySpi(); + DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener(); - discoSpi.blockCustomEvent(); + ((IgniteDiscoverySpi)ignite0.configuration().getDiscoverySpi()).setInternalListener(lsnr); + + lsnr.blockCustomEvent(CacheAffinityChangeMessage.class); stopGrid(1); @@ -687,7 +689,7 @@ public void testNodeLeaveExchangeWaitAffinityMessage() throws Exception { for (IgniteInternalFuture fut : futs) assertFalse(fut.isDone()); - discoSpi.stopBlock(); + lsnr.stopBlockCustomEvents(); checkAffinity(3, topVer(5, 0), false); @@ -1409,8 +1411,10 @@ public void testJoinExchangeBecomeCoordinator() throws Exception { public void testDelayAssignmentAffinityChanged() throws Exception { Ignite ignite0 = startServer(0, 1); - TestTcpDiscoverySpi discoSpi0 = - (TestTcpDiscoverySpi)ignite0.configuration().getDiscoverySpi(); + DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener(); + + ((IgniteDiscoverySpi)ignite0.configuration().getDiscoverySpi()).setInternalListener(lsnr); + TestRecordingCommunicationSpi commSpi0 = (TestRecordingCommunicationSpi)ignite0.configuration().getCommunicationSpi(); @@ -1418,19 +1422,19 @@ public void testDelayAssignmentAffinityChanged() throws Exception { checkAffinity(2, topVer(2, 0), true); - discoSpi0.blockCustomEvent(); + lsnr.blockCustomEvent(CacheAffinityChangeMessage.class); startServer(2, 3); checkAffinity(3, topVer(3, 0), false); - discoSpi0.waitCustomEvent(); + lsnr.waitCustomEvent(); blockSupplySend(commSpi0, CACHE_NAME1); startServer(3, 4); - discoSpi0.stopBlock(); + lsnr.stopBlockCustomEvents(); checkAffinity(4, topVer(4, 0), false); @@ -1452,8 +1456,10 @@ public void testDelayAssignmentAffinityChanged2() throws Exception { try { Ignite ignite0 = startServer(0, 1); - TestTcpDiscoverySpi discoSpi0 = - (TestTcpDiscoverySpi)ignite0.configuration().getDiscoverySpi(); + DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener(); + + ((IgniteDiscoverySpi)ignite0.configuration().getDiscoverySpi()).setInternalListener(lsnr); + TestRecordingCommunicationSpi commSpi0 = (TestRecordingCommunicationSpi)ignite0.configuration().getCommunicationSpi(); @@ -1465,11 +1471,11 @@ public void testDelayAssignmentAffinityChanged2() throws Exception { checkAffinity(3, topVer(3, 1), false); - discoSpi0.blockCustomEvent(); + lsnr.blockCustomEvent(CacheAffinityChangeMessage.class); stopNode(2, 4); - discoSpi0.waitCustomEvent(); + lsnr.waitCustomEvent(); blockSupplySend(commSpi0, CACHE_NAME1); @@ -1483,7 +1489,7 @@ public void testDelayAssignmentAffinityChanged2() throws Exception { Thread.sleep(2_000); - discoSpi0.stopBlock(); + lsnr.stopBlockCustomEvents(); boolean started = GridTestUtils.waitForCondition(new GridAbsPredicate() { @Override public boolean apply() { @@ -1534,14 +1540,16 @@ public void testDelayAssignmentCacheDestroyCreate() throws Exception { ignite0.createCache(ccfg); - TestTcpDiscoverySpi discoSpi0 = - (TestTcpDiscoverySpi)ignite0.configuration().getDiscoverySpi(); + DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener(); + + ((IgniteDiscoverySpi)ignite0.configuration().getDiscoverySpi()).setInternalListener(lsnr); + TestRecordingCommunicationSpi spi = (TestRecordingCommunicationSpi)ignite0.configuration().getCommunicationSpi(); blockSupplySend(spi, CACHE_NAME2); - discoSpi0.blockCustomEvent(); + lsnr.blockCustomEvent(CacheAffinityChangeMessage.class); startServer(1, 2); @@ -1551,7 +1559,7 @@ public void testDelayAssignmentCacheDestroyCreate() throws Exception { spi.stopBlock(); - discoSpi0.waitCustomEvent(); + lsnr.waitCustomEvent(); ignite0.destroyCache(CACHE_NAME2); @@ -1561,7 +1569,7 @@ public void testDelayAssignmentCacheDestroyCreate() throws Exception { ignite0.createCache(ccfg); - discoSpi0.stopBlock(); + lsnr.stopBlockCustomEvents(); checkAffinity(3, topVer(3, 1), false); checkAffinity(3, topVer(3, 2), false); @@ -2964,83 +2972,6 @@ public TestServiceImpl(int key) { } } - /** - * - */ - static class TestTcpDiscoverySpi extends TcpDiscoverySpi { - /** */ - private boolean blockCustomEvt; - - /** */ - private final Object mux = new Object(); - - /** */ - private List blockedMsgs = new ArrayList<>(); - - /** {@inheritDoc} */ - @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException { - synchronized (mux) { - if (blockCustomEvt) { - DiscoveryCustomMessage msg0 = GridTestUtils.getFieldValue(msg, "delegate"); - - if (msg0 instanceof CacheAffinityChangeMessage) { - log.info("Block custom message: " + msg0); - - blockedMsgs.add(msg); - - mux.notifyAll(); - - return; - } - } - } - - super.sendCustomEvent(msg); - } - - /** - * - */ - public void blockCustomEvent() { - synchronized (mux) { - assert blockedMsgs.isEmpty() : blockedMsgs; - - blockCustomEvt = true; - } - } - - /** - * @throws InterruptedException If interrupted. - */ - public void waitCustomEvent() throws InterruptedException { - synchronized (mux) { - while (blockedMsgs.isEmpty()) - mux.wait(); - } - } - - /** - * - */ - public void stopBlock() { - List msgs; - - synchronized (this) { - msgs = new ArrayList<>(blockedMsgs); - - blockCustomEvt = false; - - blockedMsgs.clear(); - } - - for (DiscoverySpiCustomMessage msg : msgs) { - log.info("Resend blocked message: " + msg); - - super.sendCustomEvent(msg); - } - } - } - /** * */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java index 3834df9b5dcc7..5dea5d9238e2d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java @@ -45,6 +45,7 @@ import static org.apache.ignite.IgniteState.STOPPED; import static org.apache.ignite.IgniteSystemProperties.IGNITE_TX_SALVAGE_TIMEOUT; import static org.apache.ignite.IgniteSystemProperties.getInteger; +import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; @@ -188,7 +189,7 @@ private void checkTransaction(TransactionConcurrency concurrency, TransactionIso return true; } - }, EVT_NODE_LEFT); + }, EVT_NODE_LEFT, EVT_NODE_FAILED); stopGrid(idx); @@ -268,7 +269,7 @@ public void testLock() throws Exception { return true; } - }, EVT_NODE_LEFT); + }, EVT_NODE_LEFT, EVT_NODE_FAILED); stopGrid(idx); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java index e71d3ee3ca290..b7ae84400c2fe 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java @@ -168,6 +168,8 @@ public void test150Clients() throws Exception { log.info("Started all clients."); + waitForTopology(CLIENTS + 1); + checkNodes(CLIENTS + 1); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java index a0be40ec79237..7785a3ce13941 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java @@ -178,7 +178,7 @@ private void manyClientsSequentially() throws Exception { log.info("All clients started."); try { - checkNodes(SRVS + CLIENTS); + checkNodes0(SRVS + CLIENTS); } finally { for (Ignite client : clients) @@ -186,6 +186,30 @@ private void manyClientsSequentially() throws Exception { } } + /** + * @param expCnt Expected number of nodes. + * @throws Exception If failed. + */ + private void checkNodes0(final int expCnt) throws Exception { + boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + try { + checkNodes(expCnt); + + return true; + } + catch (AssertionFailedError e) { + log.info("Check failed, will retry: " + e); + } + + return false; + } + }, 10_000); + + if (!wait) + checkNodes(expCnt); + } + /** * @param expCnt Expected number of nodes. */ @@ -297,23 +321,7 @@ private void manyClientsPutGet() throws Throwable { if (err0 != null) throw err0; - boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() { - @Override public boolean apply() { - try { - checkNodes(SRVS + THREADS); - - return true; - } - catch (AssertionFailedError e) { - log.info("Check failed, will retry: " + e); - } - - return false; - } - }, 10_000); - - if (!wait) - checkNodes(SRVS + THREADS); + checkNodes0(SRVS + THREADS); log.info("Stop clients."); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteOptimisticTxSuspendResumeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteOptimisticTxSuspendResumeTest.java index a55f21de6f68b..486fd6075d2ca 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteOptimisticTxSuspendResumeTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteOptimisticTxSuspendResumeTest.java @@ -713,6 +713,8 @@ private void executeTestForAllCaches(CI2> ", backups=" + ccfg.getBackups() + ", near=" + (ccfg.getNearConfiguration() != null) + "]"); + awaitPartitionMapExchange(); + int srvNum = serversNumber(); if (serversNumber() > 1) { ignite(serversNumber() + 1).createNearCache(ccfg.getName(), new NearCacheConfiguration<>()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java index e3fa1165cffbd..60f2f0a80bd3c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadMultiThreadedSelfTest.java @@ -116,6 +116,8 @@ public void testConcurrentNodesStart() throws Exception { @Nullable @Override public Object call() throws Exception { IgniteConfiguration cfg = loadConfiguration("modules/core/src/test/config/spring-multicache.xml"); + cfg.setGridLogger(getTestResources().getLogger()); + startGrid(Thread.currentThread().getName(), cfg); return null; @@ -161,6 +163,8 @@ public void testConcurrentNodesStartStop() throws Exception { @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { IgniteConfiguration cfg = loadConfiguration("modules/core/src/test/config/spring-multicache.xml"); + cfg.setGridLogger(getTestResources().getLogger()); + cfg.setIgniteInstanceName(igniteInstanceName); for (CacheConfiguration cCfg : cfg.getCacheConfiguration()) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java index 05a9759cd559c..83eff893b8894 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java @@ -308,6 +308,8 @@ private void checkActivePartitionTransfer(int keyCnt, int nodeCnt, boolean sameC // Check all left nodes. checkActiveState(ignites); + + awaitPartitionMapExchange(); // Need wait, otherwise test logic is broken if EVT_NODE_FAILED exchanges are merged. } info("Waiting for preload futures: " + F.view(futs, new IgnitePredicate>() { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java index 7b350c8a41bb8..060af21f8d363 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java @@ -34,11 +34,11 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; -import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.ignite.transactions.Transaction; import org.apache.ignite.transactions.TransactionConcurrency; @@ -65,7 +65,6 @@ public class TxRecoveryStoreEnabledTest extends GridCommonAbstractTest { IgniteConfiguration cfg = super.getConfiguration(gridName); cfg.setCommunicationSpi(new TestCommunicationSpi()); - cfg.setDiscoverySpi(new TestDiscoverySpi()); CacheConfiguration ccfg = defaultCacheConfiguration(); @@ -126,7 +125,7 @@ private void checkTxRecovery(TransactionConcurrency concurrency) throws Exceptio IgniteConfiguration cfg = node0.configuration(); ((TestCommunicationSpi)cfg.getCommunicationSpi()).block(); - ((TestDiscoverySpi)cfg.getDiscoverySpi()).simulateNodeFailure(); + ((IgniteDiscoverySpi)cfg.getDiscoverySpi()).simulateNodeFailure(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -198,16 +197,6 @@ private static class TestCacheStore extends CacheStoreAdapter } } - /** - * - */ - private static class TestDiscoverySpi extends TcpDiscoverySpi { - /** {@inheritDoc} */ - @Override protected void simulateNodeFailure() { - super.simulateNodeFailure(); - } - } - /** * */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java index 3c57957179e43..96fb8f65bb56c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java @@ -37,6 +37,7 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; /** @@ -125,7 +126,7 @@ public void testLockFromNearOrBackup() throws Exception { return true; } - }, EVT_NODE_LEFT)); + }, EVT_NODE_LEFT, EVT_NODE_FAILED)); } stopGrid(idx); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ClientReconnectContinuousQueryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ClientReconnectContinuousQueryTest.java index c8b3bb6afc95f..9b531c68277a4 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ClientReconnectContinuousQueryTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ClientReconnectContinuousQueryTest.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache.query.continuous; +import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import javax.cache.event.CacheEntryListenerException; @@ -28,6 +29,7 @@ import org.apache.ignite.events.Event; import org.apache.ignite.events.EventType; import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.managers.communication.GridIoManager; import org.apache.ignite.internal.util.nio.GridNioServer; import org.apache.ignite.internal.util.typedef.internal.U; @@ -90,7 +92,7 @@ public void testClientReconnect() throws Exception { try { startGrids(2); - IgniteEx client = grid(CLIENT_IDX); + final IgniteEx client = grid(CLIENT_IDX); client.events().localListen(new DisconnectListener(), EventType.EVT_CLIENT_NODE_DISCONNECTED); @@ -112,11 +114,19 @@ public void testClientReconnect() throws Exception { skipRead(client, true); - putSomeKeys(1_000); + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + assertTrue(disconLatch.await(10_000, TimeUnit.MILLISECONDS)); + + skipRead(client, false); - assertTrue(disconLatch.await(10_000, TimeUnit.MILLISECONDS)); + return null; + } + }); - skipRead(client, false); + putSomeKeys(1_000); + + fut.get(); assertTrue(reconLatch.await(10_000, TimeUnit.MILLISECONDS)); @@ -129,7 +139,6 @@ public void testClientReconnect() throws Exception { finally { stopAllGrids(); } - } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryRemoteFilterMissingInClassPathSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryRemoteFilterMissingInClassPathSelfTest.java index 92c1760832274..226302ff9d616 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryRemoteFilterMissingInClassPathSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryRemoteFilterMissingInClassPathSelfTest.java @@ -104,7 +104,7 @@ public void testWarningMessageOnClientNode() throws Exception { setExternalLoader = true; final Ignite ignite0 = startGrid(1); - executeContiniouseQuery(ignite0.cache("simple")); + executeContinuousQuery(ignite0.cache("simple")); log = new GridStringLogger(); clientMode = true; @@ -112,8 +112,10 @@ public void testWarningMessageOnClientNode() throws Exception { startGrid(2); - assertTrue(log.toString().contains("Failed to unmarshal continuous query remote filter on client node. " + - "Can be ignored.")); + String logStr = log.toString(); + + assertTrue(logStr.contains("Failed to unmarshal continuous query remote filter on client node. " + + "Can be ignored.") || logStr.contains("Failed to unmarshal continuous routine handler")); } /** @@ -127,7 +129,7 @@ public void testNoWarningMessageOnClientNode() throws Exception { clientMode = false; final Ignite ignite0 = startGrid(1); - executeContiniouseQuery(ignite0.cache("simple")); + executeContinuousQuery(ignite0.cache("simple")); log = new GridStringLogger(); clientMode = true; @@ -149,15 +151,18 @@ public void testExceptionOnServerNode() throws Exception { setExternalLoader = true; final Ignite ignite0 = startGrid(1); - executeContiniouseQuery(ignite0.cache("simple")); + executeContinuousQuery(ignite0.cache("simple")); log = new GridStringLogger(); setExternalLoader = false; startGrid(2); - assertTrue(log.toString().contains("class org.apache.ignite.IgniteCheckedException: " + - "Failed to find class with given class loader for unmarshalling")); + String logStr = log.toString(); + + assertTrue(logStr.contains("class org.apache.ignite.IgniteCheckedException: " + + "Failed to find class with given class loader for unmarshalling") + || logStr.contains("Failed to unmarshal continuous routine handler")); } /** @@ -171,7 +176,7 @@ public void testNoExceptionOnServerNode() throws Exception { setExternalLoader = true; final Ignite ignite0 = startGrid(1); - executeContiniouseQuery(ignite0.cache("simple")); + executeContinuousQuery(ignite0.cache("simple")); log = new GridStringLogger(); @@ -185,7 +190,7 @@ public void testNoExceptionOnServerNode() throws Exception { * @param cache Ignite cache. * @throws Exception If fail. */ - private void executeContiniouseQuery(IgniteCache cache) throws Exception { + private void executeContinuousQuery(IgniteCache cache) throws Exception { ContinuousQuery qry = new ContinuousQuery<>(); qry.setLocalListener( diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryClientReconnectTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryClientReconnectTest.java index 9ad6d4ec6c0a4..906cc7d8d190c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryClientReconnectTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryClientReconnectTest.java @@ -119,6 +119,9 @@ public void testReconnectClient() throws Exception { * @throws Exception If failed. */ public void testReconnectClientAndLeftRouter() throws Exception { + if (!tcpDiscovery()) + return; + Ignite client = grid(serverCount()); final Ignite srv = clientRouter(client); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java index 61ceef7128dbb..16ea84857840a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/version/CacheVersionedEntryAbstractTest.java @@ -18,11 +18,12 @@ package org.apache.ignite.internal.processors.cache.version; import java.util.HashSet; +import java.util.Map; import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; import javax.cache.Cache; import javax.cache.processor.EntryProcessor; import javax.cache.processor.EntryProcessorException; +import javax.cache.processor.EntryProcessorResult; import javax.cache.processor.MutableEntry; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheEntry; @@ -56,23 +57,15 @@ public abstract class CacheVersionedEntryAbstractTest extends GridCacheAbstractS public void testInvoke() throws Exception { Cache cache = grid(0).cache(DEFAULT_CACHE_NAME); - final AtomicInteger invoked = new AtomicInteger(); - - cache.invoke(100, new EntryProcessor() { - @Override public Object process(MutableEntry entry, Object... arguments) - throws EntryProcessorException { - - invoked.incrementAndGet(); - + assertNotNull(cache.invoke(100, new EntryProcessor() { + @Override public Object process(MutableEntry entry, Object... args) { CacheEntry verEntry = entry.unwrap(CacheEntry.class); checkVersionedEntry(verEntry); - return entry; + return verEntry.version(); } - }); - - assert invoked.get() > 0; + })); } /** @@ -86,23 +79,17 @@ public void testInvokeAll() throws Exception { for (int i = 0; i < ENTRIES_NUM; i++) keys.add(i); - final AtomicInteger invoked = new AtomicInteger(); - - cache.invokeAll(keys, new EntryProcessor() { - @Override public Object process(MutableEntry entry, Object... arguments) - throws EntryProcessorException { - - invoked.incrementAndGet(); - + Map> res = cache.invokeAll(keys, new EntryProcessor() { + @Override public Object process(MutableEntry entry, Object... args) { CacheEntry verEntry = entry.unwrap(CacheEntry.class); checkVersionedEntry(verEntry); - return null; + return verEntry.version(); } }); - assert invoked.get() > 0; + assertEquals(ENTRIES_NUM, res.size()); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java index f07b1a3c02d70..1a7abd45fa921 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java @@ -447,7 +447,7 @@ public void testAllEvents() throws Exception { grid(0).compute().broadcast(F.noop()); - assert latch.await(2, SECONDS); + assert latch.await(10, SECONDS) : latch; assertEquals(GRID_CNT, nodeIds.size()); assertEquals(GRID_CNT, cnt.get()); @@ -488,7 +488,7 @@ public void testEventsByType() throws Exception { grid(0).compute().broadcast(F.noop()); - assert latch.await(2, SECONDS); + assert latch.await(10, SECONDS) : latch; assertEquals(GRID_CNT, nodeIds.size()); assertEquals(GRID_CNT, cnt.get()); @@ -532,7 +532,7 @@ public void testEventsByFilter() throws Exception { grid(0).compute().broadcast(F.noop()); - assert latch.await(2, SECONDS); + assert latch.await(10, SECONDS) : latch; assertEquals(GRID_CNT, nodeIds.size()); assertEquals(GRID_CNT, cnt.get()); @@ -578,7 +578,7 @@ public void testEventsByTypeAndFilter() throws Exception { grid(0).compute().broadcast(F.noop()); grid(0).compute().withName("exclude").run(F.noop()); - assert latch.await(2, SECONDS); + assert latch.await(10, SECONDS) : latch; assertEquals(GRID_CNT, nodeIds.size()); assertEquals(GRID_CNT, cnt.get()); @@ -619,7 +619,7 @@ public void testRemoteProjection() throws Exception { grid(0).compute().broadcast(F.noop()); - assert latch.await(2, SECONDS); + assert latch.await(10, SECONDS) : latch; assertEquals(GRID_CNT - 1, nodeIds.size()); assertEquals(GRID_CNT - 1, cnt.get()); @@ -660,7 +660,7 @@ public void testProjectionWithLocalNode() throws Exception { grid(0).compute().broadcast(F.noop()); - assert latch.await(2, SECONDS); + assert latch.await(10, SECONDS) : latch; assertEquals(GRID_CNT - 1, nodeIds.size()); assertEquals(GRID_CNT - 1, cnt.get()); @@ -701,7 +701,7 @@ public void testLocalNodeOnly() throws Exception { grid(0).compute().broadcast(F.noop()); - assert latch.await(2, SECONDS); + assert latch.await(10, SECONDS) : latch; assertEquals(1, nodeIds.size()); assertEquals(1, cnt.get()); @@ -744,7 +744,7 @@ public void testStopByCallback() throws Exception { grid(0).compute().broadcast(F.noop()); - assert latch.await(2, SECONDS); + assert latch.await(10, SECONDS) : latch; assertEquals(1, nodeIds.size()); assertEquals(1, cnt.get()); @@ -785,7 +785,7 @@ public void testStopRemoteListen() throws Exception { grid(0).compute().run(F.noop()); - assert latch.await(2, SECONDS); + assert latch.await(10, SECONDS) : latch; assertEquals(1, nodeIds.size()); assertEquals(1, cnt.get()); @@ -828,7 +828,7 @@ public void testStopLocalListenByCallback() throws Exception { compute(grid(0).cluster().forLocal()).run(F.noop()); - assert latch.await(2, SECONDS); + assert latch.await(10, SECONDS) : latch; assertEquals(1, cnt.get()); @@ -878,7 +878,7 @@ public void testNodeJoin() throws Exception { grid(0).compute().broadcast(F.noop()); - assert latch.await(2, SECONDS); + assert latch.await(10, SECONDS) : latch; assertEquals(GRID_CNT + 1, nodeIds.size()); assertEquals(GRID_CNT + 1, cnt.get()); @@ -929,7 +929,7 @@ public void testNodeJoinWithProjection() throws Exception { grid(0).compute().broadcast(F.noop()); - assert latch.await(2, SECONDS); + assert latch.await(10, SECONDS) : latch; assertEquals(GRID_CNT, nodeIds.size()); assertEquals(GRID_CNT, cnt.get()); @@ -980,7 +980,7 @@ public void testNodeJoinWithP2P() throws Exception { grid(0).compute().broadcast(F.noop()); - assert latch.await(2, SECONDS); + assert latch.await(10, SECONDS) : latch; assertEquals(GRID_CNT + 1, nodeIds.size()); assertEquals(GRID_CNT + 1, cnt.get()); @@ -1036,7 +1036,7 @@ public void testResources() throws Exception { grid(0).compute().broadcast(F.noop()); - assert latch.await(2, SECONDS); + assert latch.await(10, SECONDS) : latch; assertEquals(GRID_CNT, nodeIds.size()); assertEquals(GRID_CNT, cnt.get()); @@ -1145,7 +1145,7 @@ public void testMasterNodeLeaveNoAutoUnsubscribe() throws Exception { /** * @throws Exception If failed. */ - public void testMultithreadedWithNodeRestart() throws Exception { + public void _testMultithreadedWithNodeRestart() throws Exception { final AtomicBoolean stop = new AtomicBoolean(); final BlockingQueue> queue = new LinkedBlockingQueue<>(); final Collection started = new GridConcurrentHashSet<>(); @@ -1153,9 +1153,11 @@ public void testMultithreadedWithNodeRestart() throws Exception { final Random rnd = new Random(); + final int consumeCnt = tcpDiscovery() ? CONSUME_CNT : CONSUME_CNT / 2; + IgniteInternalFuture starterFut = multithreadedAsync(new Callable() { @Override public Object call() throws Exception { - for (int i = 0; i < CONSUME_CNT; i++) { + for (int i = 0; i < consumeCnt; i++) { int idx = rnd.nextInt(GRID_CNT); try { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java index c1af32382a4cd..8f03c4cefaf0a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java @@ -49,6 +49,9 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest { /** Number of grids started for tests. */ private static final int NODES_CNT = 4; + /** */ + private static final int CLIENT_IDX = 1; + /** Test singleton service name. */ private static final String SINGLETON_NAME = "testSingleton"; @@ -61,11 +64,11 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest { cfg.setMarshaller(new BinaryMarshaller()); - cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(ipFinder).setForceServerMode(true)); + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder); cfg.setCacheConfiguration(); - if (igniteInstanceName.equals(getTestIgniteInstanceName(0))) + if (igniteInstanceName.equals(getTestIgniteInstanceName(CLIENT_IDX))) cfg.setClientMode(true); return cfg; @@ -88,8 +91,10 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest { public void testDefaultClosure() throws Exception { Set srvNames = new HashSet<>(NODES_CNT - 1); - for (int i = 1; i < NODES_CNT; ++i) - srvNames.add(getTestIgniteInstanceName(i)); + for (int i = 0; i < NODES_CNT; ++i) { + if (i != CLIENT_IDX) + srvNames.add(getTestIgniteInstanceName(i)); + } for (int i = 0 ; i < NODES_CNT; i++) { log.info("Iteration: " + i); @@ -137,7 +142,7 @@ public void testClientClosure() throws Exception { assertEquals(1, res.size()); - assertEquals(getTestIgniteInstanceName(0), F.first(res)); + assertEquals(getTestIgniteInstanceName(CLIENT_IDX), F.first(res)); } } @@ -168,7 +173,7 @@ public void testCustomClosure() throws Exception { * @throws Exception If failed. */ public void testDefaultService() throws Exception { - UUID clientNodeId = grid(0).cluster().localNode().id(); + UUID clientNodeId = grid(CLIENT_IDX).cluster().localNode().id(); for (int i = 0 ; i < NODES_CNT; i++) { log.info("Iteration: " + i); @@ -209,7 +214,7 @@ public void testDefaultService() throws Exception { * @throws Exception If failed. */ public void testClientService() throws Exception { - UUID clientNodeId = grid(0).cluster().localNode().id(); + UUID clientNodeId = grid(CLIENT_IDX).cluster().localNode().id(); for (int i = 0 ; i < NODES_CNT; i++) { log.info("Iteration: " + i); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/GridTestClockTimer.java b/modules/core/src/test/java/org/apache/ignite/internal/util/GridTestClockTimer.java index 5da9042705d3a..2d26b72bf9063 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/GridTestClockTimer.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/GridTestClockTimer.java @@ -32,6 +32,15 @@ public GridTestClockTimer() { } } + /** + * @return {@code True} if need start test time. + */ + public static boolean startTestTimer() { + synchronized (IgniteUtils.mux) { + return IgniteUtils.gridCnt == 0; + } + } + /** {@inheritDoc} */ @Override public void run() { while (true) { diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerMappingConsistencyTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerMappingConsistencyTest.java index cd8e757f945c7..78f3c03eda818 100644 --- a/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerMappingConsistencyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerMappingConsistencyTest.java @@ -30,6 +30,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; /** @@ -157,6 +158,9 @@ public void testPersistedMappingsSharedOnJoin() throws Exception { Ignite g2 = startGrid(2); startGrid(1); + assertTrue("Failed to wait for automatic grid activation", + GridTestUtils.waitForCondition(() -> g2.cluster().active(), getTestTimeout())); + IgniteCache c2 = g2.cache(CACHE_NAME); assertEquals(k, c2.get(k).val); diff --git a/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java index 7541cec455c63..a7c452122d566 100644 --- a/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java @@ -24,7 +24,6 @@ import java.io.Serializable; import java.net.URL; import java.net.URLClassLoader; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -37,22 +36,20 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteMessaging; import org.apache.ignite.cluster.ClusterGroup; import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; +import org.apache.ignite.internal.DiscoverySpiTestListener; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; import org.apache.ignite.internal.processors.continuous.StartRoutineDiscoveryMessage; +import org.apache.ignite.internal.processors.continuous.StartRoutineDiscoveryMessageV2; import org.apache.ignite.internal.processors.continuous.StopRoutineDiscoveryMessage; -import org.apache.ignite.internal.processors.marshaller.MappingAcceptedMessage; -import org.apache.ignite.internal.processors.marshaller.MappingProposedMessage; import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.typedef.P2; import org.apache.ignite.internal.util.typedef.PA; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.resources.IgniteInstanceResource; -import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; @@ -206,11 +203,7 @@ public TestMessage() { @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); - TestTcpDiscoverySpi discoSpi = new TestTcpDiscoverySpi(); - - discoSpi.setIpFinder(ipFinder); - - cfg.setDiscoverySpi(discoSpi); + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder); return cfg; } @@ -1036,7 +1029,11 @@ public void testNullMessages() throws Exception { public void testAsyncOld() throws Exception { final AtomicInteger msgCnt = new AtomicInteger(); - TestTcpDiscoverySpi discoSpi = (TestTcpDiscoverySpi)ignite2.configuration().getDiscoverySpi(); + IgniteDiscoverySpi discoSpi = (IgniteDiscoverySpi)ignite2.configuration().getDiscoverySpi(); + + DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener(); + + discoSpi.setInternalListener(lsnr); assertFalse(ignite2.message().isAsync()); @@ -1054,7 +1051,7 @@ public void testAsyncOld() throws Exception { } }, IllegalStateException.class, null); - discoSpi.blockCustomEvent(); + lsnr.blockCustomEvent(StartRoutineDiscoveryMessage.class, StartRoutineDiscoveryMessageV2.class); final String topic = "topic"; @@ -1079,7 +1076,7 @@ public void testAsyncOld() throws Exception { Assert.assertFalse(starFut.isDone()); - discoSpi.stopBlock(); + lsnr.stopBlockCustomEvents(); GridTestUtils.assertThrows(log, new Callable() { @Override public Void call() throws Exception { @@ -1095,7 +1092,7 @@ public void testAsyncOld() throws Exception { Assert.assertTrue(starFut.isDone()); - discoSpi.blockCustomEvent(); + lsnr.blockCustomEvent(StopRoutineDiscoveryMessage.class); message(ignite1.cluster().forRemotes()).send(topic, "msg1"); @@ -1125,7 +1122,7 @@ public void testAsyncOld() throws Exception { Assert.assertFalse(stopFut.isDone()); - discoSpi.stopBlock(); + lsnr.stopBlockCustomEvents(); stopFut.get(); @@ -1144,9 +1141,13 @@ public void testAsyncOld() throws Exception { public void testAsync() throws Exception { final AtomicInteger msgCnt = new AtomicInteger(); - TestTcpDiscoverySpi discoSpi = (TestTcpDiscoverySpi)ignite2.configuration().getDiscoverySpi(); + IgniteDiscoverySpi discoSpi = (IgniteDiscoverySpi)ignite2.configuration().getDiscoverySpi(); - discoSpi.blockCustomEvent(); + DiscoverySpiTestListener lsnr = new DiscoverySpiTestListener(); + + discoSpi.setInternalListener(lsnr); + + lsnr.blockCustomEvent(StartRoutineDiscoveryMessage.class, StartRoutineDiscoveryMessageV2.class); final String topic = "topic"; @@ -1167,7 +1168,7 @@ public void testAsync() throws Exception { Assert.assertFalse(starFut.isDone()); - discoSpi.stopBlock(); + lsnr.stopBlockCustomEvents(); UUID id = starFut.get(); @@ -1175,7 +1176,7 @@ public void testAsync() throws Exception { Assert.assertTrue(starFut.isDone()); - discoSpi.blockCustomEvent(); + lsnr.blockCustomEvent(StopRoutineDiscoveryMessage.class); message(ignite1.cluster().forRemotes()).send(topic, "msg1"); @@ -1195,7 +1196,7 @@ public void testAsync() throws Exception { Assert.assertFalse(stopFut.isDone()); - discoSpi.stopBlock(); + lsnr.stopBlockCustomEvents(); stopFut.get(); @@ -1208,89 +1209,6 @@ public void testAsync() throws Exception { assertEquals(1, msgCnt.get()); } - /** - * - */ - static class TestTcpDiscoverySpi extends TcpDiscoverySpi { - /** */ - private boolean blockCustomEvt; - - /** */ - private final Object mux = new Object(); - - /** */ - private List blockedMsgs = new ArrayList<>(); - - /** {@inheritDoc} */ - @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException { - synchronized (mux) { - if (blockCustomEvt) { - DiscoveryCustomMessage msg0 = GridTestUtils.getFieldValue(msg, "delegate"); - - if (msg0 instanceof MappingProposedMessage || msg0 instanceof MappingAcceptedMessage){ - super.sendCustomEvent(msg); - - return; - } - - if (msg0 instanceof StopRoutineDiscoveryMessage || msg0 instanceof StartRoutineDiscoveryMessage) { - log.info("Block custom message: " + msg0); - - blockedMsgs.add(msg); - - mux.notifyAll(); - - return; - } - } - } - - super.sendCustomEvent(msg); - } - - /** - * - */ - public void blockCustomEvent() { - synchronized (mux) { - assert blockedMsgs.isEmpty() : blockedMsgs; - - blockCustomEvt = true; - } - } - - /** - * @throws InterruptedException If interrupted. - */ - public void waitCustomEvent() throws InterruptedException { - synchronized (mux) { - while (blockedMsgs.isEmpty()) - mux.wait(); - } - } - - /** - * - */ - public void stopBlock() { - List msgs; - - synchronized (this) { - msgs = new ArrayList<>(blockedMsgs); - - blockCustomEvt = false; - - blockedMsgs.clear(); - } - - for (DiscoverySpiCustomMessage msg : msgs) { - log.info("Resend blocked message: " + msg); - - super.sendCustomEvent(msg); - } - } - } - /** * Tests that message listener registers only for one oldest node. * diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java index 54b3a785dc250..e89a4c828fb75 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java @@ -17,11 +17,21 @@ package org.apache.ignite.spi.communication.tcp; +import java.util.ArrayList; +import java.util.BitSet; +import java.util.List; +import java.util.Map; import java.util.UUID; +import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CyclicBarrier; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.util.nio.GridCommunicationClient; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiAdapter; import org.apache.ignite.spi.communication.CommunicationSpi; import org.apache.ignite.spi.communication.GridAbstractCommunicationSelfTest; @@ -85,6 +95,67 @@ protected GridTcpCommunicationSpiAbstractTest(boolean useShmem) { } } + /** + * + */ + public void testCheckConnection1() { + for (int i = 0; i < 100; i++) { + for (Map.Entry> entry : spis.entrySet()) { + TcpCommunicationSpi spi = (TcpCommunicationSpi)entry.getValue(); + + List checkNodes = new ArrayList<>(nodes); + + assert checkNodes.size() > 1; + + IgniteFuture fut = spi.checkConnection(checkNodes); + + BitSet res = fut.get(); + + for (int n = 0; n < checkNodes.size(); n++) + assertTrue(res.get(n)); + } + } + } + + /** + * @throws Exception If failed. + */ + public void testCheckConnection2() throws Exception { + final int THREADS = spis.size(); + + final CyclicBarrier b = new CyclicBarrier(THREADS); + + List futs = new ArrayList<>(); + + for (Map.Entry> entry : spis.entrySet()) { + final TcpCommunicationSpi spi = (TcpCommunicationSpi)entry.getValue(); + + futs.add(GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + List checkNodes = new ArrayList<>(nodes); + + assert checkNodes.size() > 1; + + b.await(); + + for (int i = 0; i < 100; i++) { + IgniteFuture fut = spi.checkConnection(checkNodes); + + BitSet res = fut.get(); + + for (int n = 0; n < checkNodes.size(); n++) + assertTrue(res.get(n)); + } + + return null; + } + })); + } + + for (IgniteInternalFuture f : futs) + f.get(); + } + /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { super.afterTest(); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/FilterDataForClientNodeDiscoveryTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/FilterDataForClientNodeDiscoveryTest.java index 54b48e5a9df61..9a45d2d68d5be 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/FilterDataForClientNodeDiscoveryTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/FilterDataForClientNodeDiscoveryTest.java @@ -205,6 +205,11 @@ private static class MessageForServer implements DiscoveryServerOnlyCustomMessag return false; } + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + /** {@inheritDoc} */ @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer, DiscoCache discoCache) { diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java index ca052883e35e2..51dcb236dfcf9 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java @@ -613,6 +613,16 @@ public void triggerEvent(Event evt) { return Collections.emptyMap(); } + /** {@inheritDoc} */ + @Override public boolean communicationFailureResolveSupported() { + return false; + } + + /** {@inheritDoc} */ + @Override public void resolveCommunicationFailure(ClusterNode node, Exception err) { + throw new UnsupportedOperationException(); + } + /** * @param cacheName Cache name. * @return Map representing cache. diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java b/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java index 4507572f92a7a..e2594ca04b29a 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/config/GridTestProperties.java @@ -83,6 +83,15 @@ public final class GridTestProperties { /** "True value" enables {@link BinaryBasicNameMapper} in {@link BinaryTypeConfiguration#getNameMapper()} */ public static final String BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER = "binary.marshaller.use.simple.name.mapper"; + /** + * Name of class which provides static method preprocessConfiguration(IgniteConfiguration cfg) to + * alter {@link org.apache.ignite.configuration.IgniteConfiguration} before node is started. + *

    + * Note: this pre-preprocessor is started only if test starts node using one of GridAbstractTest's startGrid + * method. + */ + public static final String IGNITE_CFG_PREPROCESSOR_CLS = "ignite.cfg.preprocessor.class"; + /** */ static { // Initialize IGNITE_HOME system property. 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 c3b262c3667da..f5784ebcdccd2 100755 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java @@ -46,6 +46,7 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteClientDisconnectedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteSystemProperties; @@ -84,6 +85,7 @@ import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteCallable; import org.apache.ignite.lang.IgniteClosure; +import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.logger.NullLogger; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.marshaller.MarshallerContextTestImpl; @@ -92,6 +94,7 @@ import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointSpi; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.discovery.DiscoverySpi; import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.TestTcpDiscoverySpi; @@ -124,6 +127,7 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; import static org.apache.ignite.internal.GridKernalState.DISCONNECTED; import static org.apache.ignite.testframework.config.GridTestProperties.BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER; +import static org.apache.ignite.testframework.config.GridTestProperties.IGNITE_CFG_PREPROCESSOR_CLS; /** * Common abstract test for Ignite tests. @@ -203,13 +207,15 @@ public abstract class GridAbstractTest extends TestCase { if (BINARY_MARSHALLER) GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName()); - Thread timer = new Thread(new GridTestClockTimer(), "ignite-clock-for-tests"); + if (GridTestClockTimer.startTestTimer()) { + Thread timer = new Thread(new GridTestClockTimer(), "ignite-clock-for-tests"); - timer.setDaemon(true); + timer.setDaemon(true); - timer.setPriority(10); + timer.setPriority(10); - timer.start(); + timer.start(); + } } /** */ @@ -838,6 +844,7 @@ protected Ignite startGrid(String igniteInstanceName) throws Exception { protected Ignite startGrid(String igniteInstanceName, GridSpringResourceContext ctx) throws Exception { return startGrid(igniteInstanceName, optimize(getConfiguration(igniteInstanceName)), ctx); } + /** * Starts new grid with given name. * @@ -852,12 +859,33 @@ protected Ignite startGrid(String igniteInstanceName, IgniteConfiguration cfg, G startingIgniteInstanceName.set(igniteInstanceName); try { + String cfgProcClsName = System.getProperty(IGNITE_CFG_PREPROCESSOR_CLS); + + if (cfgProcClsName != null) { + try { + Class cfgProc = Class.forName(cfgProcClsName); + + Method method = cfgProc.getMethod("preprocessConfiguration", IgniteConfiguration.class); + + if (!Modifier.isStatic(method.getModifiers())) + throw new Exception("Non-static pre-processor method in pre-processor class: " + cfgProcClsName); + + method.invoke(null, cfg); + } + catch (Exception e) { + log.error("Failed to pre-process IgniteConfiguration using pre-processor class: " + cfgProcClsName); + + throw new IgniteException(e); + } + } + Ignite node = IgnitionEx.start(cfg, ctx); IgniteConfiguration nodeCfg = node.configuration(); log.info("Node started with the following configuration [id=" + node.cluster().localNode().id() + ", marshaller=" + nodeCfg.getMarshaller() + + ", discovery=" + nodeCfg.getDiscoverySpi() + ", binaryCfg=" + nodeCfg.getBinaryConfiguration() + ", lateAff=" + nodeCfg.isLateAffinityAssignment() + "]"); @@ -967,6 +995,26 @@ protected Ignite startRemoteGrid(String igniteInstanceName, IgniteConfiguration if (cfg == null) cfg = optimize(getConfiguration(igniteInstanceName)); + if (locNode != null) { + DiscoverySpi discoverySpi = locNode.configuration().getDiscoverySpi(); + + if (discoverySpi != null && !(discoverySpi instanceof TcpDiscoverySpi)) { + try { + // Clone added to support ZookeeperDiscoverySpi. + Method m = discoverySpi.getClass().getDeclaredMethod("cloneSpiConfiguration"); + + m.setAccessible(true); + + cfg.setDiscoverySpi((DiscoverySpi) m.invoke(discoverySpi)); + + resetDiscovery = false; + } + catch (NoSuchMethodException e) { + // Ignore. + } + } + } + return new IgniteProcessProxy(cfg, log, locNode, resetDiscovery); } @@ -1075,7 +1123,9 @@ protected void stopAllGrids(boolean cancel) { for (Ignite g : srvs) stopGrid(g.name(), cancel, false); - assert G.allGrids().isEmpty(); + List nodes = G.allGrids(); + + assert nodes.isEmpty() : nodes; } finally { IgniteProcessProxy.killAll(); // In multi-JVM case. @@ -1176,6 +1226,14 @@ protected Ignite ignite(int idx) { return grid(idx); } + /** + * @param nodeIdx Node index. + * @return Node ID. + */ + protected final UUID nodeId(int nodeIdx) { + return ignite(nodeIdx).cluster().localNode().id(); + } + /** * Gets grid for given test. * @@ -1217,7 +1275,11 @@ protected final Ignite grid(ClusterNode node) { * @throws Exception If failed. */ protected Ignite startGrid(String igniteInstanceName, String springCfgPath) throws Exception { - return startGrid(igniteInstanceName, loadConfiguration(springCfgPath)); + IgniteConfiguration cfg = loadConfiguration(springCfgPath); + + cfg.setGridLogger(getTestResources().getLogger()); + + return startGrid(igniteInstanceName, cfg); } /** @@ -2142,6 +2204,50 @@ private void awaitTopologyChange() throws IgniteInterruptedCheckedException { } } } + /** + * @param expSize Expected nodes number. + * @throws Exception If failed. + */ + protected void waitForTopology(final int expSize) throws Exception { + assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + List nodes = G.allGrids(); + + if (nodes.size() != expSize) { + info("Wait all nodes [size=" + nodes.size() + ", exp=" + expSize + ']'); + + return false; + } + + for (Ignite node: nodes) { + try { + IgniteFuture reconnectFut = node.cluster().clientReconnectFuture(); + + if (reconnectFut != null && !reconnectFut.isDone()) { + info("Wait for size on node, reconnect is in progress [node=" + node.name() + ']'); + + return false; + } + + int sizeOnNode = node.cluster().nodes().size(); + + if (sizeOnNode != expSize) { + info("Wait for size on node [node=" + node.name() + ", size=" + sizeOnNode + ", exp=" + expSize + ']'); + + return false; + } + } + catch (IgniteClientDisconnectedException e) { + info("Wait for size on node, node disconnected [node=" + node.name() + ']'); + + return false; + } + } + + return true; + } + }, 30_000)); + } /** * @param millis Time to sleep. @@ -2171,6 +2277,17 @@ protected final int groupIdForCache(Ignite node, String cacheName) { return 0; } + /** + * @return {@code True} if nodes use {@link TcpDiscoverySpi}. + */ + protected static boolean tcpDiscovery() { + List nodes = G.allGrids(); + + assertFalse("There are no nodes", nodes.isEmpty()); + + return nodes.get(0).configuration().getDiscoverySpi() instanceof TcpDiscoverySpi; + } + /** * */ diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java index d7be576d77c1e..2b3a19cdba956 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java @@ -162,6 +162,8 @@ private static IgniteConfiguration readCfgFromFileAndDeleteFile(String fileName) cfg.setDiscoverySpi(disco); } + X.println("Configured discovery: " + cfg.getDiscoverySpi().getClass().getName()); + return cfg; } finally { diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java index 2ffa11ebf2849..be99adf3b169d 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java @@ -19,6 +19,7 @@ import junit.framework.TestSuite; import org.apache.ignite.internal.ClusterNodeMetricsSelfTest; +import org.apache.ignite.internal.ClusterNodeMetricsUpdateTest; import org.apache.ignite.internal.GridAffinityNoCacheSelfTest; import org.apache.ignite.internal.GridAffinitySelfTest; import org.apache.ignite.internal.GridAlwaysFailoverSpiFailSelfTest; @@ -121,6 +122,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridAlwaysFailoverSpiFailSelfTest.class); suite.addTestSuite(GridTaskInstanceExecutionSelfTest.class); suite.addTestSuite(ClusterNodeMetricsSelfTest.class); + suite.addTestSuite(ClusterNodeMetricsUpdateTest.class); suite.addTestSuite(GridNonHistoryMetricsSelfTest.class); suite.addTestSuite(GridCancelledJobsMetricsSelfTest.class); suite.addTestSuite(GridCollisionJobsContextSelfTest.class); 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 b9ef1e44bb351..e26b2111f90c5 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryCancelSelfTest.java @@ -117,7 +117,7 @@ public void testQueryCancelsOnGridShutdown() throws Exception { } for (Ignite g : G.allGrids()) - if (!g.configuration().getDiscoverySpi().isClientMode()) + if (!g.configuration().isClientMode()) stopGrid(g.name(), true); } }, 1); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java index 97720d5666057..bd3b0939e92bd 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractBasicSelfTest.java @@ -89,11 +89,14 @@ public abstract class DynamicIndexAbstractBasicSelfTest extends DynamicIndexAbst * @param mode Mode. * @param atomicityMode Atomicity mode. * @param near Near flag. + * @throws Exception If failed. */ private void initialize(CacheMode mode, CacheAtomicityMode atomicityMode, boolean near) - throws IgniteCheckedException { + throws Exception { createSqlCache(node(), cacheConfiguration(mode, atomicityMode, near)); + awaitPartitionMapExchange(); + grid(IDX_CLI_NEAR_ONLY).getOrCreateNearCache(CACHE_NAME, new NearCacheConfiguration<>()); assertNoIndex(CACHE_NAME, TBL_NAME, IDX_NAME_1); diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridJtaTransactionManagerSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridJtaTransactionManagerSelfTest.java index a181068e8ae1e..5cad1673c1908 100644 --- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridJtaTransactionManagerSelfTest.java +++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridJtaTransactionManagerSelfTest.java @@ -44,13 +44,7 @@ public class GridJtaTransactionManagerSelfTest extends GridCommonAbstractTest { IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName). setCacheConfiguration(defaultCacheConfiguration().setCacheMode(PARTITIONED)); - cfg.getTransactionConfiguration().setTxManagerFactory(new Factory() { - private static final long serialVersionUID = 0L; - - @Override public TransactionManager create() { - return jotm.getTransactionManager(); - } - }); + cfg.getTransactionConfiguration().setTxManagerFactory(new TestTxManagerFactory()); return cfg; } @@ -205,4 +199,17 @@ public void testJtaTxContextSwitchWithExistingTx() throws Exception { cache.removeAll(); } } + + /** + * + */ + static class TestTxManagerFactory implements Factory { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public TransactionManager create() { + return jotm.getTransactionManager(); + } + } } diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactorySelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactorySelfTest.java index f6fd5c7675540..14b7fae291c27 100644 --- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactorySelfTest.java +++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/jta/GridPartitionedCacheJtaFactorySelfTest.java @@ -30,12 +30,19 @@ public class GridPartitionedCacheJtaFactorySelfTest extends AbstractCacheJtaSelf @Override protected void configureJta(IgniteConfiguration cfg) { TransactionConfiguration txCfg = cfg.getTransactionConfiguration(); - txCfg.setTxManagerFactory(new Factory() { - private static final long serialVersionUID = 0L; + txCfg.setTxManagerFactory(new TestTxManagerFactory()); + } + + /** + * + */ + static class TestTxManagerFactory implements Factory { + /** */ + private static final long serialVersionUID = 0L; - @Override public TransactionManager create() { - return jotm.getTransactionManager(); - } - }); + /** {@inheritDoc} */ + @Override public TransactionManager create() { + return jotm.getTransactionManager(); + } } } diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala index fce47a6e621ff..d87ea0a206d88 100644 --- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala +++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala @@ -25,6 +25,7 @@ import org.apache.ignite.internal.processors.cache.query.QueryCursorEx import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata import org.apache.ignite.lang.IgniteUuid import org.apache.ignite.spark.impl._ +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode import org.apache.spark._ import org.apache.spark.rdd.RDD @@ -91,8 +92,14 @@ class IgniteRDD[K, V] ( override protected[spark] def getPreferredLocations(split: Partition): Seq[String] = { ensureCache() - ic.ignite().affinity(cacheName).mapPartitionToPrimaryAndBackups(split.index) + if (ic.ignite().configuration().getDiscoverySpi().isInstanceOf[TcpDiscoverySpi]) { + ic.ignite().affinity(cacheName).mapPartitionToPrimaryAndBackups(split.index) .map(_.asInstanceOf[TcpDiscoveryNode].socketAddresses()).flatten.map(_.getHostName).toList + } + else { + ic.ignite().affinity(cacheName).mapPartitionToPrimaryAndBackups(split.index) + .flatten(_.hostNames).toSeq + } } /** diff --git a/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java b/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java index 75128fc374c45..b45385893c1b5 100644 --- a/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java +++ b/modules/spring/src/test/java/org/apache/ignite/internal/GridFactorySelfTest.java @@ -998,7 +998,8 @@ public void testRepeatingStart() throws Exception { startGrid("1", c); - assert ((TcpDiscoverySpi)c.getDiscoverySpi()).started(); + if (tcpDiscovery()) + assert ((TcpDiscoverySpi)c.getDiscoverySpi()).started(); try { startGrid("2", c); diff --git a/modules/spring/src/test/java/org/apache/ignite/p2p/GridP2PUserVersionChangeSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/p2p/GridP2PUserVersionChangeSelfTest.java index b861e192a9978..46da3cc18dec0 100644 --- a/modules/spring/src/test/java/org/apache/ignite/p2p/GridP2PUserVersionChangeSelfTest.java +++ b/modules/spring/src/test/java/org/apache/ignite/p2p/GridP2PUserVersionChangeSelfTest.java @@ -44,6 +44,7 @@ import org.apache.ignite.testsuites.IgniteIgnore; import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; import static org.apache.ignite.events.EventType.EVT_TASK_UNDEPLOYED; @@ -255,12 +256,12 @@ public void testRedeployOnNodeRestartSharedMode() throws Exception { ignite2.events().localListen(new IgnitePredicate() { @Override public boolean apply(Event evt) { - if (evt.type() == EVT_NODE_LEFT) + if (evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED) discoLatch.countDown(); return true; } - }, EVT_NODE_LEFT); + }, EVT_NODE_LEFT, EVT_NODE_FAILED); Integer res1 = (Integer)ignite1.compute().execute(task1, ignite2.cluster().localNode().id()); diff --git a/modules/yardstick/pom-standalone.xml b/modules/yardstick/pom-standalone.xml index 577a95ede2ab2..6905d94811c22 100644 --- a/modules/yardstick/pom-standalone.xml +++ b/modules/yardstick/pom-standalone.xml @@ -52,6 +52,12 @@ ${project.version} + + org.apache.ignite + ignite-zookeeper + ${project.version} + + org.apache.ignite ignite-log4j diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml index 8cad24b75b002..9923bb7c7ddf9 100644 --- a/modules/yardstick/pom.xml +++ b/modules/yardstick/pom.xml @@ -53,6 +53,12 @@ ${project.version} + + org.apache.ignite + ignite-zookeeper + ${project.version} + + org.apache.ignite ignite-log4j diff --git a/modules/zookeeper/pom.xml b/modules/zookeeper/pom.xml index c3c36797e3d54..2d47ece408719 100644 --- a/modules/zookeeper/pom.xml +++ b/modules/zookeeper/pom.xml @@ -47,6 +47,12 @@ ${curator.version} + + org.apache.curator + curator-recipes + ${curator.version} + + org.apache.curator curator-x-discovery @@ -107,6 +113,13 @@ test + + org.apache.ignite + ignite-indexing + ${project.version} + test + + org.apache.ignite ignite-log4j @@ -121,6 +134,13 @@ test + + com.thoughtworks.xstream + xstream + 1.4.8 + test + + org.apache.ignite ignite-core @@ -128,10 +148,30 @@ test-jar test + + + org.apache.ignite + ignite-indexing + ${project.version} + test-jar + test + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + org.apache.felix diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java new file mode 100644 index 0000000000000..860c71c16e972 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java @@ -0,0 +1,557 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk; + +import java.io.IOException; +import java.io.Serializable; +import java.net.InetAddress; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.curator.utils.PathUtils; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpiInternalListener; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.A; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.lang.IgniteProductVersion; +import org.apache.ignite.resources.LoggerResource; +import org.apache.ignite.spi.IgniteSpiAdapter; +import org.apache.ignite.spi.IgniteSpiConfiguration; +import org.apache.ignite.spi.IgniteSpiContext; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.IgniteSpiMultipleInstancesSupport; +import org.apache.ignite.spi.communication.CommunicationSpi; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.discovery.DiscoveryMetricsProvider; +import org.apache.ignite.spi.discovery.DiscoverySpi; +import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; +import org.apache.ignite.spi.discovery.DiscoverySpiDataExchange; +import org.apache.ignite.spi.discovery.DiscoverySpiHistorySupport; +import org.apache.ignite.spi.discovery.DiscoverySpiListener; +import org.apache.ignite.spi.discovery.DiscoverySpiMutableCustomMessageSupport; +import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator; +import org.apache.ignite.spi.discovery.DiscoverySpiOrderSupport; +import org.apache.ignite.spi.discovery.zk.internal.ZookeeperClusterNode; +import org.apache.ignite.spi.discovery.zk.internal.ZookeeperDiscoveryImpl; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT; +import static org.apache.ignite.IgniteSystemProperties.getBoolean; + +/** + * Zookeeper Discovery Spi. + */ +@IgniteSpiMultipleInstancesSupport(true) +@DiscoverySpiOrderSupport(true) +@DiscoverySpiHistorySupport(true) +@DiscoverySpiMutableCustomMessageSupport(false) +public class ZookeeperDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, IgniteDiscoverySpi { + /** */ + public static final String DFLT_ROOT_PATH = "/apacheIgnite"; + + /** */ + public static final long DFLT_JOIN_TIMEOUT = 0; + + /** */ + @GridToStringInclude + private String zkRootPath = DFLT_ROOT_PATH; + + /** */ + @GridToStringInclude + private String zkConnectionString; + + /** */ + private long joinTimeout = DFLT_JOIN_TIMEOUT; + + /** */ + @GridToStringInclude + private long sesTimeout; + + /** */ + private boolean clientReconnectDisabled; + + /** */ + @GridToStringExclude + private DiscoverySpiListener lsnr; + + /** */ + @GridToStringExclude + private DiscoverySpiDataExchange exchange; + + /** */ + @GridToStringExclude + private DiscoverySpiNodeAuthenticator nodeAuth; + + /** */ + @GridToStringExclude + private DiscoveryMetricsProvider metricsProvider; + + /** */ + @GridToStringExclude + private ZookeeperDiscoveryImpl impl; + + /** */ + @GridToStringExclude + private Map locNodeAttrs; + + /** */ + @GridToStringExclude + private IgniteProductVersion locNodeVer; + + /** */ + @GridToStringExclude + private Serializable consistentId; + + /** Local node addresses. */ + private IgniteBiTuple, Collection> addrs; + + /** */ + @LoggerResource + @GridToStringExclude + private IgniteLogger log; + + /** */ + private IgniteDiscoverySpiInternalListener internalLsnr; + + /** + * @return Base path in ZK for znodes created by SPI. + */ + public String getZkRootPath() { + return zkRootPath; + } + + /** + * @param zkRootPath Base path in ZooKeeper for znodes created by SPI. + * @return {@code this} for chaining. + */ + @IgniteSpiConfiguration(optional = true) + public ZookeeperDiscoverySpi setZkRootPath(String zkRootPath) { + this.zkRootPath = zkRootPath; + + return this; + } + + /** + * @return ZooKeeper session timeout. + */ + public long getSessionTimeout() { + return sesTimeout; + } + + /** + * @param sesTimeout ZooKeeper session timeout. + * @return {@code this} for chaining. + */ + @IgniteSpiConfiguration(optional = true) + public ZookeeperDiscoverySpi setSessionTimeout(long sesTimeout) { + this.sesTimeout = sesTimeout; + + return this; + } + + /** + * @return Cluster join timeout. + */ + public long getJoinTimeout() { + return joinTimeout; + } + + /** + * @param joinTimeout Cluster join timeout ({@code 0} means wait forever). + * @return {@code this} for chaining. + */ + @IgniteSpiConfiguration(optional = true) + public ZookeeperDiscoverySpi setJoinTimeout(long joinTimeout) { + this.joinTimeout = joinTimeout; + + return this; + } + + /** + * @return ZooKeeper connection string + */ + public String getZkConnectionString() { + return zkConnectionString; + } + + /** + * @param zkConnectionString ZooKeeper connection string + * @return {@code this} for chaining. + */ + @IgniteSpiConfiguration(optional = false) + public ZookeeperDiscoverySpi setZkConnectionString(String zkConnectionString) { + this.zkConnectionString = zkConnectionString; + + return this; + } + + /** + * If {@code true} client does not try to reconnect. + * + * @return Client reconnect disabled flag. + */ + public boolean isClientReconnectDisabled() { + return clientReconnectDisabled; + } + + /** + * Sets client reconnect disabled flag. + * + * @param clientReconnectDisabled Client reconnect disabled flag. + * @return {@code this} for chaining. + */ + @IgniteSpiConfiguration(optional = true) + public ZookeeperDiscoverySpi setClientReconnectDisabled(boolean clientReconnectDisabled) { + this.clientReconnectDisabled = clientReconnectDisabled; + + return this; + } + + /** {@inheritDoc} */ + @Override public boolean clientReconnectSupported() { + return !clientReconnectDisabled; + } + + /** {@inheritDoc} */ + @Override public void clientReconnect() { + impl.reconnect(); + } + + /** {@inheritDoc} */ + @Override public boolean knownNode(UUID nodeId) { + return impl.knownNode(nodeId); + } + + /** {@inheritDoc} */ + @Override public boolean supportsCommunicationFailureResolve() { + return true; + } + + /** {@inheritDoc} */ + @Override public void resolveCommunicationFailure(ClusterNode node, Exception err) { + impl.resolveCommunicationError(node, err); + } + + /** {@inheritDoc} */ + @Nullable @Override public Serializable consistentId() throws IgniteSpiException { + if (consistentId == null) { + consistentId = ignite.configuration().getConsistentId(); + + if (consistentId == null) { + initAddresses(); + + final List sortedAddrs = new ArrayList<>(addrs.get1()); + + Collections.sort(sortedAddrs); + + if (getBoolean(IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT)) + consistentId = U.consistentId(sortedAddrs); + else { + Integer commPort = null; + + if (locNodeAttrs != null) { + commPort = (Integer)locNodeAttrs.get( + TcpCommunicationSpi.class.getSimpleName() + "." + TcpCommunicationSpi.ATTR_PORT); + } + else { + CommunicationSpi commSpi = ignite.configuration().getCommunicationSpi(); + + if (commSpi instanceof TcpCommunicationSpi) { + commPort = ((TcpCommunicationSpi)commSpi).boundPort(); + + if (commPort == -1) + commPort = null; + } + } + + if (commPort == null) { + U.warn(log, "Can not initialize default consistentId, TcpCommunicationSpi port is not initialized."); + + consistentId = ignite.configuration().getNodeId(); + } + else + consistentId = U.consistentId(sortedAddrs, commPort); + } + } + } + + return consistentId; + } + + /** + * + */ + private void initAddresses() { + if (addrs == null) { + String locHost = ignite != null ? ignite.configuration().getLocalHost() : null; + + InetAddress locAddr; + + try { + locAddr = U.resolveLocalHost(locHost); + } + catch (IOException e) { + throw new IgniteSpiException("Unknown local address: " + locHost, e); + } + + try { + addrs = U.resolveLocalAddresses(locAddr); + } + catch (Exception e) { + throw new IgniteSpiException("Failed to resolve local host to set of external addresses: " + locHost, + e); + } + } + } + + /** {@inheritDoc} */ + @Override public Collection getRemoteNodes() { + return impl.remoteNodes(); + } + + /** {@inheritDoc} */ + @Override public ClusterNode getLocalNode() { + return impl != null ? impl.localNode() : null; + } + + /** {@inheritDoc} */ + @Nullable @Override public ClusterNode getNode(UUID nodeId) { + return impl.node(nodeId); + } + + /** {@inheritDoc} */ + @Override public boolean pingNode(UUID nodeId) { + return impl.pingNode(nodeId); + } + + /** {@inheritDoc} */ + @Override public void setNodeAttributes(Map attrs, IgniteProductVersion ver) { + assert locNodeAttrs == null; + assert locNodeVer == null; + + if (log.isDebugEnabled()) { + log.debug("Node attributes to set: " + attrs); + log.debug("Node version to set: " + ver); + } + + locNodeAttrs = attrs; + locNodeVer = ver; + } + + /** {@inheritDoc} */ + @Override public void setListener(@Nullable DiscoverySpiListener lsnr) { + this.lsnr = lsnr; + } + + /** {@inheritDoc} */ + @Override public void setDataExchange(DiscoverySpiDataExchange exchange) { + this.exchange = exchange; + } + + /** {@inheritDoc} */ + @Override public void setMetricsProvider(DiscoveryMetricsProvider metricsProvider) { + this.metricsProvider = metricsProvider; + } + + /** {@inheritDoc} */ + @Override public void disconnect() throws IgniteSpiException { + impl.stop(); + } + + /** {@inheritDoc} */ + @Override public void setAuthenticator(DiscoverySpiNodeAuthenticator auth) { + this.nodeAuth = auth; + } + + /** + * @return Authenticator. + */ + public DiscoverySpiNodeAuthenticator getAuthenticator() { + return nodeAuth; + } + + /** {@inheritDoc} */ + @Override public long getGridStartTime() { + return impl.gridStartTime(); + } + + /** {@inheritDoc} */ + @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) { + IgniteDiscoverySpiInternalListener internalLsnr = impl.internalLsnr; + + if (internalLsnr != null) { + if (!internalLsnr.beforeSendCustomEvent(this, log, msg)) + return; + } + + impl.sendCustomMessage(msg); + } + + /** {@inheritDoc} */ + @Override public void failNode(UUID nodeId, @Nullable String warning) { + impl.failNode(nodeId, warning); + } + + /** {@inheritDoc} */ + @Override public boolean isClientMode() throws IllegalStateException { + return impl.localNode().isClient(); + } + + /** {@inheritDoc} */ + @Override protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException { + super.onContextInitialized0(spiCtx); + } + + /** {@inheritDoc} */ + @Override public void spiStart(@Nullable String igniteInstanceName) throws IgniteSpiException { + if (sesTimeout == 0) + sesTimeout = ignite.configuration().getFailureDetectionTimeout().intValue(); + + assertParameter(sesTimeout > 0, "sessionTimeout > 0"); + + A.notNullOrEmpty(zkConnectionString, "zkConnectionString can not be empty"); + + A.notNullOrEmpty(zkRootPath, "zkRootPath can not be empty"); + + zkRootPath = zkRootPath.trim(); + + if (zkRootPath.endsWith("/")) + zkRootPath = zkRootPath.substring(0, zkRootPath.length() - 1); + + try { + PathUtils.validatePath(zkRootPath); + } + catch (IllegalArgumentException e) { + throw new IgniteSpiException("zkRootPath is invalid: " + zkRootPath, e); + } + + ZookeeperClusterNode locNode = initLocalNode(); + + if (log.isInfoEnabled()) { + log.info("Start Zookeeper discovery [zkConnectionString=" + zkConnectionString + + ", sessionTimeout=" + sesTimeout + + ", zkRootPath=" + zkRootPath + ']'); + } + + impl = new ZookeeperDiscoveryImpl( + this, + igniteInstanceName, + log, + zkRootPath, + locNode, + lsnr, + exchange, + internalLsnr); + + try { + impl.startJoinAndWait(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + + throw new IgniteSpiException("Failed to join cluster, thread was interrupted", e); + } + } + + /** {@inheritDoc} */ + @Override public void setInternalListener(IgniteDiscoverySpiInternalListener lsnr) { + if (impl != null) + impl.internalLsnr = lsnr; + else + internalLsnr = lsnr; + } + + /** {@inheritDoc} */ + @Override public void simulateNodeFailure() { + impl.simulateNodeFailure(); + } + + /** {@inheritDoc} */ + @Override public void spiStop() throws IgniteSpiException { + if (impl != null) + impl.stop(); + } + + /** + * @return Local node instance. + */ + private ZookeeperClusterNode initLocalNode() { + assert ignite != null; + + initAddresses(); + + ZookeeperClusterNode locNode = new ZookeeperClusterNode( + ignite.configuration().getNodeId(), + addrs.get1(), + addrs.get2(), + locNodeVer, + locNodeAttrs, + consistentId(), + sesTimeout, + ignite.configuration().isClientMode(), + metricsProvider); + + locNode.local(true); + + DiscoverySpiListener lsnr = this.lsnr; + + if (lsnr != null) + lsnr.onLocalNodeInitialized(locNode); + + if (log.isDebugEnabled()) + log.debug("Local node initialized: " + locNode); + + if (metricsProvider != null) { + locNode.setMetrics(metricsProvider.metrics()); + locNode.setCacheMetrics(metricsProvider.cacheMetrics()); + } + + return locNode; + } + + /** + * Used in tests (called via reflection). + * + * @return Copy of SPI. + */ + private ZookeeperDiscoverySpi cloneSpiConfiguration() { + ZookeeperDiscoverySpi spi = new ZookeeperDiscoverySpi(); + + spi.setZkRootPath(zkRootPath); + spi.setZkConnectionString(zkConnectionString); + spi.setSessionTimeout(sesTimeout); + spi.setJoinTimeout(joinTimeout); + spi.setClientReconnectDisabled(clientReconnectDisabled); + + return spi; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ZookeeperDiscoverySpi.class, this); + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractCallabck.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractCallabck.java new file mode 100644 index 0000000000000..b80a9ddbf129a --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractCallabck.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import org.apache.ignite.internal.util.GridSpinBusyLock; + +/** + * + */ +abstract class ZkAbstractCallabck { + /** */ + final ZkRuntimeState rtState; + + /** */ + private final ZookeeperDiscoveryImpl impl; + + /** */ + private final GridSpinBusyLock busyLock; + + /** + * @param rtState Runtime state. + * @param impl Discovery impl. + */ + ZkAbstractCallabck(ZkRuntimeState rtState, ZookeeperDiscoveryImpl impl) { + this.rtState = rtState; + this.impl = impl; + + busyLock = impl.busyLock; + } + + /** + * @return {@code True} if is able to start processing. + */ + final boolean onProcessStart() { + boolean start = rtState.errForClose == null && busyLock.enterBusy(); + + if (!start) { + assert rtState.errForClose != null; + + onStartFailed(); + + return false; + } + + return true; + } + + /** + * + */ + void onStartFailed() { + // No-op. + } + + /** + * + */ + final void onProcessEnd() { + busyLock.leaveBusy(); + } + + /** + * @param e Error. + */ + final void onProcessError(Throwable e) { + impl.onFatalError(busyLock, e); + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractChildrenCallback.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractChildrenCallback.java new file mode 100644 index 0000000000000..2292e35056041 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractChildrenCallback.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.util.List; +import org.apache.zookeeper.AsyncCallback; +import org.apache.zookeeper.data.Stat; + +/** + * + */ +abstract class ZkAbstractChildrenCallback extends ZkAbstractCallabck implements AsyncCallback.Children2Callback { + /** + * @param rtState Runtime state. + * @param impl Discovery impl. + */ + ZkAbstractChildrenCallback(ZkRuntimeState rtState, ZookeeperDiscoveryImpl impl) { + super(rtState, impl); + } + + /** {@inheritDoc} */ + @Override public void processResult(int rc, String path, Object ctx, List children, Stat stat) { + if (!onProcessStart()) + return; + + try { + processResult0(rc, path, ctx, children, stat); + + onProcessEnd(); + } + catch (Throwable e) { + onProcessError(e); + } + } + + /** + * @param rc + * @param path + * @param ctx + * @param children + * @param stat + * @throws Exception If failed. + */ + abstract void processResult0(int rc, String path, Object ctx, List children, Stat stat) + throws Exception; +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractWatcher.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractWatcher.java new file mode 100644 index 0000000000000..9098d0520a52a --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAbstractWatcher.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; + +/** + * + */ +abstract class ZkAbstractWatcher extends ZkAbstractCallabck implements Watcher { + /** + * @param rtState Runtime state. + * @param impl Discovery impl. + */ + ZkAbstractWatcher(ZkRuntimeState rtState, ZookeeperDiscoveryImpl impl) { + super(rtState, impl); + } + + /** {@inheritDoc} */ + @Override public final void process(WatchedEvent evt) { + if (!onProcessStart()) + return; + + try { + process0(evt); + + onProcessEnd(); + } + catch (Throwable e) { + onProcessError(e); + } + } + + /** + * @param evt Event. + * @throws Exception If failed. + */ + protected abstract void process0(WatchedEvent evt) throws Exception; +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAliveNodeData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAliveNodeData.java new file mode 100644 index 0000000000000..d82437758bf83 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkAliveNodeData.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.io.Serializable; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * Zk Alive Node Data. + */ +public class ZkAliveNodeData implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + long lastProcEvt = -1; + + /** */ + transient boolean needUpdate; + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ZkAliveNodeData.class, this); + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkBulkJoinContext.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkBulkJoinContext.java new file mode 100644 index 0000000000000..a186aed526567 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkBulkJoinContext.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import org.apache.ignite.internal.util.typedef.T2; + +/** + * + */ +class ZkBulkJoinContext { + /** */ + List>> nodes; + + /** + * @param nodeEvtData Node event data. + * @param discoData Discovery data for node. + */ + void addJoinedNode(ZkJoinedNodeEvtData nodeEvtData, Map discoData) { + if (nodes == null) + nodes = new ArrayList<>(); + + nodes.add(new T2<>(nodeEvtData, discoData)); + } + + /** + * @return Number of joined nodes. + */ + int nodes() { + return nodes != null ? nodes.size() : 0; + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkClusterNodes.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkClusterNodes.java new file mode 100644 index 0000000000000..7e2ea7b6e83b3 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkClusterNodes.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentSkipListMap; +import org.apache.ignite.cluster.ClusterNode; + +/** + * Zk Cluster Nodes. + */ +public class ZkClusterNodes { + /** */ + final ConcurrentSkipListMap nodesByOrder = new ConcurrentSkipListMap<>(); + + /** */ + final ConcurrentSkipListMap nodesByInternalId = new ConcurrentSkipListMap<>(); + + /** */ + final ConcurrentHashMap nodesById = new ConcurrentHashMap<>(); + + /** + * @return Remote nodes. + */ + public Collection remoteNodes() { + List nodes = new ArrayList<>(); + + for (ClusterNode node : nodesById.values()) { + if (!node.isLocal()) + nodes.add(node); + } + + return nodes; + } + + /** + * @return Current nodes in topology. + */ + @SuppressWarnings("unchecked") + List topologySnapshot() { + return new ArrayList<>((Collection)nodesByOrder.values()); + } + + /** + * @param node New node. + */ + void addNode(ZookeeperClusterNode node) { + assert node.id() != null : node; + assert node.order() > 0 : node; + + ZookeeperClusterNode old = nodesById.put(node.id(), node); + + assert old == null : old; + + old = nodesByOrder.put(node.order(), node); + + assert old == null : old; + + old = nodesByInternalId.put(node.internalId(), node); + + assert old == null : old; + } + + /** + * @param internalId Node internal ID. + * @return Removed node. + */ + ZookeeperClusterNode removeNode(long internalId) { + ZookeeperClusterNode node = nodesByInternalId.remove(internalId); + + assert node != null : internalId; + assert node.order() > 0 : node; + + Object rvmd = nodesByOrder.remove(node.order()); + + assert rvmd != null; + + rvmd = nodesById.remove(node.id()); + + assert rvmd != null; + + return node; + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorNodeState.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorNodeState.java new file mode 100644 index 0000000000000..9c21f13889cfd --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorNodeState.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.io.Serializable; +import java.util.BitSet; + +/** + * + */ +class ZkCommunicationErrorNodeState implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + final BitSet commState; + + /** */ + final Exception err; + + /** + * @param commState Communication state. + * @param err Error if failed get communication state.. + */ + ZkCommunicationErrorNodeState(BitSet commState, Exception err) { + assert commState != null || err != null; + + this.commState = commState; + this.err = err; + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorProcessFuture.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorProcessFuture.java new file mode 100644 index 0000000000000..accda6e5fb6c1 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorProcessFuture.java @@ -0,0 +1,411 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.util.BitSet; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.future.GridFinishedFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.spi.IgniteSpiTimeoutObject; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.jboss.netty.util.internal.ConcurrentHashMap; +import org.jetbrains.annotations.Nullable; + +/** + * Future is created on each node when either connection error occurs or resolve communication error request + * received. + */ +class ZkCommunicationErrorProcessFuture extends GridFutureAdapter implements IgniteSpiTimeoutObject, Runnable { + /** */ + private final ZookeeperDiscoveryImpl impl; + + /** */ + private final IgniteLogger log; + + /** */ + private final Map> nodeFuts = new ConcurrentHashMap<>(); + + /** */ + private final long endTime; + + /** */ + private final IgniteUuid id; + + /** */ + private State state; + + /** */ + private long resolveTopVer; + + /** */ + private Set resFailedNodes; + + /** */ + private Exception resErr; + + /** */ + private ZkDistributedCollectDataFuture collectResFut; + + /** + * @param impl Discovery impl. + * @param timeout Timeout to wait before initiating resolve process. + * @return Future. + */ + static ZkCommunicationErrorProcessFuture createOnCommunicationError(ZookeeperDiscoveryImpl impl, long timeout) { + return new ZkCommunicationErrorProcessFuture(impl, State.WAIT_TIMEOUT, timeout); + } + + /** + * @param impl Discovery impl. + * @return Future. + */ + static ZkCommunicationErrorProcessFuture createOnStartResolveRequest(ZookeeperDiscoveryImpl impl) { + return new ZkCommunicationErrorProcessFuture(impl, State.RESOLVE_STARTED, 0); + } + + /** + * @param impl Discovery implementation. + * @param state Initial state. + * @param timeout Wait timeout before initiating communication errors resolve. + */ + private ZkCommunicationErrorProcessFuture(ZookeeperDiscoveryImpl impl, State state, long timeout) { + assert state != State.DONE; + + this.impl = impl; + this.log = impl.log(); + + if (state == State.WAIT_TIMEOUT) { + assert timeout > 0 : timeout; + + id = IgniteUuid.fromUuid(impl.localNode().id()); + endTime = System.currentTimeMillis() + timeout; + } + else { + id = null; + endTime = 0; + } + + this.state = state; + } + + /** {@inheritDoc} */ + @Nullable @Override public IgniteLogger logger() { + return log; + } + + /** + * @param collectResFut Collect nodes' communication status future. + */ + void nodeResultCollectFuture(ZkDistributedCollectDataFuture collectResFut) { + assert this.collectResFut == null : collectResFut; + + this.collectResFut = collectResFut; + } + + /** + * @param top Topology. + * @throws Exception If failed. + */ + void onTopologyChange(ZkClusterNodes top) throws Exception { + for (Map.Entry> e : nodeFuts.entrySet()) { + if (!top.nodesByOrder.containsKey(e.getKey())) + e.getValue().onDone(false); + } + + if (collectResFut != null) + collectResFut.onTopologyChange(top); + } + + /** + * @param rtState Runtime state. + * @param futPath Future path. + * @param nodes Nodes to ping. + */ + void checkConnection(final ZkRuntimeState rtState, final String futPath, List nodes) { + final TcpCommunicationSpi spi = (TcpCommunicationSpi)impl.spi.ignite().configuration().getCommunicationSpi(); + + IgniteFuture fut = spi.checkConnection(nodes); + + fut.listen(new IgniteInClosure>() { + @Override public void apply(final IgniteFuture fut) { + // Future completed either from NIO thread or timeout worker, save result from another thread. + impl.runInWorkerThread(new ZkRunnable(rtState, impl) { + @Override public void run0() throws Exception { + BitSet commState = null; + Exception err = null; + + try { + commState = fut.get(); + } + catch (Exception e) { + err = e; + } + + ZkCommunicationErrorNodeState state = new ZkCommunicationErrorNodeState(commState, err); + + ZkDistributedCollectDataFuture.saveNodeResult(futPath, + rtState.zkClient, + impl.localNode().order(), + impl.marshalZip(state)); + } + + @Override void onStartFailed() { + onError(rtState.errForClose); + } + }); + + } + }); + } + + /** + * + */ + void scheduleCheckOnTimeout() { + synchronized (this) { + if (state == State.WAIT_TIMEOUT) + impl.spi.getSpiContext().addTimeoutObject(this); + } + } + + /** + * @param topVer Topology version. + * @return {@code False} if future was already completed and need create another future instance. + */ + boolean onStartResolveRequest(long topVer) { + synchronized (this) { + if (state == State.DONE) + return false; + + if (state == State.WAIT_TIMEOUT) + impl.spi.getSpiContext().removeTimeoutObject(this); + + assert resolveTopVer == 0 : resolveTopVer; + + resolveTopVer = topVer; + + state = State.RESOLVE_STARTED; + } + + return true; + } + + /** + * @param err Error. + */ + void onError(Exception err) { + assert err != null; + + Map> futs; + + synchronized (this) { + if (state == State.DONE) { + assert resErr != null; + + return; + } + + state = State.DONE; + + resErr = err; + + futs = nodeFuts; // nodeFuts should not be modified after state changed to DONE. + } + + for (Map.Entry> e : futs.entrySet()) + e.getValue().onDone(err); + + onDone(err); + } + + /** + * @param failedNodes Node failed as result of resolve process. + */ + void onFinishResolve(Set failedNodes) { + Map> futs; + + synchronized (this) { + if (state == State.DONE) { + assert resErr != null; + + return; + } + + assert state == State.RESOLVE_STARTED : state; + + state = State.DONE; + + resFailedNodes = failedNodes; + + futs = nodeFuts; // nodeFuts should not be modified after state changed to DONE. + } + + for (Map.Entry> e : futs.entrySet()) { + Boolean res = !F.contains(resFailedNodes, e.getKey()); + + e.getValue().onDone(res); + } + + onDone(); + } + + /** + * @param node Node. + * @return Future finished when communication error resolve is done or {@code null} if another + * resolve process should be started. + */ + @Nullable IgniteInternalFuture nodeStatusFuture(ClusterNode node) { + GridFutureAdapter fut; + + synchronized (this) { + if (state == State.DONE) { + if (resolveTopVer != 0 && node.order() <= resolveTopVer) { + Boolean res = !F.contains(resFailedNodes, node.order()); + + return new GridFinishedFuture<>(res); + } + else + return null; + } + + fut = nodeFuts.get(node.order()); + + if (fut == null) + nodeFuts.put(node.order(), fut = new GridFutureAdapter<>()); + } + + if (impl.node(node.order()) == null) + fut.onDone(false); + + return fut; + } + + /** {@inheritDoc} */ + @Override public void run() { + // Run from zk discovery worker pool after timeout. + if (needProcessTimeout()) { + try { + UUID reqId = UUID.randomUUID(); + + if (log.isInfoEnabled()) { + log.info("Initiate cluster-wide communication error resolve process [reqId=" + reqId + + ", errNodes=" + nodeFuts.size() + ']'); + } + + impl.sendCustomMessage(new ZkCommunicationErrorResolveStartMessage(reqId)); + } + catch (Exception e) { + Collection> futs; + + synchronized (this) { + if (state != State.WAIT_TIMEOUT) + return; + + state = State.DONE; + resErr = e; + + futs = nodeFuts.values(); // nodeFuts should not be modified after state changed to DONE. + } + + for (GridFutureAdapter fut : futs) + fut.onDone(e); + + onDone(e); + } + } + } + + /** + * @return {@code True} if need initiate resolve process after timeout expired. + */ + private boolean needProcessTimeout() { + synchronized (this) { + if (state != State.WAIT_TIMEOUT) + return false; + + for (GridFutureAdapter fut : nodeFuts.values()) { + if (!fut.isDone()) + return true; + } + + state = State.DONE; + } + + onDone(null, null); + + return false; + } + + /** {@inheritDoc} */ + @Override public IgniteUuid id() { + return id; + } + + /** {@inheritDoc} */ + @Override public long endTime() { + return endTime; + } + + /** {@inheritDoc} */ + @Override public void onTimeout() { + if (needProcessTimeout()) + impl.runInWorkerThread(this); + } + + /** {@inheritDoc} */ + @Override public boolean onDone(@Nullable Void res, @Nullable Throwable err) { + if (super.onDone(res, err)) { + impl.clearCommunicationErrorProcessFuture(this); + + return true; + } + + return false; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ZkCommunicationErrorProcessFuture.class, this); + } + + /** + * + */ + enum State { + /** */ + DONE, + + /** */ + WAIT_TIMEOUT, + + /** */ + RESOLVE_STARTED + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveFinishMessage.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveFinishMessage.java new file mode 100644 index 0000000000000..9b7476c5355f5 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveFinishMessage.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.util.UUID; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +class ZkCommunicationErrorResolveFinishMessage implements DiscoverySpiCustomMessage, ZkInternalMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + final UUID futId; + + /** */ + final long topVer; + + /** */ + transient ZkCommunicationErrorResolveResult res; + + /** + * @param futId Future ID. + * @param topVer Topology version when resolve process finished. + */ + ZkCommunicationErrorResolveFinishMessage(UUID futId, long topVer) { + this.futId = futId; + this.topVer = topVer; + } + + /** {@inheritDoc} */ + @Nullable @Override public DiscoverySpiCustomMessage ackMessage() { + return null; + } + + /** {@inheritDoc} */ + @Override public boolean isMutable() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ZkCommunicationErrorResolveFinishMessage.class, this); + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveResult.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveResult.java new file mode 100644 index 0000000000000..23495aae72df1 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveResult.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.io.Serializable; +import org.apache.ignite.internal.util.GridLongList; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +class ZkCommunicationErrorResolveResult implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + final GridLongList killedNodes; + + /** */ + final Exception err; + + /** + * @param killedNodes Killed nodes. + * @param err Error. + */ + ZkCommunicationErrorResolveResult(@Nullable GridLongList killedNodes, Exception err) { + this.killedNodes = killedNodes; + this.err = err; + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveStartMessage.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveStartMessage.java new file mode 100644 index 0000000000000..0c79c36aee0a9 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationErrorResolveStartMessage.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.util.UUID; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; +import org.jetbrains.annotations.Nullable; + +/** + * Zk Communication Error Resolve Start Message. + */ +public class ZkCommunicationErrorResolveStartMessage implements DiscoverySpiCustomMessage, ZkInternalMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + final UUID id; + + /** + * @param id Unique ID. + */ + ZkCommunicationErrorResolveStartMessage(UUID id) { + this.id = id; + } + + /** {@inheritDoc} */ + @Nullable @Override public DiscoverySpiCustomMessage ackMessage() { + return null; + } + + /** {@inheritDoc} */ + @Override public boolean isMutable() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ZkCommunicationErrorResolveStartMessage.class, this); + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationFailureContext.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationFailureContext.java new file mode 100644 index 0000000000000..d27b717485e2c --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkCommunicationFailureContext.java @@ -0,0 +1,188 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.util.ArrayList; +import java.util.BitSet; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.CommunicationFailureContext; +import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; + +/** + * + */ +class ZkCommunicationFailureContext implements CommunicationFailureContext { + /** */ + private static final Comparator NODE_ORDER_CMP = new Comparator() { + @Override public int compare(ClusterNode node1, ClusterNode node2) { + return Long.compare(node1.order(), node2.order()); + } + }; + + /** */ + private Set killedNodes = new HashSet<>(); + + /** */ + private final Map nodesState; + + /** */ + private final List initialNodes; + + /** */ + private final List curNodes; + + /** */ + private final GridCacheSharedContext ctx; + + /** + * @param ctx Context. + * @param curNodes Current topology snapshot. + * @param initialNodes Topology snapshot when communication error resolve started. + * @param nodesState Nodes communication state. + */ + ZkCommunicationFailureContext( + GridCacheSharedContext ctx, + List curNodes, + List initialNodes, + Map nodesState) + { + this.ctx = ctx; + this.curNodes = Collections.unmodifiableList(curNodes); + this.initialNodes = initialNodes; + this.nodesState = nodesState; + } + + /** {@inheritDoc} */ + @Override public List topologySnapshot() { + return curNodes; + } + + /** {@inheritDoc} */ + @Override public boolean connectionAvailable(ClusterNode node1, ClusterNode node2) { + BitSet nodeState = nodesState.get(node1.id()); + + if (nodeState == null) + throw new IllegalArgumentException("Invalid node: " + node1); + + int nodeIdx = Collections.binarySearch(initialNodes, node2, NODE_ORDER_CMP); + + if (nodeIdx < 0) + throw new IllegalArgumentException("Invalid node: " + node2); + + assert nodeIdx < nodeState.size() : nodeIdx; + + return nodeState.get(nodeIdx); + } + + /** {@inheritDoc} */ + @Override public Map> startedCaches() { + Map cachesMap = ctx.affinity().caches(); + + Map> res = U.newHashMap(cachesMap.size()); + + for (DynamicCacheDescriptor desc : cachesMap.values()) { + if (desc.cacheType().userCache()) + res.put(desc.cacheName(), desc.cacheConfiguration()); + } + + return res; + } + + /** {@inheritDoc} */ + @Override public List> cacheAffinity(String cacheName) { + if (cacheName == null) + throw new NullPointerException("Null cache name."); + + DynamicCacheDescriptor cacheDesc = ctx.affinity().caches().get(CU.cacheId(cacheName)); + + if (cacheDesc == null) + throw new IllegalArgumentException("Invalid cache name: " + cacheName); + + GridAffinityAssignmentCache aff = ctx.affinity().groupAffinity(cacheDesc.groupId()); + + assert aff != null : cacheName; + + return aff.readyAssignments(aff.lastVersion()); + } + + /** {@inheritDoc} */ + @Override public List> cachePartitionOwners(String cacheName) { + if (cacheName == null) + throw new NullPointerException("Null cache name."); + + DynamicCacheDescriptor cacheDesc = ctx.affinity().caches().get(CU.cacheId(cacheName)); + + if (cacheDesc == null) + throw new IllegalArgumentException("Invalid cache name: " + cacheName); + + if (cacheDesc.cacheConfiguration().getCacheMode() == CacheMode.LOCAL) + return Collections.emptyList(); + + CacheGroupContext grp = ctx.cache().cacheGroup(cacheDesc.groupId()); + + GridDhtPartitionTopology top; + + if (grp == null) { + top = ctx.exchange().clientTopologyIfExists(cacheDesc.groupId()); + + assert top != null : cacheName; + } + else + top = grp.topology(); + + return top.allOwners(); + } + + /** {@inheritDoc} */ + @Override public void killNode(ClusterNode node) { + if (node == null) + throw new NullPointerException(); + + if (Collections.binarySearch(curNodes, node, NODE_ORDER_CMP) < 0) + throw new IllegalArgumentException("Invalid node: " + node); + + killedNodes.add(node); + } + + /** + * @return Nodes to fail. + */ + Set killedNodes() { + return killedNodes; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "ZkCommunicationFailureContext []"; + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryCustomEventData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryCustomEventData.java new file mode 100644 index 0000000000000..21dfe628e4ff9 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryCustomEventData.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.util.UUID; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; + +/** + * + */ +class ZkDiscoveryCustomEventData extends ZkDiscoveryEventData { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + final long origEvtId; + + /** */ + final UUID sndNodeId; + + /** */ + final String evtPath; + + /** Message instance (can be marshalled as part of ZkDiscoveryCustomEventData or stored in separate znode. */ + DiscoverySpiCustomMessage msg; + + /** Unmarshalled message. */ + transient DiscoverySpiCustomMessage resolvedMsg; + + /** + * @param evtId Event ID. + * @param origEvtId For acknowledge events ID of original event. + * @param topVer Topology version. + * @param sndNodeId Sender node ID. + * @param msg Message instance. + * @param evtPath Event path. + */ + ZkDiscoveryCustomEventData( + long evtId, + long origEvtId, + long topVer, + UUID sndNodeId, + DiscoverySpiCustomMessage msg, + String evtPath) + { + super(evtId, ZK_EVT_CUSTOM_EVT, topVer); + + assert sndNodeId != null; + assert msg != null || origEvtId != 0 || !F.isEmpty(evtPath); + + this.origEvtId = origEvtId; + this.msg = msg; + this.sndNodeId = sndNodeId; + this.evtPath = evtPath; + } + + /** + * @return {@code True} for custom event ack message. + */ + boolean ackEvent() { + return origEvtId != 0; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "ZkDiscoveryCustomEventData [" + + "evtId=" + eventId() + + ", topVer=" + topologyVersion() + + ", sndNode=" + sndNodeId + + ", ack=" + ackEvent() + + ']'; + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventData.java new file mode 100644 index 0000000000000..d667a17f6643c --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventData.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Set; +import org.apache.ignite.internal.util.typedef.internal.U; + +/** + * + */ +abstract class ZkDiscoveryEventData implements Serializable { + /** */ + static final byte ZK_EVT_NODE_JOIN = 1; + + /** */ + static final byte ZK_EVT_NODE_FAILED = 2; + + /** */ + static final byte ZK_EVT_CUSTOM_EVT = 3; + + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private final long evtId; + + /** */ + private final byte evtType; + + /** */ + private final long topVer; + + /** */ + private transient Set remainingAcks; + + /** */ + int flags; + + /** + * @param evtId Event ID. + * @param evtType Event type. + * @param topVer Topology version. + */ + ZkDiscoveryEventData(long evtId, byte evtType, long topVer) { + assert evtType == ZK_EVT_NODE_JOIN || evtType == ZK_EVT_NODE_FAILED || evtType == ZK_EVT_CUSTOM_EVT : evtType; + + this.evtId = evtId; + this.evtType = evtType; + this.topVer = topVer; + } + + /** + * @param nodes Current nodes in topology. + */ + void initRemainingAcks(Collection nodes) { + assert remainingAcks == null : this; + + remainingAcks = U.newHashSet(nodes.size()); + + for (ZookeeperClusterNode node : nodes) { + if (!node.isLocal() && node.order() <= topVer) { + boolean add = remainingAcks.add(node.internalId()); + + assert add : node; + } + } + } + + /** + * @param node Node. + */ + void addRemainingAck(ZookeeperClusterNode node) { + assert node.order() <= topVer : node; + + boolean add = remainingAcks.add(node.internalId()); + + assert add : node; + } + + /** + * @return {@code True} if all nodes processed event. + */ + boolean allAcksReceived() { + return remainingAcks.isEmpty(); + } + + /** + * @return Remaining acks. + */ + Set remainingAcks() { + return remainingAcks; + } + + /** + * @param nodeInternalId Node ID. + * @param ackEvtId Last event ID processed on node. + * @return {@code True} if all nodes processed event. + */ + boolean onAckReceived(Long nodeInternalId, long ackEvtId) { + assert remainingAcks != null; + + if (ackEvtId >= evtId) + remainingAcks.remove(nodeInternalId); + + return remainingAcks.isEmpty(); + } + + /** + * @param node Failed node. + * @return {@code True} if all nodes processed event. + */ + boolean onNodeFail(ZookeeperClusterNode node) { + assert remainingAcks != null : this; + + remainingAcks.remove(node.internalId()); + + return remainingAcks.isEmpty(); + } + + /** + * @param flag Flag mask. + * @return {@code True} if flag set. + */ + boolean flagSet(int flag) { + return (flags & flag) == flag; + } + + /** + * @return Event ID. + */ + long eventId() { + return evtId; + } + + /** + * @return Event type. + */ + byte eventType() { + return evtType; + } + + /** + * @return Event topology version. + */ + long topologyVersion() { + return topVer; + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventsData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventsData.java new file mode 100644 index 0000000000000..dce861b523e4c --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventsData.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.io.Serializable; +import java.util.Collection; +import java.util.TreeMap; +import java.util.UUID; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +class ZkDiscoveryEventsData implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** Unique cluster ID (generated when first node in cluster starts). */ + final UUID clusterId; + + /** Internal order of last processed custom event. */ + long procCustEvt = -1; + + /** Event ID counter. */ + long evtIdGen; + + /** Current topology version. */ + long topVer; + + /** Max node internal order in cluster. */ + long maxInternalOrder; + + /** Cluster start time (recorded when first node in cluster starts). */ + final long clusterStartTime; + + /** Events to process. */ + final TreeMap evts; + + /** ID of current active communication error resolve process. */ + private UUID commErrFutId; + + /** + * @param clusterStartTime Start time of first node in cluster. + * @return Events. + */ + static ZkDiscoveryEventsData createForNewCluster(long clusterStartTime) { + return new ZkDiscoveryEventsData( + UUID.randomUUID(), + clusterStartTime, + 1L, + new TreeMap() + ); + } + + /** + * @param clusterId Cluster ID. + * @param topVer Current topology version. + * @param clusterStartTime Cluster start time. + * @param evts Events history. + */ + private ZkDiscoveryEventsData( + UUID clusterId, + long clusterStartTime, + long topVer, + TreeMap evts) + { + this.clusterId = clusterId; + this.clusterStartTime = clusterStartTime; + this.topVer = topVer; + this.evts = evts; + } + + /** + * @param node Joined node. + */ + void onNodeJoin(ZookeeperClusterNode node) { + if (node.internalId() > maxInternalOrder) + maxInternalOrder = node.internalId(); + } + + /** + * @return Future ID. + */ + @Nullable UUID communicationErrorResolveFutureId() { + return commErrFutId; + } + + /** + * @param id Future ID. + */ + void communicationErrorResolveFutureId(@Nullable UUID id) { + commErrFutId = id; + } + + /** + * @param nodes Current nodes in topology (these nodes should ack that event processed). + * @param evt Event. + */ + void addEvent(Collection nodes, ZkDiscoveryEventData evt) { + Object old = evts.put(evt.eventId(), evt); + + assert old == null : old; + + evt.initRemainingAcks(nodes); + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeFailEventData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeFailEventData.java new file mode 100644 index 0000000000000..c76158ff090a7 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeFailEventData.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +/** + * + */ +class ZkDiscoveryNodeFailEventData extends ZkDiscoveryEventData { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private long failedNodeInternalId; + + /** + * @param evtId Event ID. + * @param topVer Topology version. + * @param failedNodeInternalId Failed node ID. + */ + ZkDiscoveryNodeFailEventData(long evtId, long topVer, long failedNodeInternalId) { + super(evtId, ZK_EVT_NODE_FAILED, topVer); + + this.failedNodeInternalId = failedNodeInternalId; + } + + /** + * @return Failed node ID. + */ + long failedNodeInternalId() { + return failedNodeInternalId; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "ZkDiscoveryNodeFailEventData [" + + "evtId=" + eventId() + + ", topVer=" + topologyVersion() + + ", nodeId=" + failedNodeInternalId + ']'; + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeJoinEventData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeJoinEventData.java new file mode 100644 index 0000000000000..e46d52dcd019e --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryNodeJoinEventData.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.util.List; + +/** + * + */ +class ZkDiscoveryNodeJoinEventData extends ZkDiscoveryEventData { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + final List joinedNodes; + + /** */ + final int dataForJoinedPartCnt; + + /** + * @param evtId Event ID. + * @param topVer Topology version. + * @param joinedNodes Joined nodes data. + * @param dataForJoinedPartCnt Data for joined part count. + */ + ZkDiscoveryNodeJoinEventData( + long evtId, + long topVer, + List joinedNodes, + int dataForJoinedPartCnt) + { + super(evtId, ZK_EVT_NODE_JOIN, topVer); + + this.joinedNodes = joinedNodes; + this.dataForJoinedPartCnt = dataForJoinedPartCnt; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "ZkDiscoveryNodeJoinEventData [" + + "evtId=" + eventId() + + ", topVer=" + topologyVersion() + + ", nodes=" + joinedNodes + ']'; + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDistributedCollectDataFuture.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDistributedCollectDataFuture.java new file mode 100644 index 0000000000000..174d698fe27e4 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDistributedCollectDataFuture.java @@ -0,0 +1,250 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.Callable; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.zookeeper.AsyncCallback; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.data.Stat; + +/** + * + */ +class ZkDistributedCollectDataFuture extends GridFutureAdapter { + /** */ + private final IgniteLogger log; + + /** */ + private final String futPath; + + /** */ + private final Set remainingNodes; + + /** */ + private final Callable lsnr; + + /** + * @param impl Disovery impl + * @param rtState Runtime state. + * @param futPath Future path. + * @param lsnr Future listener. + * @throws Exception If listener call failed. + */ + ZkDistributedCollectDataFuture( + ZookeeperDiscoveryImpl impl, + ZkRuntimeState rtState, + String futPath, + Callable lsnr) + throws Exception + { + this.log = impl.log(); + this.futPath = futPath; + this.lsnr = lsnr; + + ZkClusterNodes top = rtState.top; + + // Assume new nodes can not join while future is in progress. + + remainingNodes = U.newHashSet(top.nodesByOrder.size()); + + for (ZookeeperClusterNode node : top.nodesByInternalId.values()) + remainingNodes.add(node.order()); + + NodeResultsWatcher watcher = new NodeResultsWatcher(rtState, impl); + + if (remainingNodes.isEmpty()) + completeAndNotifyListener(); + else { + if (log.isInfoEnabled()) { + log.info("Initialize data collect future [futPath=" + futPath + ", " + + "remainingNodes=" + remainingNodes.size() + ']'); + } + + rtState.zkClient.getChildrenAsync(futPath, watcher, watcher); + } + } + + /** + * @throws Exception If listener call failed. + */ + private void completeAndNotifyListener() throws Exception { + if (super.onDone()) + lsnr.call(); + } + + /** + * @param futPath + * @param client + * @param nodeOrder + * @param data + * @throws Exception If failed. + */ + static void saveNodeResult(String futPath, ZookeeperClient client, long nodeOrder, byte[] data) throws Exception { + client.createIfNeeded(futPath + "/" + nodeOrder, data, CreateMode.PERSISTENT); + } + + /** + * @param futPath + * @param client + * @param nodeOrder + * @return Node result data. + * @throws Exception If fai.ed + */ + static byte[] readNodeResult(String futPath, ZookeeperClient client, long nodeOrder) throws Exception { + return client.getData(futPath + "/" + nodeOrder); + } + + /** + * @param futResPath Result path. + * @param client Client. + * @param data Result data. + * @throws Exception If failed. + */ + static void saveResult(String futResPath, ZookeeperClient client, byte[] data) throws Exception { + client.createIfNeeded(futResPath, data, CreateMode.PERSISTENT); + } + + static byte[] readResult(ZookeeperClient client, ZkIgnitePaths paths, UUID futId) throws Exception { + return client.getData(paths.distributedFutureResultPath(futId)); + } + + /** + * @param client Client. + * @param paths Paths utils. + * @param futId Future ID. + * @param log Ignite Logger. + * @throws Exception If failed. + */ + static void deleteFutureData(ZookeeperClient client, + ZkIgnitePaths paths, + UUID futId, + IgniteLogger log + ) throws Exception { + // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8189 + String evtDir = paths.distributedFutureBasePath(futId); + + try { + client.deleteAll(evtDir, + client.getChildren(evtDir), + -1); + } + catch (KeeperException.NoNodeException e) { + U.log(log, "Node for deletion was not found: " + e.getPath()); + + // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8189 + } + + client.deleteIfExists(evtDir, -1); + + client.deleteIfExists(paths.distributedFutureResultPath(futId), -1); + } + + /** + * @param top Current topology. + * @throws Exception If listener call failed. + */ + void onTopologyChange(ZkClusterNodes top) throws Exception { + if (remainingNodes.isEmpty()) + return; + + for (Iterator it = remainingNodes.iterator(); it.hasNext();) { + Long nodeOrder = it.next(); + + if (!top.nodesByOrder.containsKey(nodeOrder)) { + it.remove(); + + int remaining = remainingNodes.size(); + + if (log.isInfoEnabled()) { + log.info("ZkDistributedCollectDataFuture removed remaining failed node [node=" + nodeOrder + + ", remaining=" + remaining + + ", futPath=" + futPath + ']'); + } + + if (remaining == 0) { + completeAndNotifyListener(); + + break; + } + } + } + } + + /** + * + */ + class NodeResultsWatcher extends ZkAbstractWatcher implements AsyncCallback.Children2Callback { + /** + * @param rtState Runtime state. + * @param impl Discovery impl. + */ + NodeResultsWatcher(ZkRuntimeState rtState, ZookeeperDiscoveryImpl impl) { + super(rtState, impl); + } + + /** {@inheritDoc} */ + @Override protected void process0(WatchedEvent evt) { + if (evt.getType() == Watcher.Event.EventType.NodeChildrenChanged) + rtState.zkClient.getChildrenAsync(evt.getPath(), this, this); + } + + /** {@inheritDoc} */ + @Override public void processResult(int rc, String path, Object ctx, List children, Stat stat) { + if (!onProcessStart()) + return; + + try { + if (!isDone()) { + assert rc == 0 : KeeperException.Code.get(rc); + + for (int i = 0; i < children.size(); i++) { + Long nodeOrder = Long.parseLong(children.get(i)); + + if (remainingNodes.remove(nodeOrder)) { + int remaining = remainingNodes.size(); + + if (log.isInfoEnabled()) { + log.info("ZkDistributedCollectDataFuture added new result [node=" + nodeOrder + + ", remaining=" + remaining + + ", futPath=" + path + ']'); + } + + if (remaining == 0) + completeAndNotifyListener(); + } + } + } + + onProcessEnd(); + } + catch (Throwable e) { + onProcessError(e); + } + } + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkForceNodeFailMessage.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkForceNodeFailMessage.java new file mode 100644 index 0000000000000..de7291c0d453c --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkForceNodeFailMessage.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; +import org.jetbrains.annotations.Nullable; + +/** + * Zk Force Node Fail Message. + */ +public class ZkForceNodeFailMessage implements DiscoverySpiCustomMessage, ZkInternalMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + final long nodeInternalId; + + /** */ + final String warning; + + /** + * @param nodeInternalId Node ID. + * @param warning Warning to be displayed on all nodes. + */ + ZkForceNodeFailMessage(long nodeInternalId, String warning) { + this.nodeInternalId = nodeInternalId; + this.warning = warning; + } + + /** {@inheritDoc} */ + @Nullable @Override public DiscoverySpiCustomMessage ackMessage() { + return null; + } + + /** {@inheritDoc} */ + @Override public boolean isMutable() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ZkForceNodeFailMessage.class, this); + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkIgnitePaths.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkIgnitePaths.java new file mode 100644 index 0000000000000..9caf00fb64a93 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkIgnitePaths.java @@ -0,0 +1,307 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.util.UUID; + +/** + * + */ +class ZkIgnitePaths { + /** */ + static final String PATH_SEPARATOR = "/"; + + /** */ + private static final byte CLIENT_NODE_FLAG_MASK = 0x01; + + /** */ + private static final int UUID_LEN = 36; + + /** Directory to store joined node data. */ + private static final String JOIN_DATA_DIR = "jd"; + + /** Directory to store new custom events. */ + private static final String CUSTOM_EVTS_DIR = "ce"; + + /** Directory to store parts of multi-parts custom events. */ + private static final String CUSTOM_EVTS_PARTS_DIR = "cp"; + + /** Directory to store acknowledge messages for custom events. */ + private static final String CUSTOM_EVTS_ACKS_DIR = "ca"; + + /** Directory to store EPHEMERAL znodes for alive cluster nodes. */ + static final String ALIVE_NODES_DIR = "n"; + + /** Path to store discovery events {@link ZkDiscoveryEventsData}. */ + private static final String DISCO_EVENTS_PATH = "e"; + + /** */ + final String clusterDir; + + /** */ + final String aliveNodesDir; + + /** */ + final String joinDataDir; + + /** */ + final String evtsPath; + + /** */ + final String customEvtsDir; + + /** */ + final String customEvtsPartsDir; + + /** */ + final String customEvtsAcksDir; + + /** + * @param zkRootPath Base Zookeeper directory for all Ignite nodes. + */ + ZkIgnitePaths(String zkRootPath) { + clusterDir = zkRootPath; + + aliveNodesDir = zkPath(ALIVE_NODES_DIR); + joinDataDir = zkPath(JOIN_DATA_DIR); + evtsPath = zkPath(DISCO_EVENTS_PATH); + customEvtsDir = zkPath(CUSTOM_EVTS_DIR); + customEvtsPartsDir = zkPath(CUSTOM_EVTS_PARTS_DIR); + customEvtsAcksDir = zkPath(CUSTOM_EVTS_ACKS_DIR); + } + + /** + * @param path Relative path. + * @return Full path. + */ + private String zkPath(String path) { + return clusterDir + "/" + path; + } + + /** + * @param nodeId Node ID. + * @param prefixId Unique prefix ID. + * @return Path. + */ + String joiningNodeDataPath(UUID nodeId, UUID prefixId) { + return joinDataDir + '/' + prefixId + ":" + nodeId.toString(); + } + + /** + * @param path Alive node zk path. + * @return Node internal ID. + */ + static long aliveInternalId(String path) { + int idx = path.lastIndexOf('|'); + + return Integer.parseInt(path.substring(idx + 1)); + } + + /** + * @param prefix Node unique path prefix. + * @param node Node. + * @return Path. + */ + String aliveNodePathForCreate(String prefix, ZookeeperClusterNode node) { + byte flags = 0; + + if (node.isClient()) + flags |= CLIENT_NODE_FLAG_MASK; + + return aliveNodesDir + "/" + prefix + ":" + node.id() + ":" + encodeFlags(flags) + "|"; + } + + /** + * @param path Alive node zk path. + * @return {@code True} if node is client. + */ + static boolean aliveNodeClientFlag(String path) { + return (aliveFlags(path) & CLIENT_NODE_FLAG_MASK) != 0; + } + + /** + * @param path Alive node zk path. + * @return Node ID. + */ + static UUID aliveNodePrefixId(String path) { + return UUID.fromString(path.substring(0, ZkIgnitePaths.UUID_LEN)); + } + + /** + * @param path Alive node zk path. + * @return Node ID. + */ + static UUID aliveNodeId(String path) { + // ::| + int startIdx = ZkIgnitePaths.UUID_LEN + 1; + + String idStr = path.substring(startIdx, startIdx + ZkIgnitePaths.UUID_LEN); + + return UUID.fromString(idStr); + } + + /** + * @param path Event zk path. + * @return Event sequence number. + */ + static int customEventSequence(String path) { + int idx = path.lastIndexOf('|'); + + return Integer.parseInt(path.substring(idx + 1)); + } + + /** + * @param path Custom event zl path. + * @return Event node ID. + */ + static UUID customEventSendNodeId(String path) { + // ::| + int startIdx = ZkIgnitePaths.UUID_LEN + 1; + + String idStr = path.substring(startIdx, startIdx + ZkIgnitePaths.UUID_LEN); + + return UUID.fromString(idStr); + } + + /** + * @param path Event path. + * @return Event unique prefix. + */ + static String customEventPrefix(String path) { + // ::| + + return path.substring(0, ZkIgnitePaths.UUID_LEN); + } + + /** + * @param path Custom event zl path. + * @return Event node ID. + */ + static int customEventPartsCount(String path) { + // ::| + int startIdx = 2 * ZkIgnitePaths.UUID_LEN + 2; + + String cntStr = path.substring(startIdx, startIdx + 4); + + int partCnt = Integer.parseInt(cntStr); + + assert partCnt >= 1 : partCnt; + + return partCnt; + } + + /** + * @param prefix Prefix. + * @param nodeId Node ID. + * @param partCnt Parts count. + * @return Path. + */ + String createCustomEventPath(String prefix, UUID nodeId, int partCnt) { + return customEvtsDir + "/" + prefix + ":" + nodeId + ":" + String.format("%04d", partCnt) + '|'; + } + + /** + * @param prefix Prefix. + * @param nodeId Node ID. + * @return Path. + */ + String customEventPartsBasePath(String prefix, UUID nodeId) { + return customEvtsPartsDir + "/" + prefix + ":" + nodeId + ":"; + } + + /** + * @param prefix Prefix. + * @param nodeId Node ID. + * @param part Part number. + * @return Path. + */ + String customEventPartPath(String prefix, UUID nodeId, int part) { + return customEventPartsBasePath(prefix, nodeId) + String.format("%04d", part); + } + + /** + * @param evtId Event ID. + * @return Event zk path. + */ + String joinEventDataPathForJoined(long evtId) { + return evtsPath + "/fj-" + evtId; + } + + /** + * @param topVer Event topology version. + * @return Event zk path. + */ + String joinEventSecuritySubjectPath(long topVer) { + return evtsPath + "/s-" + topVer; + } + + /** + * @param origEvtId ID of original custom event. + * @return Path for custom event ack. + */ + String ackEventDataPath(long origEvtId) { + assert origEvtId != 0; + + return customEvtsAcksDir + "/" + String.valueOf(origEvtId); + } + + /** + * @param id Future ID. + * @return Future path. + */ + String distributedFutureBasePath(UUID id) { + return evtsPath + "/f-" + id; + } + + /** + * @param id Future ID. + * @return Future path. + */ + String distributedFutureResultPath(UUID id) { + return evtsPath + "/fr-" + id; + } + + /** + * @param flags Flags. + * @return Flags string. + */ + private static String encodeFlags(byte flags) { + int intVal = flags + 128; + + String str = Integer.toString(intVal, 16); + + if (str.length() == 1) + str = '0' + str; + + assert str.length() == 2 : str; + + return str; + } + + /** + * @param path Alive node zk path. + * @return Flags. + */ + private static byte aliveFlags(String path) { + int startIdx = path.lastIndexOf(':') + 1; + + String flagsStr = path.substring(startIdx, startIdx + 2); + + return (byte)(Integer.parseInt(flagsStr, 16) - 128); + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalJoinErrorMessage.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalJoinErrorMessage.java new file mode 100644 index 0000000000000..a73312cfecf05 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalJoinErrorMessage.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +/** + * + */ +class ZkInternalJoinErrorMessage implements ZkInternalMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + transient boolean notifyNode = true; + + /** */ + final long nodeInternalId; + + /** */ + final String err; + + /** + * @param nodeInternalId Joining node internal ID. + * @param err Error message. + */ + ZkInternalJoinErrorMessage(long nodeInternalId, String err) { + this.nodeInternalId = nodeInternalId; + this.err = err; + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalMessage.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalMessage.java new file mode 100644 index 0000000000000..c1d56f0eedabd --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkInternalMessage.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.io.Serializable; + +/** + * + */ +interface ZkInternalMessage extends Serializable { + // No-op. +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinEventDataForJoined.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinEventDataForJoined.java new file mode 100644 index 0000000000000..e4ae4ba0aeb95 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinEventDataForJoined.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +class ZkJoinEventDataForJoined implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private final List top; + + /** */ + private final Map discoData; + + /** */ + private final Map dupDiscoData; + + /** + * @param top Topology. + * @param discoData Discovery data. + */ + ZkJoinEventDataForJoined(List top, Map discoData, @Nullable Map dupDiscoData) { + assert top != null; + assert discoData != null && !discoData.isEmpty(); + + this.top = top; + this.discoData = discoData; + this.dupDiscoData = dupDiscoData; + } + + byte[] discoveryDataForNode(long nodeOrder) { + assert discoData != null; + + byte[] dataBytes = discoData.get(nodeOrder); + + if (dataBytes != null) + return dataBytes; + + assert dupDiscoData != null; + + Long dupDataNode = dupDiscoData.get(nodeOrder); + + assert dupDataNode != null; + + dataBytes = discoData.get(dupDataNode); + + assert dataBytes != null; + + return dataBytes; + } + + /** + * @return Current topology. + */ + List topology() { + assert top != null; + + return top; + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinedNodeEvtData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinedNodeEvtData.java new file mode 100644 index 0000000000000..3c367cf754762 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoinedNodeEvtData.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.io.Serializable; +import java.util.UUID; + +/** + * Zk Joined Node Evt Data. + */ +public class ZkJoinedNodeEvtData implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + final long topVer; + + /** */ + final long joinedInternalId; + + /** */ + final UUID nodeId; + + /** */ + final int joinDataPartCnt; + + /** */ + final int secSubjPartCnt; + + /** */ + final UUID joinDataPrefixId; + + /** */ + transient ZkJoiningNodeData joiningNodeData; + + /** + * @param topVer Topology version for node join event. + * @param nodeId Joined node ID. + * @param joinedInternalId Joined node internal ID. + * @param joinDataPrefixId Join data unique prefix. + * @param joinDataPartCnt Join data part count. + * @param secSubjPartCnt Security subject part count. + */ + ZkJoinedNodeEvtData( + long topVer, + UUID nodeId, + long joinedInternalId, + UUID joinDataPrefixId, + int joinDataPartCnt, + int secSubjPartCnt) + { + this.topVer = topVer; + this.nodeId = nodeId; + this.joinedInternalId = joinedInternalId; + this.joinDataPrefixId = joinDataPrefixId; + this.joinDataPartCnt = joinDataPartCnt; + this.secSubjPartCnt = secSubjPartCnt; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "ZkJoinedNodeData [id=" + nodeId + ", order=" + topVer + ']'; + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoiningNodeData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoiningNodeData.java new file mode 100644 index 0000000000000..ff8311d071ba8 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkJoiningNodeData.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.io.Serializable; +import java.util.Map; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * + */ +class ZkJoiningNodeData implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private int partCnt; + + /** */ + @GridToStringInclude + private ZookeeperClusterNode node; + + /** */ + @GridToStringInclude + private Map discoData; + + /** + * @param partCnt Number of parts in multi-parts message. + */ + ZkJoiningNodeData(int partCnt) { + this.partCnt = partCnt; + } + + /** + * @param node Node. + * @param discoData Discovery data. + */ + ZkJoiningNodeData(ZookeeperClusterNode node, Map discoData) { + assert node != null && node.id() != null : node; + assert discoData != null; + + this.node = node; + this.discoData = discoData; + } + + /** + * @return Number of parts in multi-parts message. + */ + int partCount() { + return partCnt; + } + + /** + * @return Node. + */ + ZookeeperClusterNode node() { + return node; + } + + /** + * @return Discovery data. + */ + Map discoveryData() { + return discoData; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ZkJoiningNodeData.class, this); + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNoServersMessage.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNoServersMessage.java new file mode 100644 index 0000000000000..626fe742d1c1b --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNoServersMessage.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +class ZkNoServersMessage implements DiscoverySpiCustomMessage, ZkInternalMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Nullable @Override public DiscoverySpiCustomMessage ackMessage() { + return null; + } + + /** {@inheritDoc} */ + @Override public boolean isMutable() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return false; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ZkNoServersMessage.class, this); + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNodeValidateResult.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNodeValidateResult.java new file mode 100644 index 0000000000000..2abfee3d61600 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkNodeValidateResult.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +/** + * + */ +class ZkNodeValidateResult { + /** */ + String err; + + /** */ + byte[] secSubjZipBytes; + + /** + * @param err Error. + */ + ZkNodeValidateResult(String err) { + this.err = err; + } + + /** + * @param secSubjZipBytes Marshalled security subject. + */ + ZkNodeValidateResult(byte[] secSubjZipBytes) { + this.secSubjZipBytes = secSubjZipBytes; + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRunnable.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRunnable.java new file mode 100644 index 0000000000000..965bdc0f45851 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRunnable.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +/** + * Zk Runnable. + */ +public abstract class ZkRunnable extends ZkAbstractCallabck implements Runnable { + /** + * @param rtState Runtime state. + * @param impl Discovery impl. + */ + ZkRunnable(ZkRuntimeState rtState, ZookeeperDiscoveryImpl impl) { + super(rtState, impl); + } + + /** {@inheritDoc} */ + @Override public void run() { + if (!onProcessStart()) + return; + + try { + run0(); + + onProcessEnd(); + } + catch (Throwable e) { + onProcessError(e); + } + } + + /** + * + */ + protected abstract void run0() throws Exception; +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRuntimeState.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRuntimeState.java new file mode 100644 index 0000000000000..cb04ac3de01e8 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkRuntimeState.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.util.List; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.spi.IgniteSpiTimeoutObject; +import org.apache.zookeeper.AsyncCallback; +import org.apache.zookeeper.Watcher; + +/** + * + */ +class ZkRuntimeState { + /** */ + ZkWatcher watcher; + + /** */ + ZkAliveNodeDataWatcher aliveNodeDataWatcher; + + /** */ + volatile Exception errForClose; + + /** */ + final boolean prevJoined; + + /** */ + ZookeeperClient zkClient; + + /** */ + long internalOrder; + + /** */ + int joinDataPartCnt; + + /** */ + long gridStartTime; + + /** */ + volatile boolean joined; + + /** */ + ZkDiscoveryEventsData evtsData; + + /** */ + boolean crd; + + /** */ + String locNodeZkPath; + + /** */ + final ZkAliveNodeData locNodeInfo = new ZkAliveNodeData(); + + /** */ + int procEvtCnt; + + /** */ + final ZkClusterNodes top = new ZkClusterNodes(); + + /** */ + List commErrProcNodes; + + /** Timeout callback registering watcher for join error + * (set this watcher after timeout as a minor optimization). + */ + ZkTimeoutObject joinErrTo; + + /** Timeout callback set to wait for join timeout. */ + ZkTimeoutObject joinTo; + + /** Timeout callback to update processed events counter. */ + ZkTimeoutObject procEvtsUpdateTo; + + /** */ + boolean updateAlives; + + /** + * @param prevJoined {@code True} if joined topology before reconnect attempt. + */ + ZkRuntimeState(boolean prevJoined) { + this.prevJoined = prevJoined; + } + + /** + * @param watcher Watcher. + * @param aliveNodeDataWatcher Alive nodes data watcher. + */ + void init(ZkWatcher watcher, ZkAliveNodeDataWatcher aliveNodeDataWatcher) { + this.watcher = watcher; + this.aliveNodeDataWatcher = aliveNodeDataWatcher; + } + + /** + * @param err Error. + */ + void onCloseStart(Exception err) { + assert err != null; + + errForClose = err; + + ZookeeperClient zkClient = this.zkClient; + + if (zkClient != null) + zkClient.onCloseStart(); + } + + /** + * + */ + interface ZkWatcher extends Watcher, AsyncCallback.Children2Callback, AsyncCallback.DataCallback { + // No-op. + } + + /** + * + */ + interface ZkAliveNodeDataWatcher extends Watcher, AsyncCallback.DataCallback { + // No-op. + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkTimeoutObject.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkTimeoutObject.java new file mode 100644 index 0000000000000..4d3d5b4885cc1 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkTimeoutObject.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.spi.IgniteSpiTimeoutObject; + +/** + * + */ +abstract class ZkTimeoutObject implements IgniteSpiTimeoutObject { + /** */ + private final IgniteUuid id = IgniteUuid.randomUuid(); + + /** */ + private final long endTime; + + /** */ + volatile boolean cancelled; + + /** + * @param timeout Timeout. + */ + ZkTimeoutObject(long timeout) { + long endTime = timeout >= 0 ? System.currentTimeMillis() + timeout : Long.MAX_VALUE; + + this.endTime = endTime >= 0 ? endTime : Long.MAX_VALUE; + } + + /** {@inheritDoc} */ + @Override public final IgniteUuid id() { + return id; + } + + /** {@inheritDoc} */ + @Override public final long endTime() { + return endTime; + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClient.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClient.java new file mode 100644 index 0000000000000..21703c66a5518 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClient.java @@ -0,0 +1,1219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.List; +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteRunnable; +import org.apache.zookeeper.AsyncCallback; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.Op; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.data.Stat; +import org.jetbrains.annotations.Nullable; + +/** + * Zookeeper Client. + */ +public class ZookeeperClient implements Watcher { + /** */ + private static final long RETRY_TIMEOUT = + IgniteSystemProperties.getLong("IGNITE_ZOOKEEPER_DISCOVERY_RETRY_TIMEOUT", 2000); + + /** */ + private static final int MAX_RETRY_COUNT = + IgniteSystemProperties.getInteger("IGNITE_ZOOKEEPER_DISCOVERY_MAX_RETRY_COUNT", 10); + + /** */ + private final AtomicInteger retryCount = new AtomicInteger(); + + /** */ + private static final int MAX_REQ_SIZE = 1048528; + + /** */ + private static final List ZK_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE; + + /** */ + private static final byte[] EMPTY_BYTES = {}; + + /** */ + private final ZooKeeper zk; + + /** */ + private final IgniteLogger log; + + /** */ + private ConnectionState state = ConnectionState.Disconnected; + + /** */ + private long connLossTimeout; + + /** */ + private volatile long connStartTime; + + /** */ + private final Object stateMux = new Object(); + + /** */ + private final IgniteRunnable connLostC; + + /** */ + private final Timer connTimer; + + /** */ + private final ArrayDeque retryQ = new ArrayDeque<>(); + + /** */ + private volatile boolean closing; + + /** + * @param log Logger. + * @param connectString ZK connection string. + * @param sesTimeout ZK session timeout. + * @param connLostC Lost connection callback. + * @throws Exception If failed. + */ + ZookeeperClient(IgniteLogger log, String connectString, int sesTimeout, IgniteRunnable connLostC) throws Exception { + this(null, log, connectString, sesTimeout, connLostC); + } + + /** + * @param igniteInstanceName Ignite instance name. + * @param log Logger. + * @param connectString ZK connection string. + * @param sesTimeout ZK session timeout. + * @param connLostC Lost connection callback. + * @throws Exception If failed. + */ + ZookeeperClient(String igniteInstanceName, + IgniteLogger log, + String connectString, + int sesTimeout, + IgniteRunnable connLostC) + throws Exception + { + this.log = log.getLogger(getClass()); + this.connLostC = connLostC; + + connLossTimeout = sesTimeout; + + long connStartTime = this.connStartTime = System.currentTimeMillis(); + + connTimer = new Timer("zk-client-timer-" + igniteInstanceName); + + String threadName = Thread.currentThread().getName(); + + // ZK generates internal threads' names using current thread name. + Thread.currentThread().setName("zk-" + igniteInstanceName); + + try { + zk = new ZooKeeper(connectString, sesTimeout, this); + } + finally { + Thread.currentThread().setName(threadName); + } + + synchronized (stateMux) { + if (connStartTime == this.connStartTime && state == ConnectionState.Disconnected) + scheduleConnectionCheck(); + } + } + + /** + * @return Zookeeper client. + */ + ZooKeeper zk() { + return zk; + } + + /** + * @return {@code True} if connected to ZooKeeper. + */ + boolean connected() { + synchronized (stateMux) { + return state == ConnectionState.Connected; + } + } + + /** {@inheritDoc} */ + @Override public void process(WatchedEvent evt) { + if (closing) + return; + + if (evt.getType() == Event.EventType.None) { + ConnectionState newState; + + synchronized (stateMux) { + if (state == ConnectionState.Lost) { + U.warn(log, "Received event after connection was lost [evtState=" + evt.getState() + "]"); + + return; + } + + if (!zk.getState().isAlive()) + return; + + Event.KeeperState zkState = evt.getState(); + + switch (zkState) { + case SaslAuthenticated: + return; // No-op. + + case AuthFailed: + newState = state; + + break; + + case Disconnected: + newState = ConnectionState.Disconnected; + + break; + + case SyncConnected: + newState = ConnectionState.Connected; + + break; + + case Expired: + U.warn(log, "Session expired, changing state to Lost"); + + newState = ConnectionState.Lost; + + break; + + default: + U.error(log, "Unexpected state for ZooKeeper client, close connection: " + zkState); + + newState = ConnectionState.Lost; + } + + if (newState != state) { + if (log.isInfoEnabled()) + log.info("ZooKeeper client state changed [prevState=" + state + ", newState=" + newState + ']'); + + state = newState; + + if (newState == ConnectionState.Disconnected) { + connStartTime = System.currentTimeMillis(); + + scheduleConnectionCheck(); + } + else if (newState == ConnectionState.Connected) { + retryCount.set(0); + + stateMux.notifyAll(); + } + else + assert state == ConnectionState.Lost : state; + } + else + return; + } + + if (newState == ConnectionState.Lost) { + closeClient(); + + notifyConnectionLost(); + } + else if (newState == ConnectionState.Connected) { + for (ZkAsyncOperation op : retryQ) + op.execute(); + } + } + } + + /** + * + */ + private void notifyConnectionLost() { + if (!closing && state == ConnectionState.Lost && connLostC != null) + connLostC.run(); + + connTimer.cancel(); + } + + /** + * @param path Path. + * @return {@code True} if node exists. + * @throws ZookeeperClientFailedException If connection to zk was lost. + * @throws InterruptedException If interrupted. + */ + boolean exists(String path) throws ZookeeperClientFailedException, InterruptedException { + for (;;) { + long connStartTime = this.connStartTime; + + try { + return zk.exists(path, false) != null; + } + catch (Exception e) { + onZookeeperError(connStartTime, e); + } + } + } + + /** + * + * @param paths Paths to create. + * @param createMode Create mode. + * @throws KeeperException.NodeExistsException If at least one of target node already exists. + * @throws ZookeeperClientFailedException If connection to zk was lost. + * @throws InterruptedException If interrupted. + */ + void createAll(List paths, CreateMode createMode) + throws ZookeeperClientFailedException, InterruptedException, KeeperException.NodeExistsException + { + // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8188 + List ops = new ArrayList<>(paths.size()); + + for (String path : paths) + ops.add(Op.create(path, EMPTY_BYTES, ZK_ACL, createMode)); + + for (;;) { + long connStartTime = this.connStartTime; + + try { + zk.multi(ops); + + return; + } + catch (KeeperException.NodeExistsException e) { + throw e; + } + catch (Exception e) { + onZookeeperError(connStartTime, e); + } + } + } + + /** + * @param path Path. + * @param data Data. + * @param overhead Extra overhead. + * @return {@code True} If data size exceeds max request size and should be splitted into multiple parts. + */ + boolean needSplitNodeData(String path, byte[] data, int overhead) { + return requestOverhead(path) + data.length + overhead > MAX_REQ_SIZE; + } + + /** + * @param path Path. + * @param data Data. + * @param overhead Extra overhead. + * @return Splitted data. + */ + List splitNodeData(String path, byte[] data, int overhead) { + int partSize = MAX_REQ_SIZE - requestOverhead(path) - overhead; + + int partCnt = data.length / partSize; + + if (data.length % partSize != 0) + partCnt++; + + assert partCnt > 1 : "Do not need split"; + + List parts = new ArrayList<>(partCnt); + + int remaining = data.length; + + for (int i = 0; i < partCnt; i++) { + int partSize0 = Math.min(remaining, partSize); + + byte[] part = new byte[partSize0]; + + System.arraycopy(data, i * partSize, part, 0, part.length); + + remaining -= partSize0; + + parts.add(part); + } + + assert remaining == 0 : remaining; + + return parts; + } + + /** + * TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8187 + * @param path Request path. + * @return Marshalled request overhead. + */ + private int requestOverhead(String path) { + return path.length(); + } + + /** + * @param path Path. + * @param data Data. + * @param createMode Create mode. + * @return Created path. + * @throws ZookeeperClientFailedException If connection to zk was lost. + * @throws InterruptedException If interrupted. + */ + String createIfNeeded(String path, byte[] data, CreateMode createMode) + throws ZookeeperClientFailedException, InterruptedException + { + assert !createMode.isSequential() : createMode; + + if (data == null) + data = EMPTY_BYTES; + + for (;;) { + long connStartTime = this.connStartTime; + + try { + return zk.create(path, data, ZK_ACL, createMode); + } + catch (KeeperException.NodeExistsException e) { + if (log.isDebugEnabled()) + log.debug("Node already exists: " + path); + + return path; + } + catch (Exception e) { + onZookeeperError(connStartTime, e); + } + } + } + + /** + * @param checkPrefix Unique prefix to check in case of retry. + * @param parentPath Parent node path. + * @param path Node to create. + * @param data Node data. + * @param createMode Create mode. + * @return Create path. + * @throws ZookeeperClientFailedException If connection to zk was lost. + * @throws InterruptedException If interrupted. + */ + String createSequential(String checkPrefix, String parentPath, String path, byte[] data, CreateMode createMode) + throws ZookeeperClientFailedException, InterruptedException + { + assert createMode.isSequential() : createMode; + + if (data == null) + data = EMPTY_BYTES; + + boolean first = true; + + for (;;) { + long connStartTime = this.connStartTime; + + try { + if (!first) { + List children = zk.getChildren(parentPath, false); + + for (int i = 0; i < children.size(); i++) { + String child = children.get(i); + + if (children.get(i).startsWith(checkPrefix)) { + String resPath = parentPath + "/" + child; + + if (log.isDebugEnabled()) + log.debug("Check before retry, node already created: " + resPath); + + return resPath; + } + } + } + + return zk.create(path, data, ZK_ACL, createMode); + } + catch (KeeperException.NodeExistsException e) { + assert !createMode.isSequential() : createMode; + + if (log.isDebugEnabled()) + log.debug("Node already exists: " + path); + + return path; + } + catch (Exception e) { + onZookeeperError(connStartTime, e); + } + + first = false; + } + } + + /** + * @param path Path. + * @return Children nodes. + * @throws ZookeeperClientFailedException If connection to zk was lost. + * @throws InterruptedException If interrupted. + */ + List getChildren(String path) + throws ZookeeperClientFailedException, InterruptedException + { + for (;;) { + long connStartTime = this.connStartTime; + + try { + return zk.getChildren(path, false); + } + catch (Exception e) { + onZookeeperError(connStartTime, e); + } + } + } + + /** + * @param path Path. + * @throws InterruptedException If interrupted. + * @throws KeeperException In case of error. + * @return {@code True} if given path exists. + */ + boolean existsNoRetry(String path) throws InterruptedException, KeeperException { + return zk.exists(path, false) != null; + } + + /** + * @param path Path. + * @param ver Expected version. + * @throws InterruptedException If interrupted. + * @throws KeeperException In case of error. + */ + void deleteIfExistsNoRetry(String path, int ver) throws InterruptedException, KeeperException { + try { + zk.delete(path, ver); + } + catch (KeeperException.NoNodeException e) { + // No-op if znode does not exist. + } + } + + /** + * @param path Path. + * @param ver Version. + * @throws ZookeeperClientFailedException If connection to zk was lost. + * @throws InterruptedException If interrupted. + */ + void deleteIfExists(String path, int ver) + throws ZookeeperClientFailedException, InterruptedException + { + try { + delete(path, ver); + } + catch (KeeperException.NoNodeException e) { + // No-op if znode does not exist. + } + } + + /** + * @param parent Parent path. + * @param paths Children paths. + * @param ver Version. + * @throws KeeperException.NoNodeException If at least one of nodes does not exist. + * @throws ZookeeperClientFailedException If connection to zk was lost. + * @throws InterruptedException If interrupted. + */ + void deleteAll(@Nullable String parent, List paths, int ver) + throws KeeperException.NoNodeException, ZookeeperClientFailedException, InterruptedException + { + if (paths.isEmpty()) + return; + + // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8188 + List ops = new ArrayList<>(paths.size()); + + for (String path : paths) { + String path0 = parent != null ? parent + "/" + path : path; + + ops.add(Op.delete(path0, ver)); + } + + for (;;) { + long connStartTime = this.connStartTime; + + try { + zk.multi(ops); + + return; + } + catch (KeeperException.NoNodeException e) { + throw e; + } + catch (Exception e) { + onZookeeperError(connStartTime, e); + } + } + } + + /** + * @param path Path. + * @param ver Version. + * @throws KeeperException.NoNodeException If target node does not exist. + * @throws ZookeeperClientFailedException If connection to zk was lost. + * @throws InterruptedException If interrupted. + */ + private void delete(String path, int ver) + throws KeeperException.NoNodeException, ZookeeperClientFailedException, InterruptedException + { + for (;;) { + long connStartTime = this.connStartTime; + + try { + zk.delete(path, ver); + + return; + } + catch (KeeperException.NoNodeException e) { + throw e; + } + catch (Exception e) { + onZookeeperError(connStartTime, e); + } + } + } + + /** + * @param path Path. + * @param data Data. + * @param ver Version. + * @throws ZookeeperClientFailedException If connection to zk was lost. + * @throws InterruptedException If interrupted. + * @throws KeeperException.NoNodeException If node does not exist. + * @throws KeeperException.BadVersionException If version does not match. + */ + void setData(String path, byte[] data, int ver) + throws ZookeeperClientFailedException, InterruptedException, KeeperException.NoNodeException, + KeeperException.BadVersionException + { + if (data == null) + data = EMPTY_BYTES; + + for (;;) { + long connStartTime = this.connStartTime; + + try { + zk.setData(path, data, ver); + + return; + } + catch (KeeperException.BadVersionException | KeeperException.NoNodeException e) { + throw e; + } + catch (Exception e) { + onZookeeperError(connStartTime, e); + } + } + } + + /** + * @param path Path. + * @param stat Optional {@link Stat} instance to return znode state. + * @return Data. + * @throws KeeperException.NoNodeException If target node does not exist. + * @throws ZookeeperClientFailedException If connection to zk was lost. + * @throws InterruptedException If interrupted. + */ + byte[] getData(String path, @Nullable Stat stat) + throws KeeperException.NoNodeException, ZookeeperClientFailedException, InterruptedException { + for (;;) { + long connStartTime = this.connStartTime; + + try { + return zk.getData(path, false, stat); + } + catch (KeeperException.NoNodeException e) { + throw e; + } + catch (Exception e) { + onZookeeperError(connStartTime, e); + } + } + } + + /** + * @param path Path. + * @return Data. + * @throws KeeperException.NoNodeException If target node does not exist. + * @throws ZookeeperClientFailedException If connection to zk was lost. + * @throws InterruptedException If interrupted. + */ + byte[] getData(String path) + throws KeeperException.NoNodeException, ZookeeperClientFailedException, InterruptedException + { + return getData(path, null); + } + + /** + * @param path Path. + */ + void deleteIfExistsAsync(String path) { + new DeleteIfExistsOperation(path).execute(); + } + + /** + * @param path Path. + * @param watcher Watcher. + * @param cb Callback. + */ + void existsAsync(String path, Watcher watcher, AsyncCallback.StatCallback cb) { + ExistsOperation op = new ExistsOperation(path, watcher, cb); + + zk.exists(path, watcher, new StatCallbackWrapper(op), null); + } + + /** + * @param path Path. + * @param watcher Watcher. + * @param cb Callback. + */ + void getChildrenAsync(String path, Watcher watcher, AsyncCallback.Children2Callback cb) { + GetChildrenOperation op = new GetChildrenOperation(path, watcher, cb); + + zk.getChildren(path, watcher, new ChildrenCallbackWrapper(op), null); + } + + /** + * @param path Path. + * @param watcher Watcher. + * @param cb Callback. + */ + void getDataAsync(String path, Watcher watcher, AsyncCallback.DataCallback cb) { + GetDataOperation op = new GetDataOperation(path, watcher, cb); + + zk.getData(path, watcher, new DataCallbackWrapper(op), null); + } + + /** + * @param path Path. + * @param data Data. + * @param createMode Create mode. + * @param cb Callback. + */ + private void createAsync(String path, byte[] data, CreateMode createMode, AsyncCallback.StringCallback cb) { + if (data == null) + data = EMPTY_BYTES; + + CreateOperation op = new CreateOperation(path, data, createMode, cb); + + zk.create(path, data, ZK_ACL, createMode, new CreateCallbackWrapper(op), null); + } + + /** + * + */ + void onCloseStart() { + closing = true; + + synchronized (stateMux) { + stateMux.notifyAll(); + } + } + + /** + * + */ + public void close() { + closeClient(); + } + + /** + * @param prevConnStartTime Time when connection was established. + * @param e Error. + * @throws ZookeeperClientFailedException If connection to zk was lost. + * @throws InterruptedException If interrupted. + */ + private void onZookeeperError(long prevConnStartTime, Exception e) + throws ZookeeperClientFailedException, InterruptedException + { + ZookeeperClientFailedException err = null; + + synchronized (stateMux) { + if (closing) + throw new ZookeeperClientFailedException("ZooKeeper client is closed."); + + U.warn(log, "Failed to execute ZooKeeper operation [err=" + e + ", state=" + state + ']'); + + if (state == ConnectionState.Lost) { + U.error(log, "Operation failed with unexpected error, connection lost: " + e, e); + + throw new ZookeeperClientFailedException(e); + } + + boolean retry = (e instanceof KeeperException) && needRetry(((KeeperException)e).code().intValue()); + + if (retry) { + long remainingTime; + + if (state == ConnectionState.Connected && connStartTime == prevConnStartTime) { + state = ConnectionState.Disconnected; + + connStartTime = System.currentTimeMillis(); + + remainingTime = connLossTimeout; + } + else { + assert connStartTime != 0; + + assert state == ConnectionState.Disconnected : state; + + remainingTime = connLossTimeout - (System.currentTimeMillis() - connStartTime); + + if (remainingTime <= 0) { + state = ConnectionState.Lost; + + U.warn(log, "Failed to establish ZooKeeper connection, close client " + + "[timeout=" + connLossTimeout + ']'); + + err = new ZookeeperClientFailedException(e); + } + } + + if (err == null) { + U.warn(log, "ZooKeeper operation failed, will retry [err=" + e + + ", retryTimeout=" + RETRY_TIMEOUT + + ", connLossTimeout=" + connLossTimeout + + ", path=" + ((KeeperException)e).getPath() + + ", remainingWaitTime=" + remainingTime + ']'); + + stateMux.wait(RETRY_TIMEOUT); + + if (closing) + throw new ZookeeperClientFailedException("ZooKeeper client is closed."); + } + } + else { + U.error(log, "Operation failed with unexpected error, close ZooKeeper client: " + e, e); + + state = ConnectionState.Lost; + + err = new ZookeeperClientFailedException(e); + } + } + + if (err != null) { + closeClient(); + + notifyConnectionLost(); + + throw err; + } + } + + /** + * @param code Zookeeper error code. + * @return {@code True} if can retry operation. + */ + private boolean needRetry(int code) { + boolean retryByErrorCode = code == KeeperException.Code.CONNECTIONLOSS.intValue() || + code == KeeperException.Code.SESSIONMOVED.intValue() || + code == KeeperException.Code.OPERATIONTIMEOUT.intValue(); + + if (retryByErrorCode) { + if (MAX_RETRY_COUNT <= 0 || retryCount.incrementAndGet() < MAX_RETRY_COUNT) + return true; + else + return false; + } + else + return false; + } + + /** + * + */ + private void closeClient() { + try { + zk.close(); + } + catch (Exception closeErr) { + U.warn(log, "Failed to close ZooKeeper client: " + closeErr, closeErr); + } + + connTimer.cancel(); + } + + /** + * + */ + private void scheduleConnectionCheck() { + assert state == ConnectionState.Disconnected : state; + + connTimer.schedule(new ConnectionTimeoutTask(connStartTime), connLossTimeout); + } + + /** + * + */ + interface ZkAsyncOperation { + /** + * + */ + void execute(); + } + + /** + * + */ + class GetChildrenOperation implements ZkAsyncOperation { + /** */ + private final String path; + + /** */ + private final Watcher watcher; + + /** */ + private final AsyncCallback.Children2Callback cb; + + /** + * @param path Path. + * @param watcher Watcher. + * @param cb Callback. + */ + GetChildrenOperation(String path, Watcher watcher, AsyncCallback.Children2Callback cb) { + this.path = path; + this.watcher = watcher; + this.cb = cb; + } + + /** {@inheritDoc} */ + @Override public void execute() { + getChildrenAsync(path, watcher, cb); + } + } + + /** + * + */ + class GetDataOperation implements ZkAsyncOperation { + /** */ + private final String path; + + /** */ + private final Watcher watcher; + + /** */ + private final AsyncCallback.DataCallback cb; + + /** + * @param path Path. + * @param watcher Watcher. + * @param cb Callback. + */ + GetDataOperation(String path, Watcher watcher, AsyncCallback.DataCallback cb) { + this.path = path; + this.watcher = watcher; + this.cb = cb; + } + + /** {@inheritDoc} */ + @Override public void execute() { + getDataAsync(path, watcher, cb); + } + } + + /** + * + */ + class ExistsOperation implements ZkAsyncOperation { + /** */ + private final String path; + + /** */ + private final Watcher watcher; + + /** */ + private final AsyncCallback.StatCallback cb; + + /** + * @param path Path. + * @param watcher Watcher. + * @param cb Callback. + */ + ExistsOperation(String path, Watcher watcher, AsyncCallback.StatCallback cb) { + this.path = path; + this.watcher = watcher; + this.cb = cb; + } + + /** {@inheritDoc} */ + @Override public void execute() { + existsAsync(path, watcher, cb); + } + } + + /** + * + */ + class CreateOperation implements ZkAsyncOperation { + /** */ + private final String path; + + /** */ + private final byte[] data; + + /** */ + private final CreateMode createMode; + + /** */ + private final AsyncCallback.StringCallback cb; + + /** + * @param path path. + * @param data Data. + * @param createMode Create mode. + * @param cb Callback. + */ + CreateOperation(String path, byte[] data, CreateMode createMode, AsyncCallback.StringCallback cb) { + this.path = path; + this.data = data; + this.createMode = createMode; + this.cb = cb; + } + + /** {@inheritDoc} */ + @Override public void execute() { + createAsync(path, data, createMode, cb); + } + } + + /** + * + */ + class DeleteIfExistsOperation implements AsyncCallback.VoidCallback, ZkAsyncOperation { + /** */ + private final String path; + + /** + * @param path Path. + */ + DeleteIfExistsOperation(String path) { + this.path = path; + } + + /** {@inheritDoc} */ + @Override public void execute() { + zk.delete(path, -1, this, null); + } + + /** {@inheritDoc} */ + @Override public void processResult(int rc, String path, Object ctx) { + if (closing) + return; + + if (rc == KeeperException.Code.NONODE.intValue()) + return; + + if (needRetry(rc)) { + U.warn(log, "Failed to execute async operation, connection lost. Will retry after connection restore [" + + "path=" + path + ']'); + + retryQ.add(this); + } + else if (rc == KeeperException.Code.SESSIONEXPIRED.intValue()) + U.warn(log, "Failed to execute async operation, connection lost [path=" + path + ']'); + else + assert rc == 0 : KeeperException.Code.get(rc); + } + } + + /** + * + */ + class CreateCallbackWrapper implements AsyncCallback.StringCallback { + /** */ + final CreateOperation op; + + /** + * @param op Operation. + */ + CreateCallbackWrapper(CreateOperation op) { + this.op = op; + } + + @Override public void processResult(int rc, String path, Object ctx, String name) { + if (closing) + return; + + if (rc == KeeperException.Code.NODEEXISTS.intValue()) + return; + + if (needRetry(rc)) { + U.warn(log, "Failed to execute async operation, connection lost. Will retry after connection restore [path=" + path + ']'); + + retryQ.add(op); + } + else if (rc == KeeperException.Code.SESSIONEXPIRED.intValue()) + U.warn(log, "Failed to execute async operation, connection lost [path=" + path + ']'); + else { + if (op.cb != null) + op.cb.processResult(rc, path, ctx, name); + } + } + } + + /** + * + */ + class ChildrenCallbackWrapper implements AsyncCallback.Children2Callback { + /** */ + private final GetChildrenOperation op; + + /** + * @param op Operation. + */ + private ChildrenCallbackWrapper(GetChildrenOperation op) { + this.op = op; + } + + /** {@inheritDoc} */ + @Override public void processResult(int rc, String path, Object ctx, List children, Stat stat) { + if (closing) + return; + + if (needRetry(rc)) { + U.warn(log, "Failed to execute async operation, connection lost. Will retry after connection restore [path=" + path + ']'); + + retryQ.add(op); + } + else if (rc == KeeperException.Code.SESSIONEXPIRED.intValue()) + U.warn(log, "Failed to execute async operation, connection lost [path=" + path + ']'); + else + op.cb.processResult(rc, path, ctx, children, stat); + } + } + + /** + * + */ + class DataCallbackWrapper implements AsyncCallback.DataCallback { + /** */ + private final GetDataOperation op; + + /** + * @param op Operation. + */ + private DataCallbackWrapper(GetDataOperation op) { + this.op = op; + } + + /** {@inheritDoc} */ + @Override public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) { + if (closing) + return; + + if (needRetry(rc)) { + U.warn(log, "Failed to execute async operation, connection lost. Will retry after connection restore [path=" + path + ']'); + + retryQ.add(op); + } + else if (rc == KeeperException.Code.SESSIONEXPIRED.intValue()) + U.warn(log, "Failed to execute async operation, connection lost [path=" + path + ']'); + else + op.cb.processResult(rc, path, ctx, data, stat); + } + } + + /** + * + */ + class StatCallbackWrapper implements AsyncCallback.StatCallback { + /** */ + private final ExistsOperation op; + + /** + * @param op Operation. + */ + private StatCallbackWrapper(ExistsOperation op) { + this.op = op; + } + + /** {@inheritDoc} */ + @Override public void processResult(int rc, String path, Object ctx, Stat stat) { + if (closing) + return; + + if (needRetry(rc)) { + U.warn(log, "Failed to execute async operation, connection lost. Will retry after connection restore [path=" + path + ']'); + + retryQ.add(op); + } + else if (rc == KeeperException.Code.SESSIONEXPIRED.intValue()) + U.warn(log, "Failed to execute async operation, connection lost [path=" + path + ']'); + else + op.cb.processResult(rc, path, ctx, stat); + } + } + + /** + * + */ + private class ConnectionTimeoutTask extends TimerTask { + /** */ + private final long connectStartTime; + + /** + * @param connectStartTime Time was connection started. + */ + ConnectionTimeoutTask(long connectStartTime) { + this.connectStartTime = connectStartTime; + } + + /** {@inheritDoc} */ + @Override public void run() { + boolean connLoss = false; + + synchronized (stateMux) { + if (closing) + return; + + if (state == ConnectionState.Disconnected && + ZookeeperClient.this.connStartTime == connectStartTime) { + + state = ConnectionState.Lost; + + U.warn(log, "Failed to establish ZooKeeper connection, close client " + + "[timeout=" + connLossTimeout + ']'); + + connLoss = true; + } + } + + if (connLoss) { + closeClient(); + + notifyConnectionLost(); + } + } + } + + /** + * + */ + private enum ConnectionState { + /** */ + Connected, + /** */ + Disconnected, + /** */ + Lost + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientFailedException.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientFailedException.java new file mode 100644 index 0000000000000..01d011b0cff5d --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientFailedException.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +/** + * + */ +class ZookeeperClientFailedException extends Exception { + /** */ + private static final long serialVersionUID = 0L; + + /** + * @param msg Message. + */ + ZookeeperClientFailedException(String msg) { + super(msg); + } + + /** + * @param cause Cause. + */ + ZookeeperClientFailedException(Throwable cause) { + super(cause); + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java new file mode 100644 index 0000000000000..3cb5fad3129cf --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java @@ -0,0 +1,362 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.cache.CacheMetrics; +import org.apache.ignite.cluster.ClusterMetrics; +import org.apache.ignite.internal.IgniteNodeAttributes; +import org.apache.ignite.internal.managers.discovery.IgniteClusterNode; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.lang.IgniteProductVersion; +import org.apache.ignite.spi.discovery.DiscoveryMetricsProvider; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_DAEMON; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_NODE_CONSISTENT_ID; + +/** + * Zookeeper Cluster Node. + */ +public class ZookeeperClusterNode implements IgniteClusterNode, Serializable, Comparable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private static final byte CLIENT_NODE_MASK = 0x01; + + /** */ + private UUID id; + + /** */ + private Serializable consistentId; + + /** */ + private long internalId; + + /** */ + private long order; + + /** */ + private IgniteProductVersion ver; + + /** Node attributes. */ + private Map attrs; + + /** Internal discovery addresses as strings. */ + private Collection addrs; + + /** Internal discovery host names as strings. */ + private Collection hostNames; + + /** */ + private long sesTimeout; + + /** Metrics provider. */ + private transient DiscoveryMetricsProvider metricsProvider; + + /** */ + private transient boolean loc; + + /** */ + private transient volatile ClusterMetrics metrics; + + /** Node cache metrics. */ + @GridToStringExclude + private transient volatile Map cacheMetrics; + + /** */ + private byte flags; + + /** Daemon node flag. */ + @GridToStringExclude + private transient boolean daemon; + + /** Daemon node initialization flag. */ + @GridToStringExclude + private transient volatile boolean daemonInit; + + /** + * @param id Node ID. + * @param addrs Node addresses. + * @param hostNames Node host names. + * @param ver Node version. + * @param attrs Node attributes. + * @param consistentId Consistent ID. + * @param sesTimeout Zookeeper session timeout. + * @param client Client node flag. + * @param metricsProvider Metrics provider. + */ + public ZookeeperClusterNode( + UUID id, + Collection addrs, + Collection hostNames, + IgniteProductVersion ver, + Map attrs, + Serializable consistentId, + long sesTimeout, + boolean client, + DiscoveryMetricsProvider metricsProvider + ) { + assert id != null; + assert consistentId != null; + + this.id = id; + this.ver = ver; + this.attrs = Collections.unmodifiableMap(attrs); + this.addrs = addrs; + this.hostNames = hostNames; + this.consistentId = consistentId; + this.sesTimeout = sesTimeout; + this.metricsProvider = metricsProvider; + + if (client) + flags |= CLIENT_NODE_MASK; + } + + /** {@inheritDoc} */ + @Override public UUID id() { + return id; + } + + /** {@inheritDoc} */ + @Override public Object consistentId() { + return consistentId; + } + + /** {@inheritDoc} */ + public void setConsistentId(Serializable consistentId) { + this.consistentId = consistentId; + + final Map map = new HashMap<>(attrs); + + map.put(ATTR_NODE_CONSISTENT_ID, consistentId); + + attrs = Collections.unmodifiableMap(map); + } + + /** {@inheritDoc} */ + @Override public boolean isCacheClient() { + return isClient(); + } + + /** {@inheritDoc} */ + @Nullable @Override public T attribute(String name) { + // Even though discovery SPI removes this attribute after authentication, keep this check for safety. + if (IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS.equals(name)) + return null; + + return (T)attrs.get(name); + } + + /** + * Sets node attributes. + * + * @param attrs Node attributes. + */ + void setAttributes(Map attrs) { + this.attrs = U.sealMap(attrs); + } + + /** + * Gets node attributes without filtering. + * + * @return Node attributes without filtering. + */ + Map getAttributes() { + return attrs; + } + + /** {@inheritDoc} */ + @Override public ClusterMetrics metrics() { + if (metricsProvider != null) { + ClusterMetrics metrics0 = metricsProvider.metrics(); + + assert metrics0 != null; + + metrics = metrics0; + + return metrics0; + } + + return metrics; + } + + /** {@inheritDoc} */ + public void setMetrics(ClusterMetrics metrics) { + assert metrics != null; + + this.metrics = metrics; + } + + /** {@inheritDoc} */ + @Override public Map cacheMetrics() { + if (metricsProvider != null) { + Map cacheMetrics0 = metricsProvider.cacheMetrics(); + + cacheMetrics = cacheMetrics0; + + return cacheMetrics0; + } + + return cacheMetrics; + } + + /** {@inheritDoc} */ + public void setCacheMetrics(Map cacheMetrics) { + this.cacheMetrics = cacheMetrics != null ? cacheMetrics : Collections.emptyMap(); + } + + /** {@inheritDoc} */ + @Override public Map attributes() { + // Even though discovery SPI removes this attribute after authentication, keep this check for safety. + return F.view(attrs, new IgnitePredicate() { + @Override public boolean apply(String s) { + return !IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS.equals(s); + } + }); + } + + /** {@inheritDoc} */ + @Override public Collection addresses() { + return addrs; + } + + /** {@inheritDoc} */ + @Override public Collection hostNames() { + return hostNames; + } + + /** {@inheritDoc} */ + @Override public long order() { + return order; + } + + /** + * @return Internal ID corresponds to Zookeeper sequential node. + */ + long internalId() { + return internalId; + } + + /** + * @param internalId Internal ID corresponds to Zookeeper sequential node. + */ + void internalId(long internalId) { + this.internalId = internalId; + } + + /** + * @param order Node order. + */ + void order(long order) { + assert order > 0 : order; + + this.order = order; + } + + /** + * @param newId New node ID. + */ + public void onClientDisconnected(UUID newId) { + id = newId; + } + + /** + * @return Session timeout. + */ + long sessionTimeout() { + return sesTimeout; + } + + /** {@inheritDoc} */ + @Override public IgniteProductVersion version() { + return ver; + } + + /** + * @param loc Local node flag. + */ + public void local(boolean loc) { + this.loc = loc; + } + + /** {@inheritDoc} */ + @Override public boolean isLocal() { + return loc; + } + + /** {@inheritDoc} */ + @Override public boolean isDaemon() { + if (!daemonInit) { + daemon = "true".equalsIgnoreCase((String)attribute(ATTR_DAEMON)); + + daemonInit = true; + } + + return daemon; + } + + /** {@inheritDoc} */ + @Override public boolean isClient() { + return (CLIENT_NODE_MASK & flags) != 0; + } + + /** {@inheritDoc} */ + @Override public int compareTo(@Nullable ZookeeperClusterNode node) { + if (node == null) + return 1; + + int res = Long.compare(order, node.order); + + if (res == 0) { + assert id().equals(node.id()) : "Duplicate order [this=" + this + ", other=" + node + ']'; + + res = id().compareTo(node.id()); + } + + return res; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return id.hashCode(); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object obj) { + return F.eqNodes(this, obj); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "ZookeeperClusterNode [id=" + id + + ", addrs=" + addrs + + ", order=" + order + + ", loc=" + loc + + ", client=" + isClient() + ']'; + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java new file mode 100644 index 0000000000000..7708358f687dc --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java @@ -0,0 +1,4464 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.io.ByteArrayInputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.BitSet; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.zip.DataFormatException; +import java.util.zip.Deflater; +import java.util.zip.Inflater; +import java.util.zip.InflaterInputStream; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteClientDisconnectedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteInterruptedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CommunicationFailureResolver; +import org.apache.ignite.events.EventType; +import org.apache.ignite.internal.ClusterMetricsSnapshot; +import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; +import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.IgnitionEx; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; +import org.apache.ignite.internal.events.DiscoveryCustomEvent; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpiInternalListener; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.security.SecurityContext; +import org.apache.ignite.internal.util.GridLongList; +import org.apache.ignite.internal.util.GridSpinBusyLock; +import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.io.GridByteArrayOutputStream; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.internal.util.typedef.internal.LT; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.lang.IgniteRunnable; +import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.marshaller.MarshallerUtils; +import org.apache.ignite.marshaller.jdk.JdkMarshaller; +import org.apache.ignite.plugin.security.SecurityCredentials; +import org.apache.ignite.spi.IgniteNodeValidationResult; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.IgniteSpiTimeoutObject; +import org.apache.ignite.spi.discovery.DiscoveryDataBag; +import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; +import org.apache.ignite.spi.discovery.DiscoverySpiDataExchange; +import org.apache.ignite.spi.discovery.DiscoverySpiListener; +import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator; +import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi; +import org.apache.ignite.thread.IgniteThreadPoolExecutor; +import org.apache.zookeeper.AsyncCallback; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.data.Stat; +import org.jboss.netty.util.internal.ConcurrentHashMap; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED; +import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_RECONNECTED; +import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; +import static org.apache.ignite.events.EventType.EVT_NODE_JOINED; +import static org.apache.ignite.events.EventType.EVT_NODE_SEGMENTED; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGNITE_INSTANCE_NAME; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2; +import static org.apache.zookeeper.CreateMode.EPHEMERAL_SEQUENTIAL; +import static org.apache.zookeeper.CreateMode.PERSISTENT; + +/** + * Zookeeper Discovery Impl. + */ +public class ZookeeperDiscoveryImpl { + /** */ + static final String IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD = "IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD"; + + /** */ + static final String IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_TIMEOUT = "IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_TIMEOUT"; + + /** */ + static final String IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS = "IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS"; + + /** */ + private static final String IGNITE_ZOOKEEPER_DISCOVERY_SPI_EVTS_THROTTLE = "IGNITE_ZOOKEEPER_DISCOVERY_SPI_EVTS_THROTTLE"; + + /** */ + final ZookeeperDiscoverySpi spi; + + /** */ + private final String igniteInstanceName; + + /** */ + private final String connectString; + + /** */ + private final int sesTimeout; + + /** */ + private final JdkMarshaller marsh = new JdkMarshaller(); + + /** */ + private final ZkIgnitePaths zkPaths; + + /** */ + private final IgniteLogger log; + + /** */ + final GridSpinBusyLock busyLock = new GridSpinBusyLock(); + + /** */ + private final ZookeeperClusterNode locNode; + + /** */ + private final DiscoverySpiListener lsnr; + + /** */ + private final DiscoverySpiDataExchange exchange; + + /** */ + private final boolean clientReconnectEnabled; + + /** */ + private final GridFutureAdapter joinFut = new GridFutureAdapter<>(); + + /** */ + private final int evtsAckThreshold; + + /** */ + private IgniteThreadPoolExecutor utilityPool; + + /** */ + private ZkRuntimeState rtState; + + /** */ + private volatile ConnectionState connState = ConnectionState.STARTED; + + /** */ + private final AtomicBoolean stop = new AtomicBoolean(); + + /** */ + private final Object stateMux = new Object(); + + /** */ + public volatile IgniteDiscoverySpiInternalListener internalLsnr; + + /** */ + private final ConcurrentHashMap pingFuts = new ConcurrentHashMap<>(); + + /** */ + private final AtomicReference commErrProcFut = new AtomicReference<>(); + + /** */ + private long prevSavedEvtsTopVer; + + /** + * @param spi Discovery SPI. + * @param igniteInstanceName Instance name. + * @param log Logger. + * @param zkRootPath Zookeeper base path node all nodes. + * @param locNode Local node instance. + * @param lsnr Discovery events listener. + * @param exchange Discovery data exchange. + * @param internalLsnr Internal listener (used for testing only). + */ + public ZookeeperDiscoveryImpl( + ZookeeperDiscoverySpi spi, + String igniteInstanceName, + IgniteLogger log, + String zkRootPath, + ZookeeperClusterNode locNode, + DiscoverySpiListener lsnr, + DiscoverySpiDataExchange exchange, + IgniteDiscoverySpiInternalListener internalLsnr) { + assert locNode.id() != null && locNode.isLocal() : locNode; + + MarshallerUtils.setNodeName(marsh, igniteInstanceName); + + zkPaths = new ZkIgnitePaths(zkRootPath); + + this.spi = spi; + this.igniteInstanceName = igniteInstanceName; + this.connectString = spi.getZkConnectionString(); + this.sesTimeout = (int)spi.getSessionTimeout(); + this.log = log.getLogger(getClass()); + this.locNode = locNode; + this.lsnr = lsnr; + this.exchange = exchange; + this.clientReconnectEnabled = locNode.isClient() && !spi.isClientReconnectDisabled(); + + int evtsAckThreshold = IgniteSystemProperties.getInteger(IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD, 5); + + if (evtsAckThreshold <= 0) + evtsAckThreshold = 1; + + this.evtsAckThreshold = evtsAckThreshold; + + if (internalLsnr != null) + this.internalLsnr = internalLsnr; + } + + /** + * @return Exception. + */ + private static IgniteClientDisconnectedCheckedException disconnectError() { + return new IgniteClientDisconnectedCheckedException(null, "Client node disconnected."); + } + + /** + * @return Logger. + */ + IgniteLogger log() { + return log; + } + + /** + * @return Local node instance. + */ + public ClusterNode localNode() { + return locNode; + } + + /** + * @param nodeId Node ID. + * @return Node instance. + */ + @Nullable public ZookeeperClusterNode node(UUID nodeId) { + assert nodeId != null; + + return rtState.top.nodesById.get(nodeId); + } + + /** + * @param nodeOrder Node order. + * @return Node instance. + */ + @Nullable public ZookeeperClusterNode node(long nodeOrder) { + assert nodeOrder > 0 : nodeOrder; + + return rtState.top.nodesByOrder.get(nodeOrder); + } + + /** + * @param fut Future to remove. + */ + void clearCommunicationErrorProcessFuture(ZkCommunicationErrorProcessFuture fut) { + assert fut.isDone() : fut; + + commErrProcFut.compareAndSet(fut, null); + } + + /** + * @param node0 Problem node ID + * @param err Connect error. + */ + public void resolveCommunicationError(ClusterNode node0, Exception err) { + ZookeeperClusterNode node = node(node0.id()); + + if (node == null) + throw new IgniteSpiException(new ClusterTopologyCheckedException("Node failed: " + node0.id())); + + IgniteInternalFuture nodeStatusFut; + + for (;;) { + checkState(); + + ZkCommunicationErrorProcessFuture fut = commErrProcFut.get(); + + if (fut == null || fut.isDone()) { + ZkCommunicationErrorProcessFuture newFut = ZkCommunicationErrorProcessFuture.createOnCommunicationError( + this, + node.sessionTimeout() + 1000); + + if (commErrProcFut.compareAndSet(fut, newFut)) { + fut = newFut; + + if (log.isInfoEnabled()) { + log.info("Created new communication error process future [errNode=" + node0.id() + + ", err=" + err + ']'); + } + + try { + checkState(); + } + catch (Exception e) { + fut.onError(e); + + throw e; + } + + fut.scheduleCheckOnTimeout(); + } + else { + fut = commErrProcFut.get(); + + if (fut == null) + continue; + } + } + + nodeStatusFut = fut.nodeStatusFuture(node); + + if (nodeStatusFut != null) + break; + else { + try { + fut.get(); + } + catch (IgniteCheckedException e) { + U.warn(log, "Previous communication error process future failed: " + e); + } + } + } + + try { + if (!nodeStatusFut.get()) + throw new IgniteSpiException(new ClusterTopologyCheckedException("Node failed: " + node0.id())); + } + catch (IgniteCheckedException e) { + throw new IgniteSpiException(e); + } + } + + /** + * @param nodeId Node ID. + * @return Ping result. + */ + public boolean pingNode(UUID nodeId) { + checkState(); + + ZkRuntimeState rtState = this.rtState; + + ZookeeperClusterNode node = rtState.top.nodesById.get(nodeId); + + if (node == null) + return false; + + if (node.isLocal()) + return true; + + PingFuture fut = pingFuts.get(node.order()); + + if (fut == null) { + fut = new PingFuture(rtState, node); + + PingFuture old = pingFuts.putIfAbsent(node.order(), fut); + + if (old == null) { + if (fut.checkNodeAndState()) + spi.getSpiContext().addTimeoutObject(fut); + else + assert fut.isDone(); + } + else + fut = old; + } + + try { + return fut.get(); + } + catch (IgniteCheckedException e) { + throw new IgniteSpiException(e); + } + } + + /** + * @param nodeId Node ID. + * @param warning Warning. + */ + public void failNode(UUID nodeId, @Nullable String warning) { + ZookeeperClusterNode node = rtState.top.nodesById.get(nodeId); + + if (node == null) { + if (log.isDebugEnabled()) + log.debug("Ignore forcible node fail request, node does not exist: " + nodeId); + + return; + } + + if (!node.isClient()) { + U.warn(log, "Ignore forcible node fail request for non-client node: " + node); + + return; + } + + sendCustomMessage(new ZkForceNodeFailMessage(node.internalId(), warning)); + } + + /** + * + */ + public void reconnect() { + assert clientReconnectEnabled; + + synchronized (stateMux) { + if (connState == ConnectionState.STARTED) { + connState = ConnectionState.DISCONNECTED; + + rtState.onCloseStart(disconnectError()); + } + else + return; + } + + busyLock.block(); + + busyLock.unblock(); + + rtState.zkClient.close(); + + UUID newId = UUID.randomUUID(); + + U.quietAndWarn(log, "Local node will try to reconnect to cluster with new id due to network problems [" + + "newId=" + newId + + ", prevId=" + locNode.id() + + ", locNode=" + locNode + ']'); + + runInWorkerThread(new ReconnectClosure(newId)); + } + + /** + * @param newId New ID. + */ + private void doReconnect(UUID newId) { + if (rtState.joined) { + assert rtState.evtsData != null; + + lsnr.onDiscovery(EVT_CLIENT_NODE_DISCONNECTED, + rtState.evtsData.topVer, + locNode, + rtState.top.topologySnapshot(), + Collections.>emptyMap(), + null); + } + + try { + locNode.onClientDisconnected(newId); + + joinTopology(rtState); + } + catch (Exception e) { + if (stopping()) { + if (log.isDebugEnabled()) + log.debug("Reconnect failed, node is stopping [err=" + e + ']'); + + return; + } + + U.error(log, "Failed to reconnect: " + e, e); + + onSegmented(e); + } + } + + /** + * @return {@code True} if started to stop. + */ + private boolean stopping() { + if (stop.get()) + return true; + + synchronized (stateMux) { + if (connState == ConnectionState.STOPPED) + return true; + } + + return false; + } + + /** + * @param e Error. + */ + private void onSegmented(Exception e) { + rtState.errForClose = e; + + if (rtState.joined || joinFut.isDone()) { + synchronized (stateMux) { + connState = ConnectionState.STOPPED; + } + + notifySegmented(); + } + else + joinFut.onDone(e); + } + + /** + * + */ + private void notifySegmented() { + List nodes = rtState.top.topologySnapshot(); + + if (nodes.isEmpty()) + nodes = Collections.singletonList((ClusterNode)locNode); + + lsnr.onDiscovery(EVT_NODE_SEGMENTED, + rtState.evtsData != null ? rtState.evtsData.topVer : 1L, + locNode, + nodes, + Collections.>emptyMap(), + null); + } + + /** + * @return Remote nodes. + */ + public Collection remoteNodes() { + checkState(); + + return rtState.top.remoteNodes(); + } + + /** + * + */ + private void checkState() { + switch (connState) { + case STARTED: + break; + + case STOPPED: + throw new IgniteSpiException("Node stopped."); + + case DISCONNECTED: + throw new IgniteClientDisconnectedException(null, "Client is disconnected."); + } + } + + /** + * @param nodeId Node ID. + * @return {@code True} if node joined or joining topology. + */ + public boolean knownNode(UUID nodeId) { + while (!busyLock.enterBusy()) + checkState(); + + try { + List children = rtState.zkClient.getChildren(zkPaths.aliveNodesDir); + + for (int i = 0; i < children.size(); i++) { + UUID id = ZkIgnitePaths.aliveNodeId(children.get(i)); + + if (nodeId.equals(id)) + return true; + } + + return false; + } + catch (ZookeeperClientFailedException e) { + if (clientReconnectEnabled) + throw new IgniteClientDisconnectedException(null, "Client is disconnected."); + + throw new IgniteException(e); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + + throw new IgniteInterruptedException(e); + } + finally { + busyLock.leaveBusy(); + } + } + + /** + * @param msg Message. + */ + public void sendCustomMessage(DiscoverySpiCustomMessage msg) { + assert msg != null; + + byte[] msgBytes; + + try { + msgBytes = marshalZip(msg); + } + catch (IgniteCheckedException e) { + throw new IgniteSpiException("Failed to marshal custom message: " + msg, e); + } + + while (!busyLock.enterBusy()) + checkState(); + + try { + ZookeeperClient zkClient = rtState.zkClient; + + saveCustomMessage(zkClient, msgBytes); + } + catch (ZookeeperClientFailedException e) { + if (clientReconnectEnabled) + throw new IgniteClientDisconnectedException(null, "Client is disconnected."); + + throw new IgniteException(e); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + + throw new IgniteInterruptedException(e); + } + finally { + busyLock.leaveBusy(); + } + } + + /** + * @param zkClient Client. + * @param msgBytes Marshalled message. + * @throws ZookeeperClientFailedException If connection to zk was lost. + * @throws InterruptedException If interrupted. + */ + private void saveCustomMessage(ZookeeperClient zkClient, byte[] msgBytes) + throws ZookeeperClientFailedException, InterruptedException + { + String prefix = UUID.randomUUID().toString(); + + int partCnt = 1; + + int overhead = 10; + + UUID locId = locNode.id(); + + String path = zkPaths.createCustomEventPath(prefix, locId, partCnt); + + if (zkClient.needSplitNodeData(path, msgBytes, overhead)) { + List parts = zkClient.splitNodeData(path, msgBytes, overhead); + + String partsBasePath = zkPaths.customEventPartsBasePath(prefix, locId); + + saveMultipleParts(zkClient, partsBasePath, parts); + + msgBytes = null; + + partCnt = parts.size(); + } + + zkClient.createSequential(prefix, + zkPaths.customEvtsDir, + zkPaths.createCustomEventPath(prefix, locId, partCnt), + msgBytes, + CreateMode.PERSISTENT_SEQUENTIAL); + } + + /** + * @return Cluster start time. + */ + public long gridStartTime() { + return rtState.gridStartTime; + } + + /** + * Starts join procedure and waits for {@link EventType#EVT_NODE_JOINED} event for local node. + * + * @throws InterruptedException If interrupted. + */ + public void startJoinAndWait() throws InterruptedException { + joinTopology(null); + + for (;;) { + try { + joinFut.get(10_000); + + break; + } + catch (IgniteFutureTimeoutCheckedException e) { + U.warn(log, "Waiting for local join event [nodeId=" + locNode.id() + ", name=" + igniteInstanceName + ']'); + } + catch (Exception e) { + IgniteSpiException spiErr = X.cause(e, IgniteSpiException.class); + + if (spiErr != null) + throw spiErr; + + throw new IgniteSpiException("Failed to join cluster", e); + } + } + } + + /** + * @param prevState Previous state in case of connect retry. + * @throws InterruptedException If interrupted. + */ + private void joinTopology(@Nullable ZkRuntimeState prevState) throws InterruptedException { + if (!busyLock.enterBusy()) + return; + + try { + boolean reconnect = prevState != null; + + // Need fire EVT_CLIENT_NODE_RECONNECTED event if reconnect after already joined. + boolean prevJoined = prevState != null && prevState.joined; + + IgniteDiscoverySpiInternalListener internalLsnr = this.internalLsnr; + + if (internalLsnr != null) + internalLsnr.beforeJoin(locNode, log); + + if (locNode.isClient() && reconnect) + locNode.setAttributes(spi.getSpiContext().nodeAttributes()); + + marshalCredentialsOnJoin(locNode); + + synchronized (stateMux) { + if (connState == ConnectionState.STOPPED) + return; + + connState = ConnectionState.STARTED; + } + + ZkRuntimeState rtState = this.rtState = new ZkRuntimeState(prevJoined); + + DiscoveryDataBag discoDataBag = new DiscoveryDataBag(locNode.id(), locNode.isClient()); + + exchange.collect(discoDataBag); + + ZkJoiningNodeData joinData = new ZkJoiningNodeData(locNode, discoDataBag.joiningNodeData()); + + byte[] joinDataBytes; + + try { + joinDataBytes = marshalZip(joinData); + } + catch (Exception e) { + throw new IgniteSpiException("Failed to marshal joining node data", e); + } + + try { + rtState.zkClient = new ZookeeperClient( + igniteInstanceName, + log, + connectString, + sesTimeout, + new ConnectionLossListener()); + } + catch (Exception e) { + throw new IgniteSpiException("Failed to create Zookeeper client", e); + } + + startJoin(rtState, prevState, joinDataBytes); + } + finally { + busyLock.leaveBusy(); + } + } + + /** + * @throws InterruptedException If interrupted. + */ + private void initZkNodes() throws InterruptedException { + try { + ZookeeperClient client = rtState.zkClient; + + if (!client.exists(zkPaths.clusterDir)) { + createRootPathParents(zkPaths.clusterDir, client); + + client.createIfNeeded(zkPaths.clusterDir, null, PERSISTENT); + } + + List createdDirs = client.getChildren(zkPaths.clusterDir); + + String[] requiredDirs = { + zkPaths.evtsPath, + zkPaths.joinDataDir, + zkPaths.customEvtsDir, + zkPaths.customEvtsPartsDir, + zkPaths.customEvtsAcksDir, + zkPaths.aliveNodesDir}; + + List dirs = new ArrayList<>(); + + for (String dir : requiredDirs) { + String dir0 = dir.substring(zkPaths.clusterDir.length() + 1); + + if (!createdDirs.contains(dir0)) + dirs.add(dir); + } + + try { + if (!dirs.isEmpty()) + client.createAll(dirs, PERSISTENT); + } + catch (KeeperException.NodeExistsException e) { + if (log.isDebugEnabled()) + log.debug("Failed to create nodes using bulk operation: " + e); + + for (String dir : dirs) + client.createIfNeeded(dir, null, PERSISTENT); + } + } + catch (ZookeeperClientFailedException e) { + throw new IgniteSpiException("Failed to initialize Zookeeper nodes", e); + } + } + + /** + * @param rootDir Root directory. + * @param client Client. + * @throws ZookeeperClientFailedException If connection to zk was lost. + * @throws InterruptedException If interrupted. + */ + private void createRootPathParents(String rootDir, ZookeeperClient client) + throws ZookeeperClientFailedException, InterruptedException { + int startIdx = 0; + + for (;;) { + int separatorIdx = rootDir.indexOf(ZkIgnitePaths.PATH_SEPARATOR, startIdx); + + if (separatorIdx == -1) + break; + + if (separatorIdx > 0) { + String path = rootDir.substring(0, separatorIdx); + + client.createIfNeeded(path, null, CreateMode.PERSISTENT); + } + + startIdx = separatorIdx + 1; + } + } + + /** + * @param zkClient Client. + * @param basePath Base path. + * @param partCnt Parts count. + */ + private void deleteMultiplePartsAsync(ZookeeperClient zkClient, String basePath, int partCnt) { + for (int i = 0; i < partCnt; i++) { + String path = multipartPathName(basePath, i); + + zkClient.deleteIfExistsAsync(path); + } + } + + /** + * @param zkClient Client. + * @param basePath Base path. + * @param partCnt Parts count. + * @return Read parts. + * @throws Exception If failed. + */ + private byte[] readMultipleParts(ZookeeperClient zkClient, String basePath, int partCnt) + throws Exception { + assert partCnt >= 1; + + if (partCnt > 1) { + List parts = new ArrayList<>(partCnt); + + int totSize = 0; + + for (int i = 0; i < partCnt; i++) { + byte[] part = zkClient.getData(multipartPathName(basePath, i)); + + parts.add(part); + + totSize += part.length; + } + + byte[] res = new byte[totSize]; + + int pos = 0; + + for (int i = 0; i < partCnt; i++) { + byte[] part = parts.get(i); + + System.arraycopy(part, 0, res, pos, part.length); + + pos += part.length; + } + + return res; + } + else + return zkClient.getData(multipartPathName(basePath, 0)); + } + + /** + * @param zkClient Client. + * @param basePath Base path. + * @param parts Data parts. + * @return Number of parts. + * @throws ZookeeperClientFailedException If client failed. + * @throws InterruptedException If interrupted. + */ + private int saveMultipleParts(ZookeeperClient zkClient, String basePath, List parts) + throws ZookeeperClientFailedException, InterruptedException + { + assert parts.size() > 1; + + for (int i = 0; i < parts.size(); i++) { + byte[] part = parts.get(i); + + String path = multipartPathName(basePath, i); + + zkClient.createIfNeeded(path, part, PERSISTENT); + } + + return parts.size(); + } + + /** + * @param basePath Base path. + * @param part Part number. + * @return Path. + */ + private static String multipartPathName(String basePath, int part) { + return basePath + String.format("%04d", part); + } + + /** + * @param rtState Runtime state. + * @param joinDataBytes Joining node data. + * @param prevState Previous state in case of connect retry. + * @throws InterruptedException If interrupted. + */ + private void startJoin(ZkRuntimeState rtState, @Nullable ZkRuntimeState prevState, final byte[] joinDataBytes) + throws InterruptedException + { + try { + long startTime = System.currentTimeMillis(); + + initZkNodes(); + + String prefix = UUID.randomUUID().toString(); + + rtState.init(new ZkWatcher(rtState), new AliveNodeDataWatcher(rtState)); + + ZookeeperClient zkClient = rtState.zkClient; + + final int OVERHEAD = 5; + + // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8193 + String joinDataPath = zkPaths.joinDataDir + "/" + prefix + ":" + locNode.id(); + + if (zkClient.needSplitNodeData(joinDataPath, joinDataBytes, OVERHEAD)) { + List parts = zkClient.splitNodeData(joinDataPath, joinDataBytes, OVERHEAD); + + rtState.joinDataPartCnt = parts.size(); + + saveMultipleParts(zkClient, joinDataPath + ":", parts); + + joinDataPath = zkClient.createIfNeeded( + joinDataPath, + marshalZip(new ZkJoiningNodeData(parts.size())), + PERSISTENT); + } + else { + joinDataPath = zkClient.createIfNeeded( + joinDataPath, + joinDataBytes, + PERSISTENT); + } + + rtState.locNodeZkPath = zkClient.createSequential( + prefix, + zkPaths.aliveNodesDir, + zkPaths.aliveNodePathForCreate(prefix, locNode), + null, + EPHEMERAL_SEQUENTIAL); + + rtState.internalOrder = ZkIgnitePaths.aliveInternalId(rtState.locNodeZkPath); + + if (log.isInfoEnabled()) { + log.info("Node started join [nodeId=" + locNode.id() + + ", instanceName=" + locNode.attribute(ATTR_IGNITE_INSTANCE_NAME) + + ", zkSessionId=0x" + Long.toHexString(rtState.zkClient.zk().getSessionId()) + + ", joinDataSize=" + joinDataBytes.length + + (rtState.joinDataPartCnt > 1 ? (", joinDataPartCnt=" + rtState.joinDataPartCnt) : "") + + ", consistentId=" + locNode.consistentId() + + ", initTime=" + (System.currentTimeMillis() - startTime) + + ", nodePath=" + rtState.locNodeZkPath + ']'); + } + + /* + If node can not join due to validation error this error is reported in join data, + As a minor optimization do not start watch join data immediately, but only if do not receive + join event after some timeout. + */ + CheckJoinErrorWatcher joinErrorWatcher = new CheckJoinErrorWatcher(5000, joinDataPath, rtState); + + rtState.joinErrTo = joinErrorWatcher.timeoutObj; + + if (locNode.isClient() && spi.getJoinTimeout() > 0) { + ZkTimeoutObject joinTimeoutObj = prevState != null ? prevState.joinTo : null; + + if (joinTimeoutObj == null) { + joinTimeoutObj = new JoinTimeoutObject(spi.getJoinTimeout()); + + spi.getSpiContext().addTimeoutObject(joinTimeoutObj); + } + + rtState.joinTo = joinTimeoutObj; + } + + if (!locNode.isClient()) + zkClient.getChildrenAsync(zkPaths.aliveNodesDir, null, new CheckCoordinatorCallback(rtState)); + + zkClient.getDataAsync(zkPaths.evtsPath, rtState.watcher, rtState.watcher); + + spi.getSpiContext().addTimeoutObject(rtState.joinErrTo); + } + catch (IgniteCheckedException | ZookeeperClientFailedException e) { + throw new IgniteSpiException("Failed to initialize Zookeeper nodes", e); + } + } + + /** + * Authenticate local node. + * + * @param nodeAuth Authenticator. + * @param locCred Local security credentials for authentication. + * @throws IgniteSpiException If any error occurs. + */ + private void localAuthentication(DiscoverySpiNodeAuthenticator nodeAuth, SecurityCredentials locCred){ + assert nodeAuth != null; + assert locCred != null; + + try { + SecurityContext subj = nodeAuth.authenticateNode(locNode, locCred); + + // Note: exception message is checked in tests. + if (subj == null) + throw new IgniteSpiException("Authentication failed for local node."); + + if (!(subj instanceof Serializable)) + throw new IgniteSpiException("Authentication subject is not Serializable."); + + Map attrs = new HashMap<>(locNode.attributes()); + + attrs.put(ATTR_SECURITY_SUBJECT_V2, U.marshal(marsh, subj)); + + locNode.setAttributes(attrs); + } + catch (Exception e) { + throw new IgniteSpiException("Failed to authenticate local node (will shutdown local node).", e); + } + } + + /** + * @param node Node. + * @param zipBytes Zip-compressed marshalled security subject. + * @throws Exception If failed. + */ + private void setNodeSecuritySubject(ZookeeperClusterNode node, byte[] zipBytes) throws Exception { + assert zipBytes != null; + + Map attrs = new HashMap<>(node.getAttributes()); + + attrs.put(ATTR_SECURITY_SUBJECT_V2, unzip(zipBytes)); + + node.setAttributes(attrs); + } + + /** + * @param node Node. + * @return Credentials. + * @throws IgniteCheckedException If failed to unmarshal. + */ + private SecurityCredentials unmarshalCredentials(ZookeeperClusterNode node) throws Exception { + byte[] credBytes = (byte[])node.getAttributes().get(ATTR_SECURITY_CREDENTIALS); + + if (credBytes == null) + return null; + + return unmarshalZip(credBytes); + } + + /** + * Marshalls credentials with discovery SPI marshaller (will replace attribute value). + * + * @param node Node to marshall credentials for. + * @throws IgniteSpiException If marshalling failed. + */ + private void marshalCredentialsOnJoin(ZookeeperClusterNode node) throws IgniteSpiException { + try { + // Use security-unsafe getter. + Map attrs0 = node.getAttributes(); + + Object creds = attrs0.get(ATTR_SECURITY_CREDENTIALS); + + if (creds != null) { + Map attrs = new HashMap<>(attrs0); + + assert !(creds instanceof byte[]); + + attrs.put(ATTR_SECURITY_CREDENTIALS, marshalZip(creds)); + + node.setAttributes(attrs); + } + } + catch (IgniteCheckedException e) { + throw new IgniteSpiException("Failed to marshal node security credentials: " + node.id(), e); + } + } + + /** + * + */ + private class UpdateProcessedEventsTimeoutObject extends ZkTimeoutObject { + /** */ + private final ZkRuntimeState rtState; + + /** + * @param rtState Runtime state. + * @param timeout Timeout. + */ + UpdateProcessedEventsTimeoutObject(ZkRuntimeState rtState, long timeout) { + super(timeout); + + this.rtState = rtState; + } + + /** {@inheritDoc} */ + @Override public void onTimeout() { + runInWorkerThread(new ZkRunnable(rtState, ZookeeperDiscoveryImpl.this) { + @Override protected void run0() throws Exception { + updateProcessedEventsOnTimeout(rtState, UpdateProcessedEventsTimeoutObject.this); + } + }); + } + } + + /** + * + */ + private class JoinTimeoutObject extends ZkTimeoutObject { + /** + * @param timeout Timeout. + */ + JoinTimeoutObject(long timeout) { + super(timeout); + } + + /** {@inheritDoc} */ + @Override public void onTimeout() { + if (cancelled || rtState.joined) + return; + + runInWorkerThread(new Runnable() { + @Override public void run() { + synchronized (stateMux) { + if (cancelled || rtState.joined) + return; + + if (connState == ConnectionState.STOPPED) + return; + + connState = ConnectionState.STOPPED; + } + + U.warn(log, "Failed to connect to cluster, either connection to ZooKeeper can not be established or there " + + "are no alive server nodes (consider increasing 'joinTimeout' configuration property) [" + + "joinTimeout=" + spi.getJoinTimeout() + ']'); + + // Note: exception message is checked in tests. + onSegmented(new IgniteSpiException("Failed to connect to cluster within configured timeout")); + } + }); + } + } + + /** + * + */ + private class CheckJoinErrorWatcher extends ZkAbstractWatcher implements AsyncCallback.DataCallback { + /** */ + private final String joinDataPath; + + /** */ + private ZkTimeoutObject timeoutObj; + + /** + * @param timeout Timeout. + * @param joinDataPath0 Node joined data path. + * @param rtState0 State. + */ + CheckJoinErrorWatcher(long timeout, String joinDataPath0, ZkRuntimeState rtState0) { + super(rtState0, ZookeeperDiscoveryImpl.this); + + this.joinDataPath = joinDataPath0; + + timeoutObj = new ZkTimeoutObject(timeout) { + @Override public void onTimeout() { + if (rtState.errForClose != null || rtState.joined) + return; + + synchronized (stateMux) { + if (connState != ConnectionState.STARTED) + return; + } + + rtState.zkClient.getDataAsync(joinDataPath, + CheckJoinErrorWatcher.this, + CheckJoinErrorWatcher.this); + } + }; + } + + /** {@inheritDoc} */ + @Override public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) { + if (rc != 0) + return; + + if (!onProcessStart()) + return; + + try { + Object obj = unmarshalZip(data); + + if (obj instanceof ZkInternalJoinErrorMessage) { + ZkInternalJoinErrorMessage joinErr = (ZkInternalJoinErrorMessage)obj; + + onSegmented(new IgniteSpiException(joinErr.err)); + } + + onProcessEnd(); + } + catch (Throwable e) { + onProcessError(e); + } + } + + /** {@inheritDoc} */ + @Override public void process0(WatchedEvent evt) { + if (rtState.errForClose != null || rtState.joined) + return; + + if (evt.getType() == Event.EventType.NodeDataChanged) + rtState.zkClient.getDataAsync(evt.getPath(), this, this); + } + } + + /** + * @param aliveNodes Alive nodes. + * @throws Exception If failed. + */ + private void checkIsCoordinator(final List aliveNodes) throws Exception { + assert !locNode.isClient(); + + TreeMap aliveSrvs = new TreeMap<>(); + + long locInternalOrder = rtState.internalOrder; + + for (String aliveNodePath : aliveNodes) { + if (ZkIgnitePaths.aliveNodeClientFlag(aliveNodePath)) + continue; + + Long internalId = ZkIgnitePaths.aliveInternalId(aliveNodePath); + + aliveSrvs.put(internalId, aliveNodePath); + } + + assert !aliveSrvs.isEmpty(); + + Map.Entry crdE = aliveSrvs.firstEntry(); + + if (locInternalOrder == crdE.getKey()) + onBecomeCoordinator(aliveNodes); + else { + assert aliveSrvs.size() > 1 : aliveSrvs; + + Map.Entry prevE = aliveSrvs.floorEntry(locInternalOrder - 1); + + assert prevE != null; + + if (log.isInfoEnabled()) { + log.info("Discovery coordinator already exists, watch for previous server node [" + + "locId=" + locNode.id() + + ", watchPath=" + prevE.getValue() + ']'); + } + + PreviousNodeWatcher watcher = new ServerPreviousNodeWatcher(rtState); + + rtState.zkClient.existsAsync(zkPaths.aliveNodesDir + "/" + prevE.getValue(), watcher, watcher); + } + } + + /** + * @param aliveNodes Alive nodes. + * @throws Exception If failed. + */ + private void checkClientsStatus(final List aliveNodes) throws Exception { + assert locNode.isClient() : locNode; + assert rtState.joined; + assert rtState.evtsData != null; + + TreeMap aliveClients = new TreeMap<>(); + + String srvPath = null; + Long srvInternalOrder = null; + + long locInternalOrder = rtState.internalOrder; + + for (String aliveNodePath : aliveNodes) { + Long internalId = ZkIgnitePaths.aliveInternalId(aliveNodePath); + + if (ZkIgnitePaths.aliveNodeClientFlag(aliveNodePath)) + aliveClients.put(internalId, aliveNodePath); + else { + if (srvInternalOrder == null || internalId < srvInternalOrder) { + srvPath = aliveNodePath; + srvInternalOrder = internalId; + } + } + } + + assert !aliveClients.isEmpty(); + + Map.Entry oldest = aliveClients.firstEntry(); + + boolean oldestClient = locInternalOrder == oldest.getKey(); + + if (srvPath == null) { + if (oldestClient) { + Stat stat = new Stat(); + + ZkDiscoveryEventsData prevEvts = rtState.evtsData; + + byte[] evtsBytes = rtState.zkClient.getData(zkPaths.evtsPath, stat); + + assert evtsBytes.length > 0; + + ZkDiscoveryEventsData newEvts = unmarshalZip(evtsBytes); + + if (prevEvts.clusterId.equals(newEvts.clusterId)) { + U.warn(log, "All server nodes failed, notify all clients [locId=" + locNode.id() + ']'); + + generateNoServersEvent(newEvts, stat); + } + else + U.warn(log, "All server nodes failed (received events from new cluster)."); + } + } + else { + String watchPath; + + if (oldestClient) { + watchPath = srvPath; + + if (log.isInfoEnabled()) { + log.info("Servers exists, watch for server node [locId=" + locNode.id() + + ", watchPath=" + watchPath + ']'); + } + } + else { + assert aliveClients.size() > 1 : aliveClients; + + Map.Entry prevE = aliveClients.floorEntry(locInternalOrder - 1); + + assert prevE != null; + + watchPath = prevE.getValue(); + + if (log.isInfoEnabled()) { + log.info("Servers exists, watch for previous node [locId=" + locNode.id() + + ", watchPath=" + watchPath + ']'); + } + } + + PreviousNodeWatcher watcher = new ClientPreviousNodeWatcher(rtState); + + rtState.zkClient.existsAsync(zkPaths.aliveNodesDir + "/" + watchPath, watcher, watcher); + } + } + + /** + * @param evtsData Events data. + * @param evtsStat Events zookeeper state. + * @throws Exception If failed. + */ + private void generateNoServersEvent(ZkDiscoveryEventsData evtsData, Stat evtsStat) throws Exception { + evtsData.evtIdGen++; + + ZkDiscoveryCustomEventData evtData = new ZkDiscoveryCustomEventData( + evtsData.evtIdGen, + 0L, + evtsData.topVer, + locNode.id(), + new ZkNoServersMessage(), + null); + + Collection nodesToAck = Collections.emptyList(); + + evtsData.addEvent(nodesToAck, evtData); + + byte[] newEvtsBytes = marshalZip(evtsData); + + try { + rtState.zkClient.setData(zkPaths.evtsPath, newEvtsBytes, evtsStat.getVersion()); + } + catch (KeeperException.BadVersionException e) { + // Version can change if new cluster started and saved new events. + if (log.isDebugEnabled()) + log.debug("Failed to save no servers message"); + } + } + + /** + * @param lastEvts Last events from previous coordinator. + * @throws Exception If failed. + */ + private void previousCoordinatorCleanup(ZkDiscoveryEventsData lastEvts) throws Exception { + for (ZkDiscoveryEventData evtData : lastEvts.evts.values()) { + if (evtData instanceof ZkDiscoveryCustomEventData) { + ZkDiscoveryCustomEventData evtData0 = (ZkDiscoveryCustomEventData)evtData; + + // It is possible previous coordinator failed before finished cleanup. + if (evtData0.msg instanceof ZkCommunicationErrorResolveFinishMessage) { + try { + ZkCommunicationErrorResolveFinishMessage msg = + (ZkCommunicationErrorResolveFinishMessage)evtData0.msg; + + ZkCommunicationErrorResolveResult res = unmarshalZip( + ZkDistributedCollectDataFuture.readResult(rtState.zkClient, zkPaths, msg.futId)); + + deleteAliveNodes(res.killedNodes); + } + catch (KeeperException.NoNodeException ignore) { + // No-op. + } + } + else if (evtData0.resolvedMsg instanceof ZkForceNodeFailMessage) + deleteAliveNode(((ZkForceNodeFailMessage)evtData0.resolvedMsg).nodeInternalId); + } + } + } + + /** + * @param aliveNodes Alive nodes paths. + * @throws Exception If failed. + */ + private void onBecomeCoordinator(List aliveNodes) throws Exception { + ZkDiscoveryEventsData prevEvts = processNewEvents(rtState.zkClient.getData(zkPaths.evtsPath)); + + rtState.crd = true; + + if (rtState.joined) { + if (log.isInfoEnabled()) + log.info("Node is new discovery coordinator [locId=" + locNode.id() + ']'); + + assert locNode.order() > 0 : locNode; + assert rtState.evtsData != null; + + previousCoordinatorCleanup(rtState.evtsData); + + UUID futId = rtState.evtsData.communicationErrorResolveFutureId(); + + if (futId != null) { + if (log.isInfoEnabled()) { + log.info("New discovery coordinator will handle already started cluster-wide communication " + + "error resolve [reqId=" + futId + ']'); + } + + ZkCommunicationErrorProcessFuture fut = commErrProcFut.get(); + + ZkDistributedCollectDataFuture collectResFut = collectCommunicationStatusFuture(futId); + + if (fut != null) + fut.nodeResultCollectFuture(collectResFut); + } + + for (ZkDiscoveryEventData evtData : rtState.evtsData.evts.values()) + evtData.initRemainingAcks(rtState.top.nodesByOrder.values()); + + handleProcessedEvents("crd"); + } + else { + String locAlivePath = rtState.locNodeZkPath.substring(rtState.locNodeZkPath.lastIndexOf('/') + 1); + + deleteJoiningNodeData(locNode.id(), + ZkIgnitePaths.aliveNodePrefixId(locAlivePath), + rtState.joinDataPartCnt); + + DiscoverySpiNodeAuthenticator nodeAuth = spi.getAuthenticator(); + + if (nodeAuth != null) { + try { + if (log.isInfoEnabled()) { + log.info("Node is first server node in cluster, try authenticate local node " + + "[locId=" + locNode.id() + ']'); + } + + localAuthentication(nodeAuth, unmarshalCredentials(locNode)); + } + catch (Exception e) { + U.warn(log, "Local node authentication failed: " + e, e); + + onSegmented(e); + + // Stop any further processing. + throw new ZookeeperClientFailedException("Local node authentication failed: " + e); + } + } + + newClusterStarted(prevEvts); + } + + rtState.zkClient.getChildrenAsync(zkPaths.aliveNodesDir, rtState.watcher, rtState.watcher); + rtState.zkClient.getChildrenAsync(zkPaths.customEvtsDir, rtState.watcher, rtState.watcher); + + for (String alivePath : aliveNodes) + watchAliveNodeData(alivePath); + } + + /** + * @param alivePath Node path. + */ + private void watchAliveNodeData(String alivePath) { + assert rtState.locNodeZkPath != null; + + String path = zkPaths.aliveNodesDir + "/" + alivePath; + + if (!path.equals(rtState.locNodeZkPath)) + rtState.zkClient.getDataAsync(path, rtState.aliveNodeDataWatcher, rtState.aliveNodeDataWatcher); + } + + /** + * @param aliveNodes ZK nodes representing alive cluster nodes. + * @throws Exception If failed. + */ + private void generateTopologyEvents(List aliveNodes) throws Exception { + assert rtState.crd; + + if (log.isInfoEnabled()) + log.info("Process alive nodes change [alives=" + aliveNodes.size() + "]"); + + if (rtState.updateAlives) { + aliveNodes = rtState.zkClient.getChildren(zkPaths.aliveNodesDir); + + rtState.updateAlives = false; + } + + TreeMap alives = new TreeMap<>(); + + for (String child : aliveNodes) { + Long internalId = ZkIgnitePaths.aliveInternalId(child); + + Object old = alives.put(internalId, child); + + assert old == null; + } + + TreeMap curTop = new TreeMap<>(rtState.top.nodesByOrder); + + int newEvts = 0; + + final int MAX_NEW_EVTS = IgniteSystemProperties.getInteger(IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS, 100); + + List failedNodes = null; + + for (Map.Entry e : rtState.top.nodesByInternalId.entrySet()) { + if (!alives.containsKey(e.getKey())) { + ZookeeperClusterNode failedNode = e.getValue(); + + if (failedNodes == null) + failedNodes = new ArrayList<>(); + + failedNodes.add(failedNode); + + generateNodeFail(curTop, failedNode); + + newEvts++; + + if (newEvts == MAX_NEW_EVTS) { + saveAndProcessNewEvents(); + + if (log.isInfoEnabled()) { + log.info("Delay alive nodes change process, max event threshold reached [newEvts=" + newEvts + + ", totalEvts=" + rtState.evtsData.evts.size() + ']'); + } + + handleProcessedEventsOnNodesFail(failedNodes); + + throttleNewEventsGeneration(); + + rtState.zkClient.getChildrenAsync(zkPaths.aliveNodesDir, rtState.watcher, rtState.watcher); + + return; + } + } + } + + // Process failures before processing join, otherwise conflicts are possible in case of fast node stop/re-start. + if (newEvts > 0) { + saveAndProcessNewEvents(); + + handleProcessedEventsOnNodesFail(failedNodes); + + rtState.zkClient.getChildrenAsync(zkPaths.aliveNodesDir, rtState.watcher, rtState.watcher); + + return; + } + + generateJoinEvents(curTop, alives, MAX_NEW_EVTS); + + if (failedNodes != null) + handleProcessedEventsOnNodesFail(failedNodes); + } + + /** + * @param curTop Current topology. + * @param alives Alive znodes. + * @param MAX_NEW_EVTS Max event to process. + * @throws Exception If failed. + */ + private void generateJoinEvents(TreeMap curTop, + TreeMap alives, + final int MAX_NEW_EVTS) throws Exception + { + ZkBulkJoinContext joinCtx = new ZkBulkJoinContext(); + + for (Map.Entry e : alives.entrySet()) { + Long internalId = e.getKey(); + + if (!rtState.top.nodesByInternalId.containsKey(internalId)) { + UUID rslvFutId = rtState.evtsData.communicationErrorResolveFutureId(); + + if (rslvFutId != null) { + if (log.isInfoEnabled()) { + log.info("Delay alive nodes change process while communication error resolve " + + "is in progress [reqId=" + rslvFutId + ']'); + } + + break; + } + + processJoinOnCoordinator(joinCtx, curTop, internalId, e.getValue()); + + if (joinCtx.nodes() == MAX_NEW_EVTS) { + generateBulkJoinEvent(curTop, joinCtx); + + if (log.isInfoEnabled()) { + log.info("Delay alive nodes change process, max event threshold reached [" + + "newEvts=" + joinCtx.nodes() + + ", totalEvts=" + rtState.evtsData.evts.size() + ']'); + } + + throttleNewEventsGeneration(); + + rtState.zkClient.getChildrenAsync(zkPaths.aliveNodesDir, rtState.watcher, rtState.watcher); + + return; + } + } + } + + if (joinCtx.nodes() > 0) + generateBulkJoinEvent(curTop, joinCtx); + } + + /** + * @param curTop Current topology. + * @param joinCtx Joined nodes context. + * @throws Exception If failed. + */ + private void generateBulkJoinEvent(TreeMap curTop, ZkBulkJoinContext joinCtx) + throws Exception + { + rtState.evtsData.evtIdGen++; + + long evtId = rtState.evtsData.evtIdGen; + + List>> nodes = joinCtx.nodes; + + assert nodes != null && nodes.size() > 0; + + int nodeCnt = nodes.size(); + + List joinedNodes = new ArrayList<>(nodeCnt); + + Map discoDataMap = U.newHashMap(nodeCnt); + Map dupDiscoData = null; + + for (int i = 0; i < nodeCnt; i++) { + T2> nodeEvtData = nodes.get(i); + + Map discoData = nodeEvtData.get2(); + + byte[] discoDataBytes = U.marshal(marsh, discoData); + + Long dupDataNode = null; + + for (Map.Entry e : discoDataMap.entrySet()) { + if (Arrays.equals(discoDataBytes, e.getValue())) { + dupDataNode = e.getKey(); + + break; + } + } + + long nodeTopVer = nodeEvtData.get1().topVer; + + if (dupDataNode != null) { + if (dupDiscoData == null) + dupDiscoData = new HashMap<>(); + + Long old = dupDiscoData.put(nodeTopVer, dupDataNode); + + assert old == null : old; + } + else + discoDataMap.put(nodeTopVer, discoDataBytes); + + joinedNodes.add(nodeEvtData.get1()); + } + + int overhead = 5; + + ZkJoinEventDataForJoined dataForJoined = new ZkJoinEventDataForJoined( + new ArrayList<>(curTop.values()), + discoDataMap, + dupDiscoData); + + byte[] dataForJoinedBytes = marshalZip(dataForJoined); + + long addDataStart = System.currentTimeMillis(); + + int dataForJoinedPartCnt = saveData(zkPaths.joinEventDataPathForJoined(evtId), + dataForJoinedBytes, + overhead); + + long addDataTime = System.currentTimeMillis() - addDataStart; + + ZkDiscoveryNodeJoinEventData evtData = new ZkDiscoveryNodeJoinEventData( + evtId, + rtState.evtsData.topVer, + joinedNodes, + dataForJoinedPartCnt); + + rtState.evtsData.addEvent(curTop.values(), evtData); + + if (log.isInfoEnabled()) { + if (nodeCnt > 1) { + log.info("Generated NODE_JOINED bulk event [" + + "nodeCnt=" + nodeCnt + + ", dataForJoinedSize=" + dataForJoinedBytes.length + + ", dataForJoinedPartCnt=" + dataForJoinedPartCnt + + ", addDataTime=" + addDataTime + + ", evt=" + evtData + ']'); + } + else { + log.info("Generated NODE_JOINED event [" + + "dataForJoinedSize=" + dataForJoinedBytes.length + + ", dataForJoinedPartCnt=" + dataForJoinedPartCnt + + ", addDataTime=" + addDataTime + + ", evt=" + evtData + ']'); + } + } + + saveAndProcessNewEvents(); + } + + /** + * + */ + private void throttleNewEventsGeneration() { + long delay = IgniteSystemProperties.getLong(IGNITE_ZOOKEEPER_DISCOVERY_SPI_EVTS_THROTTLE, 0); + + if (delay > 0) { + if (log.isInfoEnabled()) + log.info("Sleep delay before generate new events [delay=" + delay + ']'); + + try { + Thread.sleep(delay); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } + + /** + * @param nodeId Node ID. + * @param prefixId Path prefix. + * @return Join data. + * @throws Exception If failed. + */ + private ZkJoiningNodeData unmarshalJoinData(UUID nodeId, UUID prefixId) throws Exception { + String joinDataPath = zkPaths.joiningNodeDataPath(nodeId, prefixId); + + byte[] joinData = rtState.zkClient.getData(joinDataPath); + + Object dataObj = unmarshalZip(joinData); + + if (!(dataObj instanceof ZkJoiningNodeData)) + throw new Exception("Invalid joined node data: " + dataObj); + + ZkJoiningNodeData joiningNodeData = (ZkJoiningNodeData)dataObj; + + if (joiningNodeData.partCount() > 1) { + joinData = readMultipleParts(rtState.zkClient, joinDataPath + ":", joiningNodeData.partCount()); + + joiningNodeData = unmarshalZip(joinData); + } + + return joiningNodeData; + } + + /** + * @param nodeId Node ID. + * @param prefixId Path prefix. + * @param aliveNodePath Node path. + * @return Join data. + * @throws Exception If failed. + */ + private Object unmarshalJoinDataOnCoordinator(UUID nodeId, UUID prefixId, String aliveNodePath) throws Exception { + String joinDataPath = zkPaths.joiningNodeDataPath(nodeId, prefixId); + + byte[] joinData = rtState.zkClient.getData(joinDataPath); + + Object dataObj; + + try { + dataObj = unmarshalZip(joinData); + + if (dataObj instanceof ZkInternalJoinErrorMessage) + return dataObj; + } + catch (Exception e) { + U.error(log, "Failed to unmarshal joining node data [nodePath=" + aliveNodePath + "']", e); + + return new ZkInternalJoinErrorMessage(ZkIgnitePaths.aliveInternalId(aliveNodePath), + "Failed to unmarshal join data: " + e); + } + + assert dataObj instanceof ZkJoiningNodeData : dataObj; + + ZkJoiningNodeData joiningNodeData = (ZkJoiningNodeData)dataObj; + + if (joiningNodeData.partCount() > 1) { + joinData = readMultipleParts(rtState.zkClient, joinDataPath + ":", joiningNodeData.partCount()); + + try { + joiningNodeData = unmarshalZip(joinData); + } + catch (Exception e) { + U.error(log, "Failed to unmarshal joining node data [nodePath=" + aliveNodePath + "']", e); + + return new ZkInternalJoinErrorMessage(ZkIgnitePaths.aliveInternalId(aliveNodePath), + "Failed to unmarshal join data: " + e); + } + } + + assert joiningNodeData.node() != null : joiningNodeData; + + return joiningNodeData; + } + + /** + * @param joinCtx Joined nodes context. + * @param curTop Current nodes. + * @param internalId Joined node internal ID. + * @param aliveNodePath Joined node path. + * @throws Exception If failed. + */ + private void processJoinOnCoordinator( + ZkBulkJoinContext joinCtx, + TreeMap curTop, + long internalId, + String aliveNodePath) + throws Exception + { + UUID nodeId = ZkIgnitePaths.aliveNodeId(aliveNodePath); + UUID prefixId = ZkIgnitePaths.aliveNodePrefixId(aliveNodePath); + + Object data = unmarshalJoinDataOnCoordinator(nodeId, prefixId, aliveNodePath); + + if (data instanceof ZkJoiningNodeData) { + ZkJoiningNodeData joiningNodeData = (ZkJoiningNodeData)data; + + ZkNodeValidateResult validateRes = validateJoiningNode(joiningNodeData.node()); + + if (validateRes.err == null) { + ZookeeperClusterNode joinedNode = joiningNodeData.node(); + + assert nodeId.equals(joinedNode.id()) : joiningNodeData.node(); + + addJoinedNode( + joinCtx, + curTop, + joiningNodeData, + internalId, + prefixId, + validateRes.secSubjZipBytes); + + watchAliveNodeData(aliveNodePath); + } + else { + ZkInternalJoinErrorMessage joinErr = new ZkInternalJoinErrorMessage( + ZkIgnitePaths.aliveInternalId(aliveNodePath), + validateRes.err); + + processJoinError(aliveNodePath, nodeId, prefixId, joinErr); + } + } + else { + assert data instanceof ZkInternalJoinErrorMessage : data; + + processJoinError(aliveNodePath, nodeId, prefixId, (ZkInternalJoinErrorMessage)data); + } + } + + /** + * @param aliveNodePath Joined node path. + * @param nodeId Node ID. + * @param prefixId Path prefix ID. + * @param joinErr Join error message. + * @throws Exception If failed. + */ + private void processJoinError(String aliveNodePath, + UUID nodeId, + UUID prefixId, + ZkInternalJoinErrorMessage joinErr) throws Exception { + ZookeeperClient client = rtState.zkClient; + + if (joinErr.notifyNode) { + String joinDataPath = zkPaths.joiningNodeDataPath(nodeId, prefixId); + + client.setData(joinDataPath, marshalZip(joinErr), -1); + + client.deleteIfExists(zkPaths.aliveNodesDir + "/" + aliveNodePath, -1); + } + else { + if (log.isInfoEnabled()) + log.info("Ignore join data, node was failed by previous coordinator: " + aliveNodePath); + + client.deleteIfExists(zkPaths.aliveNodesDir + "/" + aliveNodePath, -1); + } + } + + /** + * @param node Joining node. + * @return Validation result. + */ + private ZkNodeValidateResult validateJoiningNode(ZookeeperClusterNode node) { + ZookeeperClusterNode node0 = rtState.top.nodesById.get(node.id()); + + if (node0 != null) { + U.error(log, "Failed to include node in cluster, node with the same ID already exists [joiningNode=" + node + + ", existingNode=" + node0 + ']'); + + // Note: exception message is checked in tests. + return new ZkNodeValidateResult("Node with the same ID already exists: " + node0); + } + + ZkNodeValidateResult res = authenticateNode(node); + + if (res.err != null) + return res; + + IgniteNodeValidationResult err = spi.getSpiContext().validateNode(node); + + if (err != null) { + LT.warn(log, err.message()); + + res.err = err.sendMessage(); + } + + return res; + } + + /** + * @param node Node. + * @return Validation result. + */ + private ZkNodeValidateResult authenticateNode(ZookeeperClusterNode node) { + DiscoverySpiNodeAuthenticator nodeAuth = spi.getAuthenticator(); + + if (nodeAuth == null) + return new ZkNodeValidateResult((byte[])null); + + SecurityCredentials cred; + + try { + cred = unmarshalCredentials(node); + } + catch (Exception e) { + U.error(log, "Failed to unmarshal node credentials: " + e, e); + + return new ZkNodeValidateResult("Failed to unmarshal node credentials"); + } + + SecurityContext subj = nodeAuth.authenticateNode(node, cred); + + if (subj == null) { + U.warn(log, "Authentication failed [nodeId=" + node.id() + + ", addrs=" + U.addressesAsString(node) + ']', + "Authentication failed [nodeId=" + U.id8(node.id()) + ", addrs=" + + U.addressesAsString(node) + ']'); + + // Note: exception message test is checked in tests. + return new ZkNodeValidateResult("Authentication failed"); + } + + if (!(subj instanceof Serializable)) { + U.warn(log, "Authentication subject is not Serializable [nodeId=" + node.id() + + ", addrs=" + U.addressesAsString(node) + ']', + "Authentication subject is not Serializable [nodeId=" + U.id8(node.id()) + + ", addrs=" + + U.addressesAsString(node) + ']'); + + return new ZkNodeValidateResult("Authentication subject is not serializable"); + } + + byte[] secSubjZipBytes; + + try { + secSubjZipBytes = marshalZip(subj); + } + catch (Exception e) { + U.error(log, "Failed to marshal node security subject: " + e, e); + + return new ZkNodeValidateResult("Failed to marshal node security subject"); + } + + return new ZkNodeValidateResult(secSubjZipBytes); + } + + /** + * @throws Exception If failed. + */ + private void saveAndProcessNewEvents() throws Exception { + if (stopping()) + return; + + long start = System.currentTimeMillis(); + + byte[] evtsBytes = marshalZip(rtState.evtsData); + + rtState.zkClient.setData(zkPaths.evtsPath, evtsBytes, -1); + + long time = System.currentTimeMillis() - start; + + if (prevSavedEvtsTopVer != rtState.evtsData.topVer) { + if (log.isInfoEnabled()) { + log.info("Discovery coordinator saved new topology events [topVer=" + rtState.evtsData.topVer + + ", size=" + evtsBytes.length + + ", evts=" + rtState.evtsData.evts.size() + + ", lastEvt=" + rtState.evtsData.evtIdGen + + ", saveTime=" + time + ']'); + } + + prevSavedEvtsTopVer = rtState.evtsData.topVer; + } + else if (log.isDebugEnabled()) { + log.debug("Discovery coordinator saved new topology events [topVer=" + rtState.evtsData.topVer + + ", size=" + evtsBytes.length + + ", evts=" + rtState.evtsData.evts.size() + + ", lastEvt=" + rtState.evtsData.evtIdGen + + ", saveTime=" + time + ']'); + } + + processNewEvents(rtState.evtsData); + } + + /** + * @param curTop Current topology. + * @param failedNode Failed node. + */ + private void generateNodeFail(TreeMap curTop, ZookeeperClusterNode failedNode) { + Object rmvd = curTop.remove(failedNode.order()); + + assert rmvd != null; + + rtState.evtsData.topVer++; + rtState.evtsData.evtIdGen++; + + ZkDiscoveryNodeFailEventData evtData = new ZkDiscoveryNodeFailEventData( + rtState.evtsData.evtIdGen, + rtState.evtsData.topVer, + failedNode.internalId()); + + rtState.evtsData.addEvent(curTop.values(), evtData); + + if (log.isInfoEnabled()) + log.info("Generated NODE_FAILED event [evt=" + evtData + ']'); + } + + /** + * @param curTop Current nodes. + * @param joiningNodeData Join data. + * @param internalId Joined node internal ID. + * @param prefixId Unique path prefix. + * @param secSubjZipBytes Marshalled security subject. + * @throws Exception If failed. + */ + private void addJoinedNode( + ZkBulkJoinContext joinCtx, + TreeMap curTop, + ZkJoiningNodeData joiningNodeData, + long internalId, + UUID prefixId, + @Nullable byte[] secSubjZipBytes) + throws Exception + { + ZookeeperClusterNode joinedNode = joiningNodeData.node(); + + UUID nodeId = joinedNode.id(); + + rtState.evtsData.topVer++; + + joinedNode.order(rtState.evtsData.topVer); + joinedNode.internalId(internalId); + + DiscoveryDataBag joiningNodeBag = new DiscoveryDataBag(nodeId, joiningNodeData.node().isClient()); + + joiningNodeBag.joiningNodeData(joiningNodeData.discoveryData()); + + exchange.onExchange(joiningNodeBag); + + DiscoveryDataBag collectBag = new DiscoveryDataBag(nodeId, + new HashSet(), + joiningNodeData.node().isClient()); + + collectBag.joiningNodeData(joiningNodeBag.joiningNodeData()); + + exchange.collect(collectBag); + + Map commonData = collectBag.commonData(); + + Object old = curTop.put(joinedNode.order(), joinedNode); + + assert old == null; + + int overhead = 5; + + int secSubjPartCnt = 0; + + if (secSubjZipBytes != null) { + secSubjPartCnt = saveData(zkPaths.joinEventSecuritySubjectPath(joinedNode.order()), + secSubjZipBytes, + overhead); + + assert secSubjPartCnt > 0 : secSubjPartCnt; + + setNodeSecuritySubject(joinedNode, secSubjZipBytes); + } + + ZkJoinedNodeEvtData nodeEvtData = new ZkJoinedNodeEvtData( + rtState.evtsData.topVer, + joinedNode.id(), + joinedNode.internalId(), + prefixId, + joiningNodeData.partCount(), + secSubjPartCnt); + + nodeEvtData.joiningNodeData = joiningNodeData; + + joinCtx.addJoinedNode(nodeEvtData, commonData); + + rtState.evtsData.onNodeJoin(joinedNode); + } + + /** + * @param path Path to save. + * @param bytes Bytes to save. + * @param overhead Extra overhead. + * @return Parts count. + * @throws Exception If failed. + */ + private int saveData(String path, byte[] bytes, int overhead) throws Exception { + int dataForJoinedPartCnt = 1; + + if (rtState.zkClient.needSplitNodeData(path, bytes, overhead)) { + dataForJoinedPartCnt = saveMultipleParts(rtState.zkClient, + path, + rtState.zkClient.splitNodeData(path, bytes, overhead)); + } + else { + rtState.zkClient.createIfNeeded(multipartPathName(path, 0), + bytes, + PERSISTENT); + } + + return dataForJoinedPartCnt; + } + + /** + * @param prevEvts Events from previous cluster. + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + private void newClusterStarted(@Nullable ZkDiscoveryEventsData prevEvts) throws Exception { + assert !locNode.isClient() : locNode; + + long locInternalId = rtState.internalOrder; + + assert prevEvts == null || prevEvts.maxInternalOrder < locInternalId; + + spi.getSpiContext().removeTimeoutObject(rtState.joinErrTo); + + cleanupPreviousClusterData(prevEvts != null ? prevEvts.maxInternalOrder + 1 : -1L); + + rtState.joined = true; + rtState.gridStartTime = System.currentTimeMillis(); + + rtState.evtsData = ZkDiscoveryEventsData.createForNewCluster(rtState.gridStartTime); + + if (log.isInfoEnabled()) { + log.info("New cluster started [locId=" + locNode.id() + + ", clusterId=" + rtState.evtsData.clusterId + + ", startTime=" + rtState.evtsData.clusterStartTime + ']'); + } + + locNode.internalId(locInternalId); + locNode.order(1); + + rtState.evtsData.onNodeJoin(locNode); + + rtState.top.addNode(locNode); + + final List topSnapshot = Collections.singletonList((ClusterNode)locNode); + + lsnr.onDiscovery(EVT_NODE_JOINED, + 1L, + locNode, + topSnapshot, + Collections.>emptyMap(), + null); + + // Reset events (this is also notification for clients left from previous cluster). + rtState.zkClient.setData(zkPaths.evtsPath, marshalZip(rtState.evtsData), -1); + + joinFut.onDone(); + } + + /** + * @param startInternalOrder Starting internal order for cluster (znodes having lower order belong + * to clients from previous cluster and should be removed). + + * @throws Exception If failed. + */ + private void cleanupPreviousClusterData(long startInternalOrder) throws Exception { + long start = System.currentTimeMillis(); + + ZookeeperClient client = rtState.zkClient; + + // TODO ZK: use multi, better batching + max-size safe + NoNodeException safe. + List evtChildren = rtState.zkClient.getChildren(zkPaths.evtsPath); + + for (String evtPath : evtChildren) { + String evtDir = zkPaths.evtsPath + "/" + evtPath; + + removeChildren(evtDir); + } + + client.deleteAll(zkPaths.evtsPath, evtChildren, -1); + + client.deleteAll(zkPaths.customEvtsDir, + client.getChildren(zkPaths.customEvtsDir), + -1); + + rtState.zkClient.deleteAll(zkPaths.customEvtsPartsDir, + rtState.zkClient.getChildren(zkPaths.customEvtsPartsDir), + -1); + + rtState.zkClient.deleteAll(zkPaths.customEvtsAcksDir, + rtState.zkClient.getChildren(zkPaths.customEvtsAcksDir), + -1); + + if (startInternalOrder > 0) { + for (String alive : rtState.zkClient.getChildren(zkPaths.aliveNodesDir)) { + if (ZkIgnitePaths.aliveInternalId(alive) < startInternalOrder) + rtState.zkClient.deleteIfExists(zkPaths.aliveNodesDir + "/" + alive, -1); + } + } + + long time = System.currentTimeMillis() - start; + + if (time > 0) { + if (log.isInfoEnabled()) + log.info("Previous cluster data cleanup time: " + time); + } + } + + /** + * @param path Path. + * @throws Exception If failed. + */ + private void removeChildren(String path) throws Exception { + rtState.zkClient.deleteAll(path, rtState.zkClient.getChildren(path), -1); + } + + /** + * @param zkClient Client. + * @param evtPath Event path. + * @param sndNodeId Sender node ID. + * @return Event data. + * @throws Exception If failed. + */ + private byte[] readCustomEventData(ZookeeperClient zkClient, String evtPath, UUID sndNodeId) throws Exception { + int partCnt = ZkIgnitePaths.customEventPartsCount(evtPath); + + if (partCnt > 1) { + String partsBasePath = zkPaths.customEventPartsBasePath( + ZkIgnitePaths.customEventPrefix(evtPath), sndNodeId); + + return readMultipleParts(zkClient, partsBasePath, partCnt); + } + else + return zkClient.getData(zkPaths.customEvtsDir + "/" + evtPath); + } + + /** + * @param customEvtNodes ZK nodes representing custom events to process. + * @throws Exception If failed. + */ + private void generateCustomEvents(List customEvtNodes) throws Exception { + assert rtState.crd; + + ZookeeperClient zkClient = rtState.zkClient; + ZkDiscoveryEventsData evtsData = rtState.evtsData; + + TreeMap unprocessedEvts = null; + + for (int i = 0; i < customEvtNodes.size(); i++) { + String evtPath = customEvtNodes.get(i); + + int evtSeq = ZkIgnitePaths.customEventSequence(evtPath); + + if (evtSeq > evtsData.procCustEvt) { + if (unprocessedEvts == null) + unprocessedEvts = new TreeMap<>(); + + unprocessedEvts.put(evtSeq, evtPath); + } + } + + if (unprocessedEvts == null) + return; + + for (Map.Entry evtE : unprocessedEvts.entrySet()) { + evtsData.procCustEvt = evtE.getKey(); + + String evtPath = evtE.getValue(); + + UUID sndNodeId = ZkIgnitePaths.customEventSendNodeId(evtPath); + + ZookeeperClusterNode sndNode = rtState.top.nodesById.get(sndNodeId); + + if (sndNode != null) { + byte[] evtBytes = readCustomEventData(zkClient, evtPath, sndNodeId); + + DiscoverySpiCustomMessage msg; + + try { + msg = unmarshalZip(evtBytes); + } + catch (Exception e) { + U.error(log, "Failed to unmarshal custom discovery message: " + e, e); + + deleteCustomEventDataAsync(rtState.zkClient, evtPath); + + continue; + } + + generateAndProcessCustomEventOnCoordinator(evtPath, sndNode, msg); + } + else { + U.warn(log, "Ignore custom event from unknown node: " + sndNodeId); + + deleteCustomEventDataAsync(rtState.zkClient, evtPath); + } + } + } + + /** + * @param evtPath Event data path. + * @param sndNode Sender node. + * @param msg Message instance. + * @throws Exception If failed. + */ + private void generateAndProcessCustomEventOnCoordinator(String evtPath, + ZookeeperClusterNode sndNode, + DiscoverySpiCustomMessage msg) throws Exception + { + ZookeeperClient zkClient = rtState.zkClient; + ZkDiscoveryEventsData evtsData = rtState.evtsData; + + ZookeeperClusterNode failedNode = null; + + if (msg instanceof ZkForceNodeFailMessage) { + ZkForceNodeFailMessage msg0 = (ZkForceNodeFailMessage)msg; + + failedNode = rtState.top.nodesByInternalId.get(msg0.nodeInternalId); + + if (failedNode != null) + evtsData.topVer++; + else { + if (log.isDebugEnabled()) + log.debug("Ignore forcible node fail request for unknown node: " + msg0.nodeInternalId); + + deleteCustomEventDataAsync(zkClient, evtPath); + + return; + } + } + else if (msg instanceof ZkCommunicationErrorResolveStartMessage) { + ZkCommunicationErrorResolveStartMessage msg0 = + (ZkCommunicationErrorResolveStartMessage)msg; + + if (evtsData.communicationErrorResolveFutureId() != null) { + if (log.isInfoEnabled()) { + log.info("Ignore communication error resolve message, resolve process " + + "already started [sndNode=" + sndNode + ']'); + } + + deleteCustomEventDataAsync(zkClient, evtPath); + + return; + } + else { + if (log.isInfoEnabled()) { + log.info("Start cluster-wide communication error resolve [sndNode=" + sndNode + + ", reqId=" + msg0.id + + ", topVer=" + evtsData.topVer + ']'); + } + + zkClient.createIfNeeded(zkPaths.distributedFutureBasePath(msg0.id), + null, + PERSISTENT); + + evtsData.communicationErrorResolveFutureId(msg0.id); + } + } + + evtsData.evtIdGen++; + + ZkDiscoveryCustomEventData evtData = new ZkDiscoveryCustomEventData( + evtsData.evtIdGen, + 0L, + evtsData.topVer, + sndNode.id(), + null, + evtPath); + + evtData.resolvedMsg = msg; + + if (log.isDebugEnabled()) + log.debug("Generated CUSTOM event [evt=" + evtData + ", msg=" + msg + ']'); + + boolean fastStopProcess = false; + + if (msg instanceof ZkInternalMessage) + processInternalMessage(evtData, (ZkInternalMessage)msg); + else { + notifyCustomEvent(evtData, msg); + + if (msg.stopProcess()) { + if (log.isDebugEnabled()) + log.debug("Fast stop process custom event [evt=" + evtData + ", msg=" + msg + ']'); + + fastStopProcess = true; + + // No need to process this event on others nodes, skip this event. + evtsData.evts.remove(evtData.eventId()); + + evtsData.evtIdGen--; + + DiscoverySpiCustomMessage ack = msg.ackMessage(); + + if (ack != null) { + evtData = createAckEvent(ack, evtData); + + if (log.isDebugEnabled()) + log.debug("Generated CUSTOM event (ack for fast stop process) [evt=" + evtData + ", msg=" + msg + ']'); + + notifyCustomEvent(evtData, ack); + } + else + evtData = null; + } + } + + if (evtData != null) { + evtsData.addEvent(rtState.top.nodesByOrder.values(), evtData); + + rtState.locNodeInfo.lastProcEvt = evtData.eventId(); + + saveAndProcessNewEvents(); + + if (fastStopProcess) + deleteCustomEventDataAsync(zkClient, evtPath); + + if (failedNode != null) { + deleteAliveNode(failedNode.internalId()); + + handleProcessedEventsOnNodesFail(Collections.singletonList(failedNode)); + + rtState.updateAlives = true; + } + } + } + + /** + * @param internalId Node internal ID. + * @throws Exception If failed. + */ + private void deleteAliveNode(long internalId) throws Exception { + for (String child : rtState.zkClient.getChildren(zkPaths.aliveNodesDir)) { + if (ZkIgnitePaths.aliveInternalId(child) == internalId) { + // Need use sync delete to do not process again join of this node again. + rtState.zkClient.deleteIfExists(zkPaths.aliveNodesDir + "/" + child, -1); + + return; + } + } + } + + /** + * @param zkClient Client. + * @param evtPath Event path. + */ + private void deleteCustomEventDataAsync(ZookeeperClient zkClient, String evtPath) { + if (log.isDebugEnabled()) + log.debug("Delete custom event data: " + evtPath); + + String prefix = ZkIgnitePaths.customEventPrefix(evtPath); + UUID sndNodeId = ZkIgnitePaths.customEventSendNodeId(evtPath); + int partCnt = ZkIgnitePaths.customEventPartsCount(evtPath); + + assert partCnt >= 1 : partCnt; + + if (partCnt > 1) { + for (int i = 0; i < partCnt; i++) { + String path0 = zkPaths.customEventPartPath(prefix, sndNodeId, i); + + zkClient.deleteIfExistsAsync(path0); + } + } + + zkClient.deleteIfExistsAsync(zkPaths.customEvtsDir + "/" + evtPath); + } + + /** + * @param data Marshalled events. + * @throws Exception If failed. + * @return Events. + */ + @Nullable private ZkDiscoveryEventsData processNewEvents(byte[] data) throws Exception { + ZkDiscoveryEventsData newEvts = data.length > 0 ? (ZkDiscoveryEventsData)unmarshalZip(data) : null; + + if (rtState.joined && (newEvts == null || !rtState.evtsData.clusterId.equals(newEvts.clusterId))) { + assert locNode.isClient() : locNode; + + throw localNodeFail("All server nodes failed, client node disconnected (received events from new custer) " + + "[locId=" + locNode.id() + ']', true); + } + + if (newEvts == null) + return null; + + assert !rtState.crd; + + // Need keep processed custom events since they contain message object which is needed to create ack. + if (!locNode.isClient() && rtState.evtsData != null) { + for (Map.Entry e : rtState.evtsData.evts.entrySet()) { + ZkDiscoveryEventData evtData = e.getValue(); + + if (evtData.eventType() == ZkDiscoveryEventData.ZK_EVT_CUSTOM_EVT) { + ZkDiscoveryCustomEventData evtData0 = + (ZkDiscoveryCustomEventData)newEvts.evts.get(evtData.eventId()); + + if (evtData0 != null) + evtData0.resolvedMsg = ((ZkDiscoveryCustomEventData)evtData).resolvedMsg; + } + } + } + + processNewEvents(newEvts); + + if (rtState.joined) + rtState.evtsData = newEvts; + + return newEvts; + } + + /** + * @param evtsData Events. + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + private void processNewEvents(final ZkDiscoveryEventsData evtsData) throws Exception { + TreeMap evts = evtsData.evts; + + ZookeeperClient zkClient = rtState.zkClient; + + boolean evtProcessed = false; + boolean updateNodeInfo = false; + + try { + for (ZkDiscoveryEventData evtData : evts.tailMap(rtState.locNodeInfo.lastProcEvt, false).values()) { + if (log.isDebugEnabled()) + log.debug("New discovery event data [evt=" + evtData + ", evtsHist=" + evts.size() + ']'); + + switch (evtData.eventType()) { + case ZkDiscoveryEventData.ZK_EVT_NODE_JOIN: { + evtProcessed = processBulkJoin(evtsData, (ZkDiscoveryNodeJoinEventData)evtData); + + break; + } + + case ZkDiscoveryEventData.ZK_EVT_NODE_FAILED: { + if (!rtState.joined) + break; + + evtProcessed = true; + + notifyNodeFail((ZkDiscoveryNodeFailEventData)evtData); + + break; + } + + case ZkDiscoveryEventData.ZK_EVT_CUSTOM_EVT: { + if (!rtState.joined) + break; + + evtProcessed = true; + + ZkDiscoveryCustomEventData evtData0 = (ZkDiscoveryCustomEventData)evtData; + + if (evtData0.ackEvent() && evtData0.topologyVersion() < locNode.order()) + break; + + DiscoverySpiCustomMessage msg; + + if (rtState.crd) { + assert evtData0.resolvedMsg != null : evtData0; + + msg = evtData0.resolvedMsg; + } + else { + if (evtData0.msg == null) { + if (evtData0.ackEvent()) { + String path = zkPaths.ackEventDataPath(evtData0.origEvtId); + + msg = unmarshalZip(zkClient.getData(path)); + } + else { + assert evtData0.evtPath != null : evtData0; + + byte[] msgBytes = readCustomEventData(zkClient, + evtData0.evtPath, + evtData0.sndNodeId); + + msg = unmarshalZip(msgBytes); + } + } + else + msg = evtData0.msg; + + evtData0.resolvedMsg = msg; + } + + if (msg instanceof ZkInternalMessage) + processInternalMessage(evtData0, (ZkInternalMessage)msg); + else { + notifyCustomEvent(evtData0, msg); + + if (!evtData0.ackEvent()) + updateNodeInfo = true; + } + + break; + } + + default: + assert false : "Invalid event: " + evtData; + } + + if (rtState.joined) { + rtState.locNodeInfo.lastProcEvt = evtData.eventId(); + + rtState.procEvtCnt++; + + if (rtState.procEvtCnt % evtsAckThreshold == 0) + updateNodeInfo = true; + } + } + } + catch (KeeperException.NoNodeException e) { + // Can get NoNodeException if local node was forcible failed, + // in this case coordinator does not wait when this node process all events. + boolean exists; + + try { + exists = rtState.zkClient.exists(rtState.locNodeZkPath); + } + catch (Exception e0) { + if (log.isDebugEnabled()) + log.debug("Failed to check is local node is alive:" + e0); + + exists = true; + } + + if (!exists) { + U.warn(log, "Failed to process discovery event, local node was forced to stop.", e); + + throw localNodeFail("Local node was forced to stop.", true); + } + + throw e; + } + + if (rtState.crd) + handleProcessedEvents("procEvt"); + else + onEventProcessed(rtState, updateNodeInfo, evtProcessed); + + ZkCommunicationErrorProcessFuture commErrFut = commErrProcFut.get(); + + if (commErrFut != null) + commErrFut.onTopologyChange(rtState.top); // This can add new event, notify out of event process loop. + } + + private boolean processBulkJoin(ZkDiscoveryEventsData evtsData, ZkDiscoveryNodeJoinEventData evtData) + throws Exception + { + boolean evtProcessed = false; + + for (int i = 0; i < evtData.joinedNodes.size(); i++) { + ZkJoinedNodeEvtData joinedEvtData = evtData.joinedNodes.get(i); + + if (!rtState.joined) { + UUID joinedId = joinedEvtData.nodeId; + + boolean locJoin = joinedEvtData.joinedInternalId == rtState.internalOrder; + + if (locJoin) { + assert locNode.id().equals(joinedId); + + processLocalJoin(evtsData, joinedEvtData, evtData); + + evtProcessed = true; + } + } + else { + ZkJoiningNodeData joiningData; + + if (rtState.crd) { + assert joinedEvtData.joiningNodeData != null; + + joiningData = joinedEvtData.joiningNodeData; + } + else { + joiningData = unmarshalJoinData(joinedEvtData.nodeId, joinedEvtData.joinDataPrefixId); + + DiscoveryDataBag dataBag = new DiscoveryDataBag(joinedEvtData.nodeId, joiningData.node().isClient()); + + dataBag.joiningNodeData(joiningData.discoveryData()); + + exchange.onExchange(dataBag); + } + + if (joinedEvtData.secSubjPartCnt > 0 && joiningData.node().attribute(ATTR_SECURITY_SUBJECT_V2) == null) + readAndInitSecuritySubject(joiningData.node(), joinedEvtData); + + notifyNodeJoin(joinedEvtData, joiningData); + } + } + + return evtProcessed; + } + + /** + * @param rtState Runtime state. + * @param updateNodeInfo {@code True} if need update processed events without delay. + * @param evtProcessed {@code True} if new event was processed. + * @throws Exception If failed. + */ + private void onEventProcessed(ZkRuntimeState rtState, + boolean updateNodeInfo, + boolean evtProcessed) throws Exception + { + synchronized (stateMux) { + if (updateNodeInfo) { + assert rtState.locNodeZkPath != null; + + if (log.isDebugEnabled()) + log.debug("Update processed events: " + rtState.locNodeInfo.lastProcEvt); + + updateProcessedEvents(rtState); + + if (rtState.procEvtsUpdateTo != null) { + spi.getSpiContext().removeTimeoutObject(rtState.procEvtsUpdateTo); + + rtState.procEvtsUpdateTo = null; + } + } + else if (evtProcessed) { + rtState.locNodeInfo.needUpdate = true; + + if (rtState.procEvtsUpdateTo == null) { + long updateTimeout = IgniteSystemProperties.getLong(IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_TIMEOUT, + 60_000); + + if (updateTimeout > 0) { + rtState.procEvtsUpdateTo = new UpdateProcessedEventsTimeoutObject(rtState, updateTimeout); + + spi.getSpiContext().addTimeoutObject(rtState.procEvtsUpdateTo); + } + } + } + } + } + + /** + * @param rtState Runtime state. + * @param procEvtsUpdateTo Timeout object. + * @throws Exception If failed. + */ + private void updateProcessedEventsOnTimeout(ZkRuntimeState rtState, ZkTimeoutObject procEvtsUpdateTo) + throws Exception + { + synchronized (stateMux) { + if (rtState.procEvtsUpdateTo == procEvtsUpdateTo && rtState.locNodeInfo.needUpdate) { + if (log.isDebugEnabled()) + log.debug("Update processed events on timeout: " + rtState.locNodeInfo.lastProcEvt); + + updateProcessedEvents(rtState); + } + } + } + + /** + * @param rtState Runtime state. + * @throws Exception If failed. + */ + private void updateProcessedEvents(ZkRuntimeState rtState) throws Exception { + try { + rtState.zkClient.setData(rtState.locNodeZkPath, marshalZip(rtState.locNodeInfo), -1); + + rtState.locNodeInfo.needUpdate = false; + } + catch (KeeperException.NoNodeException e) { + // Possible if node is forcible failed. + if (log.isDebugEnabled()) + log.debug("Failed to update processed events, no node: " + rtState.locNodeInfo.lastProcEvt); + } + } + + /** + * @param node Node. + * @param joinedEvtData Joined node information. + * @throws Exception If failed. + */ + private void readAndInitSecuritySubject(ZookeeperClusterNode node, ZkJoinedNodeEvtData joinedEvtData) throws Exception { + if (joinedEvtData.secSubjPartCnt > 0) { + byte[] zipBytes = readMultipleParts(rtState.zkClient, + zkPaths.joinEventSecuritySubjectPath(joinedEvtData.topVer), + joinedEvtData.secSubjPartCnt); + + setNodeSecuritySubject(node, zipBytes); + } + } + + /** + * @param evtsData Events data. + * @param evtData Local join event data. + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + private void processLocalJoin(ZkDiscoveryEventsData evtsData, + ZkJoinedNodeEvtData joinedEvtData, + ZkDiscoveryNodeJoinEventData evtData) + throws Exception + { + synchronized (stateMux) { + if (connState == ConnectionState.STOPPED) + return; + + if (rtState.joinTo != null) { + spi.getSpiContext().removeTimeoutObject(rtState.joinTo); + + rtState.joinTo.cancelled = true; + rtState.joinTo = null; + } + + spi.getSpiContext().removeTimeoutObject(rtState.joinErrTo); + + if (log.isInfoEnabled()) + log.info("Local join event data: " + joinedEvtData + ']'); + + String path = zkPaths.joinEventDataPathForJoined(evtData.eventId()); + + byte[] dataForJoinedBytes = readMultipleParts(rtState.zkClient, path, evtData.dataForJoinedPartCnt); + + ZkJoinEventDataForJoined dataForJoined = unmarshalZip(dataForJoinedBytes); + + rtState.gridStartTime = evtsData.clusterStartTime; + + locNode.internalId(joinedEvtData.joinedInternalId); + locNode.order(joinedEvtData.topVer); + + readAndInitSecuritySubject(locNode, joinedEvtData); + + byte[] discoDataBytes = dataForJoined.discoveryDataForNode(locNode.order()); + + Map commonDiscoData = + marsh.unmarshal(discoDataBytes, U.resolveClassLoader(spi.ignite().configuration())); + + DiscoveryDataBag dataBag = new DiscoveryDataBag(locNode.id(), locNode.isClient()); + + dataBag.commonData(commonDiscoData); + + exchange.onExchange(dataBag); + + List allNodes = dataForJoined.topology(); + + for (int i = 0; i < allNodes.size(); i++) { + ZookeeperClusterNode node = allNodes.get(i); + + // Need filter since ZkJoinEventDataForJoined contains single topology snapshot for all joined nodes. + if (node.order() >= locNode.order()) + break; + + node.setMetrics(new ClusterMetricsSnapshot()); + + rtState.top.addNode(node); + } + + rtState.top.addNode(locNode); + + final List topSnapshot = rtState.top.topologySnapshot(); + + lsnr.onDiscovery(EVT_NODE_JOINED, + joinedEvtData.topVer, + locNode, + topSnapshot, + Collections.>emptyMap(), + null); + + if (rtState.prevJoined) { + lsnr.onDiscovery(EVT_CLIENT_NODE_RECONNECTED, + joinedEvtData.topVer, + locNode, + topSnapshot, + Collections.>emptyMap(), + null); + + U.quietAndWarn(log, "Client node was reconnected after it was already considered failed [locId=" + locNode.id() + ']'); + } + + rtState.joined = true; + } + + joinFut.onDone(); + + if (locNode.isClient()) + rtState.zkClient.getChildrenAsync(zkPaths.aliveNodesDir, null, new CheckClientsStatusCallback(rtState)); + } + + /** + * @param evtData Event daa. + * @param msg Message. + * @throws Exception If failed. + */ + private void processInternalMessage(ZkDiscoveryCustomEventData evtData, ZkInternalMessage msg) throws Exception { + if (msg instanceof ZkForceNodeFailMessage) + processForceNodeFailMessage((ZkForceNodeFailMessage)msg, evtData); + else if (msg instanceof ZkCommunicationErrorResolveStartMessage) { + processCommunicationErrorResolveStartMessage( + (ZkCommunicationErrorResolveStartMessage)msg, + evtData); + } + else if (msg instanceof ZkCommunicationErrorResolveFinishMessage) { + processCommunicationErrorResolveFinishMessage( + (ZkCommunicationErrorResolveFinishMessage)msg); + } + else if (msg instanceof ZkNoServersMessage) + processNoServersMessage((ZkNoServersMessage)msg); + } + + /** + * @param msg Message. + * @throws Exception If failed. + */ + private void processNoServersMessage(ZkNoServersMessage msg) throws Exception { + assert locNode.isClient() : locNode; + + throw localNodeFail("All server nodes failed, client node disconnected " + + "(received 'no-servers' message) [locId=" + locNode.id() + ']', true); + } + + /** + * @param msg Message. + * @param evtData Event data. + * @throws Exception If failed. + */ + private void processForceNodeFailMessage(ZkForceNodeFailMessage msg, ZkDiscoveryCustomEventData evtData) + throws Exception { + ClusterNode creatorNode = rtState.top.nodesById.get(evtData.sndNodeId); + + ZookeeperClusterNode node = rtState.top.nodesByInternalId.get(msg.nodeInternalId); + + assert node != null : msg.nodeInternalId; + + if (msg.warning != null) { + U.warn(log, "Received force EVT_NODE_FAILED event with warning [" + + "nodeId=" + node.id() + + ", msg=" + msg.warning + + ", nodeInitiatedEvt=" + (creatorNode != null ? creatorNode : evtData.sndNodeId) + ']'); + } + else { + U.warn(log, "Received force EVT_NODE_FAILED event [" + + "nodeId=" + node.id() + + ", nodeInitiatedEvt=" + (creatorNode != null ? creatorNode : evtData.sndNodeId) + ']'); + } + + if (node.isLocal()) + throw localNodeFail("Received force EVT_NODE_FAILED event for local node.", true); + else + notifyNodeFail(node.internalId(), evtData.topologyVersion()); + } + + /** + * @param msg Message. + * @throws Exception If failed. + */ + private void processCommunicationErrorResolveFinishMessage(ZkCommunicationErrorResolveFinishMessage msg) + throws Exception + { + UUID futId = msg.futId; + + assert futId != null; + + if (log.isInfoEnabled()) + log.info("Received communication error resolve finish message [reqId=" + futId + ']'); + + rtState.commErrProcNodes = null; + + ZkCommunicationErrorResolveResult res = msg.res; + + if (res == null) + res = unmarshalZip(ZkDistributedCollectDataFuture.readResult(rtState.zkClient, zkPaths, futId)); + + ZkCommunicationErrorProcessFuture fut = commErrProcFut.get(); + + assert fut != null; + + Set failedNodes = null; + + if (res.err != null) + U.error(log, "Communication error resolve failed: " + res.err, res.err); + else { + if (res.killedNodes != null) { + failedNodes = U.newHashSet(res.killedNodes.size()); + + for (int i = 0; i < res.killedNodes.size(); i++) { + long internalId = res.killedNodes.get(i); + + if (internalId == locNode.internalId()) { + fut.onError(new IgniteCheckedException("Local node is forced to stop " + + "by communication error resolver")); + + if (rtState.crd) + deleteAliveNodes(res.killedNodes); + + throw localNodeFail("Local node is forced to stop by communication error resolver " + + "[nodeId=" + locNode.id() + ']', false); + } + + ZookeeperClusterNode node = rtState.top.nodesByInternalId.get(internalId); + + assert node != null : internalId; + + failedNodes.add(node.order()); + } + + long topVer = msg.topVer; + + for (int i = 0; i < res.killedNodes.size(); i++) { + long nodeInternalId = res.killedNodes.get(i); + + ClusterNode node = rtState.top.nodesByInternalId.get(nodeInternalId); + + assert node != null : nodeInternalId; + + if (log.isInfoEnabled()) + log.info("Node stop is forced by communication error resolver [nodeId=" + node.id() + ']'); + + notifyNodeFail(nodeInternalId, ++topVer); + } + } + } + + fut.onFinishResolve(failedNodes); + + if (rtState.crd) + deleteAliveNodes(res.killedNodes); + } + + /** + * @param internalIds Nodes internal IDs. + * @throws Exception If failed. + */ + private void deleteAliveNodes(@Nullable GridLongList internalIds) throws Exception { + if (internalIds == null) + return; + + List alives = rtState.zkClient.getChildren(zkPaths.aliveNodesDir); + + for (int i = 0; i < alives.size(); i++) { + String alive = alives.get(i); + + if (internalIds.contains(ZkIgnitePaths.aliveInternalId(alive))) + rtState.zkClient.deleteIfExistsAsync(zkPaths.aliveNodesDir + "/" + alive); + } + } + + /** + * @param msg Message. + * @param evtData Event data. + * @throws Exception If failed. + */ + private void processCommunicationErrorResolveStartMessage(ZkCommunicationErrorResolveStartMessage msg, + ZkDiscoveryCustomEventData evtData) throws Exception { + ZkCommunicationErrorProcessFuture fut; + + for (;;) { + fut = commErrProcFut.get(); + + if (fut == null || fut.isDone()) { + ZkCommunicationErrorProcessFuture newFut = + ZkCommunicationErrorProcessFuture.createOnStartResolveRequest(this); + + if (commErrProcFut.compareAndSet(fut, newFut)) + fut = newFut; + else + fut = commErrProcFut.get(); + } + + if (fut.onStartResolveRequest(evtData.topologyVersion())) + break; + else { + try { + fut.get(); + } + catch (Exception e) { + U.warn(log, "Previous communication error process future failed: " + e); + } + } + } + + if (log.isInfoEnabled()) { + log.info("Received communication error resolve request [reqId=" + msg.id + + ", topVer=" + rtState.top.topologySnapshot() + ']'); + } + + assert !fut.isDone() : fut; + + final String futPath = zkPaths.distributedFutureBasePath(msg.id); + final ZkCommunicationErrorProcessFuture fut0 = fut; + + rtState.commErrProcNodes = rtState.top.topologySnapshot(); + + if (rtState.crd) { + ZkDistributedCollectDataFuture nodeResFut = collectCommunicationStatusFuture(msg.id); + + fut.nodeResultCollectFuture(nodeResFut); + } + + runInWorkerThread(new ZkRunnable(rtState, this) { + @Override protected void run0() throws Exception { + fut0.checkConnection(rtState, futPath, rtState.commErrProcNodes); + } + }); + } + + /** + * @param futId Future ID. + * @return Future. + * @throws Exception If failed. + */ + private ZkDistributedCollectDataFuture collectCommunicationStatusFuture(UUID futId) throws Exception { + return new ZkDistributedCollectDataFuture(this, rtState, zkPaths.distributedFutureBasePath(futId), + new Callable() { + @Override public Void call() throws Exception { + // Future is completed from ZK event thread. + onCommunicationErrorResolveStatusReceived(rtState); + + return null; + } + } + ); + } + + /** + * @param rtState Runtime state. + * @throws Exception If failed. + */ + private void onCommunicationErrorResolveStatusReceived(final ZkRuntimeState rtState) throws Exception { + ZkDiscoveryEventsData evtsData = rtState.evtsData; + + UUID futId = evtsData.communicationErrorResolveFutureId(); + + if (log.isInfoEnabled()) + log.info("Received communication status from all nodes [reqId=" + futId + ']'); + + assert futId != null; + + String futPath = zkPaths.distributedFutureBasePath(futId); + + List initialNodes = rtState.commErrProcNodes; + + assert initialNodes != null; + + rtState.commErrProcNodes = null; + + List topSnapshot = rtState.top.topologySnapshot(); + + Map nodesRes = U.newHashMap(topSnapshot.size()); + + Exception err = null; + + for (ClusterNode node : topSnapshot) { + byte[] stateBytes = ZkDistributedCollectDataFuture.readNodeResult(futPath, + rtState.zkClient, + node.order()); + + ZkCommunicationErrorNodeState nodeState = unmarshalZip(stateBytes); + + if (nodeState.err != null) { + if (err == null) + err = new Exception("Failed to resolve communication error."); + + err.addSuppressed(nodeState.err); + } + else { + assert nodeState.commState != null; + + nodesRes.put(node.id(), nodeState.commState); + } + } + + long topVer = evtsData.topVer; + + GridLongList killedNodesList = null; + + if (err == null) { + boolean fullyConnected = true; + + for (Map.Entry e : nodesRes.entrySet()) { + if (!checkFullyConnected(e.getValue(), initialNodes, rtState.top)) { + fullyConnected = false; + + break; + } + } + + if (fullyConnected) { + if (log.isInfoEnabled()) { + log.info("Finish communication error resolve process automatically, there are no " + + "communication errors [reqId=" + futId + ']'); + } + } + else { + CommunicationFailureResolver rslvr = spi.ignite().configuration().getCommunicationFailureResolver(); + + if (rslvr != null) { + if (log.isInfoEnabled()) { + log.info("Call communication error resolver [reqId=" + futId + + ", rslvr=" + rslvr.getClass().getSimpleName() + ']'); + } + + ZkCommunicationFailureContext ctx = new ZkCommunicationFailureContext( + ((IgniteKernal)spi.ignite()).context().cache().context(), + topSnapshot, + initialNodes, + nodesRes); + + try { + rslvr.resolve(ctx); + + Set killedNodes = ctx.killedNodes(); + + if (killedNodes != null) { + if (log.isInfoEnabled()) { + log.info("Communication error resolver forced nodes stop [reqId=" + futId + + ", killNodeCnt=" + killedNodes.size() + + ", nodeIds=" + U.nodeIds(killedNodes) + ']'); + } + + killedNodesList = new GridLongList(killedNodes.size()); + + for (ClusterNode killedNode : killedNodes) { + killedNodesList.add(((ZookeeperClusterNode)killedNode).internalId()); + + evtsData.topVer++; + } + } + } + catch (Exception e) { + err = e; + + U.error(log, "Failed to resolve communication error with configured resolver [reqId=" + futId + ']', e); + } + } + } + } + + evtsData.communicationErrorResolveFutureId(null); + + ZkCommunicationErrorResolveResult res = new ZkCommunicationErrorResolveResult(killedNodesList, err); + + ZkCommunicationErrorResolveFinishMessage msg = new ZkCommunicationErrorResolveFinishMessage(futId, topVer); + + msg.res = res; + + ZkDistributedCollectDataFuture.saveResult(zkPaths.distributedFutureResultPath(futId), + rtState.zkClient, + marshalZip(res)); + + evtsData.evtIdGen++; + + ZkDiscoveryCustomEventData evtData = new ZkDiscoveryCustomEventData( + evtsData.evtIdGen, + 0L, + topVer, + locNode.id(), + msg, + null); + + evtData.resolvedMsg = msg; + + evtsData.addEvent(rtState.top.nodesByOrder.values(), evtData); + + saveAndProcessNewEvents(); + + // Need re-check alive nodes in case join was delayed. + rtState.zkClient.getChildrenAsync(zkPaths.aliveNodesDir, rtState.watcher, rtState.watcher); + } + + /** + * @param commState Node communication state. + * @param initialNodes Topology snapshot when communication error resolve started. + * @param top Current topology. + * @return {@code True} if node has connection to all alive nodes. + */ + private boolean checkFullyConnected(BitSet commState, List initialNodes, ZkClusterNodes top) { + int startIdx = 0; + + for (;;) { + int idx = commState.nextClearBit(startIdx); + + if (idx >= initialNodes.size()) + return true; + + ClusterNode node = initialNodes.get(idx); + + if (top.nodesById.containsKey(node.id())) + return false; + + startIdx = idx + 1; + } + } + + /** + * + */ + public void simulateNodeFailure() { + ZkRuntimeState rtState = this.rtState; + + ZookeeperClient client = rtState.zkClient; + + client.deleteIfExistsAsync(zkPaths.aliveNodesDir); + + rtState.onCloseStart(new IgniteCheckedException("Simulate node failure error.")); + + rtState.zkClient.close(); + } + + /** + * @param evtData Event data. + * @param msg Custom message. + */ + @SuppressWarnings("unchecked") + private void notifyCustomEvent(final ZkDiscoveryCustomEventData evtData, final DiscoverySpiCustomMessage msg) { + assert !(msg instanceof ZkInternalMessage) : msg; + + if (log.isDebugEnabled()) + log.debug(" [topVer=" + evtData.topologyVersion() + ", msg=" + msg + ']'); + + final ZookeeperClusterNode sndNode = rtState.top.nodesById.get(evtData.sndNodeId); + + assert sndNode != null : evtData; + + final List topSnapshot = rtState.top.topologySnapshot(); + + lsnr.onDiscovery( + DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT, + evtData.topologyVersion(), + sndNode, + topSnapshot, + Collections.>emptyMap(), + msg); + } + + /** + * @param joinedEvtData Event data. + * @param joiningData Joining node data. + */ + @SuppressWarnings("unchecked") + private void notifyNodeJoin(ZkJoinedNodeEvtData joinedEvtData, ZkJoiningNodeData joiningData) { + final ZookeeperClusterNode joinedNode = joiningData.node(); + + joinedNode.order(joinedEvtData.topVer); + joinedNode.internalId(joinedEvtData.joinedInternalId); + + joinedNode.setMetrics(new ClusterMetricsSnapshot()); + + rtState.top.addNode(joinedNode); + + final List topSnapshot = rtState.top.topologySnapshot(); + + lsnr.onDiscovery(EVT_NODE_JOINED, + joinedEvtData.topVer, + joinedNode, + topSnapshot, + Collections.>emptyMap(), + null); + } + + /** + * @param evtData Event data. + */ + private void notifyNodeFail(final ZkDiscoveryNodeFailEventData evtData) { + notifyNodeFail(evtData.failedNodeInternalId(), evtData.topologyVersion()); + } + + /** + * @param nodeInternalOrder Node order. + * @param topVer Topology version. + */ + private void notifyNodeFail(long nodeInternalOrder, long topVer) { + final ZookeeperClusterNode failedNode = rtState.top.removeNode(nodeInternalOrder); + + assert failedNode != null && !failedNode.isLocal() : failedNode; + + PingFuture pingFut = pingFuts.get(failedNode.order()); + + if (pingFut != null) + pingFut.onDone(false); + + final List topSnapshot = rtState.top.topologySnapshot(); + + lsnr.onDiscovery(EVT_NODE_FAILED, + topVer, + failedNode, + topSnapshot, + Collections.>emptyMap(), + null); + } + + /** + * @param msg Message to log. + * @param clientReconnect {@code True} if allow client reconnect. + * @return Exception to be thrown. + */ + private ZookeeperClientFailedException localNodeFail(String msg, boolean clientReconnect) { + U.warn(log, msg); + +// if (locNode.isClient() && rtState.zkClient.connected()) { +// String path = rtState.locNodeZkPath.substring(rtState.locNodeZkPath.lastIndexOf('/') + 1); +// +// String joinDataPath = zkPaths.joiningNodeDataPath(locNode.id(), ZkIgnitePaths.aliveNodePrefixId(path)); +// +// try { +// if (rtState.zkClient.existsNoRetry(joinDataPath)) +// rtState.zkClient.deleteIfExistsNoRetry(joinDataPath, -1); +// } +// catch (Exception e) { +// if (log.isDebugEnabled()) +// log.debug("Failed to clean local node's join data on stop: " + e); +// } +// } + + if (rtState.zkClient.connected()) + rtState.zkClient.close(); + + if (clientReconnect && clientReconnectEnabled) { + assert locNode.isClient() : locNode; + + boolean reconnect = false; + + synchronized (stateMux) { + if (connState == ConnectionState.STARTED) { + reconnect = true; + + connState = ConnectionState.DISCONNECTED; + + rtState.onCloseStart(disconnectError()); + } + } + + if (reconnect) { + UUID newId = UUID.randomUUID(); + + U.quietAndWarn(log, "Client node will try to reconnect with new id [" + + "newId=" + newId + + ", prevId=" + locNode.id() + + ", locNode=" + locNode + ']'); + + runInWorkerThread(new ReconnectClosure(newId)); + } + } + else { + rtState.errForClose = new IgniteCheckedException(msg); + + notifySegmented(); + } + + // Stop any further processing. + return new ZookeeperClientFailedException(msg); + } + + /** + * @param ctx Context for logging. + * @throws Exception If failed. + */ + private void handleProcessedEvents(String ctx) throws Exception { + Iterator it = rtState.evtsData.evts.values().iterator(); + + List newEvts = null; + + ZkDiscoveryEventData prevEvtData = null; + + while (it.hasNext()) { + ZkDiscoveryEventData evtData = it.next(); + + if (evtData.allAcksReceived()) { + if (prevEvtData != null) { + if (log.isInfoEnabled()) { + log.info("Previous event is not acked [" + + "evtId=" + evtData.eventId() + + ", prevEvtData=" + prevEvtData + + ", remaining=" + prevEvtData.remainingAcks() + ']'); + } + } + + prevEvtData = null; + + switch (evtData.eventType()) { + case ZkDiscoveryEventData.ZK_EVT_NODE_JOIN: { + handleProcessedJoinEventAsync((ZkDiscoveryNodeJoinEventData)evtData); + + break; + } + + case ZkDiscoveryEventData.ZK_EVT_CUSTOM_EVT: { + DiscoverySpiCustomMessage ack = handleProcessedCustomEvent(ctx, + (ZkDiscoveryCustomEventData)evtData); + + if (ack != null) { + ZkDiscoveryCustomEventData ackEvtData = createAckEvent( + ack, + (ZkDiscoveryCustomEventData)evtData); + + if (newEvts == null) + newEvts = new ArrayList<>(); + + newEvts.add(ackEvtData); + } + + break; + } + + case ZkDiscoveryEventData.ZK_EVT_NODE_FAILED: { + if (log.isDebugEnabled()) + log.debug("All nodes processed node fail [evtData=" + evtData + ']'); + + break; // Do not need addition cleanup. + } + } + + it.remove(); + } + else + prevEvtData = evtData; + } + + if (newEvts != null) { + Collection nodes = rtState.top.nodesByOrder.values(); + + for (int i = 0; i < newEvts.size(); i++) + rtState.evtsData.addEvent(nodes, newEvts.get(i)); + + saveAndProcessNewEvents(); + } + } + + /** + * @param ack Ack message. + * @param origEvt Original custom event. + * @return Event data. + * @throws Exception If failed. + */ + private ZkDiscoveryCustomEventData createAckEvent( + DiscoverySpiCustomMessage ack, + ZkDiscoveryCustomEventData origEvt) throws Exception { + assert ack != null; + + rtState.evtsData.evtIdGen++; + + long evtId = rtState.evtsData.evtIdGen; + + byte[] ackBytes = marshalZip(ack); + + String path = zkPaths.ackEventDataPath(origEvt.eventId()); + + if (log.isDebugEnabled()) + log.debug("Create ack event: " + path); + + // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8194 + rtState.zkClient.createIfNeeded( + path, + ackBytes, + CreateMode.PERSISTENT); + + ZkDiscoveryCustomEventData ackEvtData = new ZkDiscoveryCustomEventData( + evtId, + origEvt.eventId(), + origEvt.topologyVersion(), // Use topology version from original event. + locNode.id(), + null, + null); + + ackEvtData.resolvedMsg = ack; + + if (log.isDebugEnabled()) { + log.debug("Generated CUSTOM event ack [origEvtId=" + origEvt.eventId() + + ", evt=" + ackEvtData + + ", evtSize=" + ackBytes.length + + ", msg=" + ack + ']'); + } + + return ackEvtData; + } + + /** + * @param failedNodes Failed nodes. + * @throws Exception If failed. + */ + private void handleProcessedEventsOnNodesFail(List failedNodes) throws Exception { + boolean processed = false; + + for (Iterator> it = rtState.evtsData.evts.entrySet().iterator(); it.hasNext();) { + Map.Entry e = it.next(); + + ZkDiscoveryEventData evtData = e.getValue(); + + for (int i = 0; i < failedNodes.size(); i++) { + ZookeeperClusterNode failedNode = failedNodes.get(i); + + if (evtData.onNodeFail(failedNode)) + processed = true; + } + } + + if (processed) + handleProcessedEvents("fail-" + U.nodeIds(failedNodes)); + } + + /** + * @param evtData Event data. + * @throws Exception If failed. + */ + private void handleProcessedJoinEventAsync(ZkDiscoveryNodeJoinEventData evtData) throws Exception { + if (log.isDebugEnabled()) + log.debug("All nodes processed node join [evtData=" + evtData + ']'); + + for (int i = 0; i < evtData.joinedNodes.size(); i++) { + ZkJoinedNodeEvtData joinedEvtData = evtData.joinedNodes.get(i); + + deleteJoiningNodeData(joinedEvtData.nodeId, joinedEvtData.joinDataPrefixId, joinedEvtData.joinDataPartCnt); + + if (joinedEvtData.secSubjPartCnt > 0) { + deleteMultiplePartsAsync(rtState.zkClient, + zkPaths.joinEventSecuritySubjectPath(evtData.eventId()), + joinedEvtData.secSubjPartCnt); + } + } + + deleteDataForJoinedAsync(evtData); + } + + /** + * @param nodeId Node ID. + * @param joinDataPrefixId Path prefix. + * @param partCnt Parts count. + */ + private void deleteJoiningNodeData(UUID nodeId, UUID joinDataPrefixId, int partCnt) { + String evtDataPath = zkPaths.joiningNodeDataPath(nodeId, joinDataPrefixId); + + if (log.isDebugEnabled()) + log.debug("Delete joining node data [path=" + evtDataPath + ']'); + + rtState.zkClient.deleteIfExistsAsync(evtDataPath); + + if (partCnt > 1) + deleteMultiplePartsAsync(rtState.zkClient, evtDataPath + ":", partCnt); + } + + /** + * @param evtData Event data. + */ + private void deleteDataForJoinedAsync(ZkDiscoveryNodeJoinEventData evtData) { + String dataForJoinedPath = zkPaths.joinEventDataPathForJoined(evtData.eventId()); + + if (log.isDebugEnabled()) + log.debug("Delete data for joined node [path=" + dataForJoinedPath + ']'); + + deleteMultiplePartsAsync(rtState.zkClient, dataForJoinedPath, evtData.dataForJoinedPartCnt); + } + + /** + * @param ctx Context for log. + * @param evtData Event data. + * @return Ack message. + * @throws Exception If failed. + */ + @Nullable private DiscoverySpiCustomMessage handleProcessedCustomEvent(String ctx, ZkDiscoveryCustomEventData evtData) + throws Exception { + if (log.isDebugEnabled()) + log.debug("All nodes processed custom event [ctx=" + ctx + ", evtData=" + evtData + ']'); + + if (!evtData.ackEvent()) { + if (evtData.evtPath != null) + deleteCustomEventDataAsync(rtState.zkClient, evtData.evtPath); + else { + if (evtData.resolvedMsg instanceof ZkCommunicationErrorResolveFinishMessage) { + UUID futId = ((ZkCommunicationErrorResolveFinishMessage)evtData.resolvedMsg).futId; + + ZkDistributedCollectDataFuture.deleteFutureData(rtState.zkClient, zkPaths, futId, log); + } + } + + assert evtData.resolvedMsg != null || locNode.order() > evtData.topologyVersion() : evtData; + + if (evtData.resolvedMsg != null) + return evtData.resolvedMsg.ackMessage(); + } + else { + String path = zkPaths.ackEventDataPath(evtData.origEvtId); + + if (log.isDebugEnabled()) + log.debug("Delete path: " + path); + + rtState.zkClient.deleteIfExistsAsync(path); + } + + return null; + } + + /** + * @param c Closure to run. + */ + void runInWorkerThread(Runnable c) { + IgniteThreadPoolExecutor pool; + + synchronized (stateMux) { + if (connState == ConnectionState.STOPPED) { + LT.warn(log, "Do not run closure, node is stopped."); + + return; + } + + if (utilityPool == null) { + utilityPool = new IgniteThreadPoolExecutor("zk-discovery-pool", + igniteInstanceName, + 0, + 1, + 2000, + new LinkedBlockingQueue()); + } + + pool = utilityPool; + } + + pool.submit(c); + } + + /** + * + */ + public void stop() { + stop0(new IgniteSpiException("Node stopped")); + } + + /** + * @param e Error. + */ + private void stop0(Throwable e) { + if (!stop.compareAndSet(false, true)) + return; + + ZkRuntimeState rtState = this.rtState; + + if (rtState.zkClient != null && rtState.locNodeZkPath != null && rtState.zkClient.connected()) { + try { + rtState.zkClient.deleteIfExistsNoRetry(rtState.locNodeZkPath, -1); + } + catch (Exception err) { + if (log.isDebugEnabled()) + log.debug("Failed to delete local node's znode on stop: " + err); + } + } + + IgniteCheckedException err = new IgniteCheckedException("Node stopped."); + + synchronized (stateMux) { + connState = ConnectionState.STOPPED; + + rtState.onCloseStart(err); + } + + IgniteUtils.shutdownNow(ZookeeperDiscoveryImpl.class, utilityPool, log); + + busyLock.block(); + + busyLock.unblock(); + + joinFut.onDone(e); + + ZookeeperClient zkClient = rtState.zkClient; + + if (zkClient != null) + zkClient.close(); + + finishFutures(err); + } + + /** + * @param err Error. + */ + private void finishFutures(IgniteCheckedException err) { + ZkCommunicationErrorProcessFuture commErrFut = commErrProcFut.get(); + + if (commErrFut != null) + commErrFut.onError(err); + + for (PingFuture fut : pingFuts.values()) + fut.onDone(err); + } + + /** + * @param busyLock Busy lock. + * @param err Error. + */ + void onFatalError(GridSpinBusyLock busyLock, Throwable err) { + busyLock.leaveBusy(); + + if (err instanceof ZookeeperClientFailedException) + return; // Processed by ZookeeperClient listener. + + Ignite ignite = spi.ignite(); + + if (stopping() || ignite == null) + return; + + U.error(log, "Fatal error in ZookeeperDiscovery. " + + "Stopping the node in order to prevent cluster wide instability.", err); + + stop0(err); + + new Thread(new Runnable() { + @Override public void run() { + try { + IgnitionEx.stop(igniteInstanceName, true, true); + + U.log(log, "Stopped the node successfully in response to fatal error in ZookeeperDiscoverySpi."); + } + catch (Throwable e) { + U.error(log, "Failed to stop the node successfully in response to fatal error in " + + "ZookeeperDiscoverySpi.", e); + } + } + }, "node-stop-thread").start(); + + if (err instanceof Error) + throw (Error)err; + } + + /** + * @param zipBytes Zip-compressed bytes. + * @return Unmarshalled object. + * @throws IgniteCheckedException If failed. + */ + private T unmarshalZip(byte[] zipBytes) throws Exception { + assert zipBytes != null && zipBytes.length > 0; + + InflaterInputStream in = new InflaterInputStream(new ByteArrayInputStream(zipBytes)); + + return marsh.unmarshal(in, U.resolveClassLoader(spi.ignite().configuration())); + } + + /** + * @param obj Object. + * @return Bytes. + * @throws IgniteCheckedException If failed. + */ + byte[] marshalZip(Object obj) throws IgniteCheckedException { + assert obj != null; + + return zip(U.marshal(marsh, obj)); + } + + /** + * @param bytes Bytes to compress. + * @return Zip-compressed bytes. + */ + private static byte[] zip(byte[] bytes) { + Deflater deflater = new Deflater(); + + deflater.setInput(bytes); + deflater.finish(); + + GridByteArrayOutputStream out = new GridByteArrayOutputStream(bytes.length); + + final byte[] buf = new byte[bytes.length]; + + while (!deflater.finished()) { + int cnt = deflater.deflate(buf); + + out.write(buf, 0, cnt); + } + + return out.toByteArray(); + } + + /** + * @param zipBytes Zip-compressed bytes. + * @return Uncompressed bytes. + * @throws DataFormatException If compressed data format is invalid. + */ + public static byte[] unzip(byte[] zipBytes) throws DataFormatException { + Inflater inflater = new Inflater(); + + inflater.setInput(zipBytes); + + GridByteArrayOutputStream out = new GridByteArrayOutputStream(zipBytes.length * 2); + + final byte[] buf = new byte[zipBytes.length]; + + while (!inflater.finished()) { + int cnt = inflater.inflate(buf); + + out.write(buf, 0, cnt); + } + + return out.toByteArray(); + } + + /** + * + */ + private class ReconnectClosure implements Runnable { + /** */ + private final UUID newId; + + /** + * @param newId New ID. + */ + ReconnectClosure(UUID newId) { + assert newId != null; + + this.newId = newId; + } + + /** {@inheritDoc} */ + @Override public void run() { + finishFutures(disconnectError()); + + busyLock.block(); + + busyLock.unblock(); + + doReconnect(newId); + } + } + + /** + * + */ + private class ConnectionLossListener implements IgniteRunnable { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public void run() { + if (clientReconnectEnabled) { + synchronized (stateMux) { + if (connState == ConnectionState.STARTED) { + connState = ConnectionState.DISCONNECTED; + + rtState.onCloseStart(disconnectError()); + } + else + return; + } + + UUID newId = UUID.randomUUID(); + + U.quietAndWarn(log, "Connection to Zookeeper server is lost, local node will try to reconnect with new id [" + + "newId=" + newId + + ", prevId=" + locNode.id() + + ", locNode=" + locNode + ']'); + + runInWorkerThread(new ReconnectClosure(newId)); + } + else { + U.warn(log, "Connection to Zookeeper server is lost, local node SEGMENTED."); + + onSegmented(new IgniteSpiException("Zookeeper connection loss.")); + } + } + } + + /** + * + */ + private class ZkWatcher extends ZkAbstractWatcher implements ZkRuntimeState.ZkWatcher { + /** + * @param rtState Runtime state. + */ + ZkWatcher(ZkRuntimeState rtState) { + super(rtState, ZookeeperDiscoveryImpl.this); + } + + /** {@inheritDoc} */ + @Override public void process0(WatchedEvent evt) { + if (evt.getType() == Event.EventType.NodeDataChanged) { + if (evt.getPath().equals(zkPaths.evtsPath)) { + if (!rtState.crd) + rtState.zkClient.getDataAsync(evt.getPath(), this, this); + } + else + U.warn(log, "Received NodeDataChanged for unexpected path: " + evt.getPath()); + } + else if (evt.getType() == Event.EventType.NodeChildrenChanged) { + if (evt.getPath().equals(zkPaths.aliveNodesDir)) + rtState.zkClient.getChildrenAsync(evt.getPath(), this, this); + else if (evt.getPath().equals(zkPaths.customEvtsDir)) + rtState.zkClient.getChildrenAsync(evt.getPath(), this, this); + else + U.warn(log, "Received NodeChildrenChanged for unexpected path: " + evt.getPath()); + } + } + + /** {@inheritDoc} */ + @Override public void processResult(int rc, String path, Object ctx, List children, Stat stat) { + if (!onProcessStart()) + return; + + try { + assert rc == 0 : KeeperException.Code.get(rc); + + if (path.equals(zkPaths.aliveNodesDir)) + generateTopologyEvents(children); + else if (path.equals(zkPaths.customEvtsDir)) + generateCustomEvents(children); + else + U.warn(log, "Children callback for unexpected path: " + path); + + onProcessEnd(); + } + catch (Throwable e) { + onProcessError(e); + } + } + + /** {@inheritDoc} */ + @Override public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) { + if (!onProcessStart()) + return; + + try { + assert rc == 0 : KeeperException.Code.get(rc); + + if (path.equals(zkPaths.evtsPath)) { + if (!rtState.crd) + processNewEvents(data); + } + else + U.warn(log, "Data callback for unknown path: " + path); + + onProcessEnd(); + } + catch (Throwable e) { + onProcessError(e); + } + } + } + + /** + * + */ + private class AliveNodeDataWatcher extends ZkAbstractWatcher implements ZkRuntimeState.ZkAliveNodeDataWatcher { + /** + * @param rtState Runtime state. + */ + AliveNodeDataWatcher(ZkRuntimeState rtState) { + super(rtState, ZookeeperDiscoveryImpl.this); + } + + /** {@inheritDoc} */ + @Override public void process0(WatchedEvent evt) { + if (evt.getType() == Event.EventType.NodeDataChanged) + rtState.zkClient.getDataAsync(evt.getPath(), this, this); + } + + /** {@inheritDoc} */ + @Override public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) { + if (!onProcessStart()) + return; + + try { + assert rtState.crd; + + processResult0(rc, path, data); + + onProcessEnd(); + } + catch (Throwable e) { + onProcessError(e); + } + } + + /** + * @param rc Result code. + * @param path Path. + * @param data Data. + * @throws Exception If failed. + */ + private void processResult0(int rc, String path, byte[] data) throws Exception { + if (rc == KeeperException.Code.NONODE.intValue()) { + if (log.isDebugEnabled()) + log.debug("Alive node callaback, no node: " + path); + + return; + } + + assert rc == 0 : KeeperException.Code.get(rc); + + if (data.length > 0) { + ZkAliveNodeData nodeData = unmarshalZip(data); + + Long nodeInternalId = ZkIgnitePaths.aliveInternalId(path); + + Iterator it = rtState.evtsData.evts.values().iterator(); + + boolean processed = false; + + while (it.hasNext()) { + ZkDiscoveryEventData evtData = it.next(); + + if (evtData.onAckReceived(nodeInternalId, nodeData.lastProcEvt)) + processed = true; + } + + if (processed) + handleProcessedEvents("ack-" + nodeInternalId); + } + } + } + + /** + * + */ + private abstract class PreviousNodeWatcher extends ZkAbstractWatcher implements AsyncCallback.StatCallback { + /** + * @param rtState Runtime state. + */ + PreviousNodeWatcher(ZkRuntimeState rtState) { + super(rtState, ZookeeperDiscoveryImpl.this); + } + + /** {@inheritDoc} */ + @Override public void process0(WatchedEvent evt) { + if (evt.getType() == Event.EventType.NodeDeleted) + onPreviousNodeFail(); + else { + if (evt.getType() != Event.EventType.None) + rtState.zkClient.existsAsync(evt.getPath(), this, this); + } + } + + /** {@inheritDoc} */ + @Override public void processResult(int rc, String path, Object ctx, Stat stat) { + if (!onProcessStart()) + return; + + try { + assert rc == 0 || rc == KeeperException.Code.NONODE.intValue() : KeeperException.Code.get(rc); + + if (rc == KeeperException.Code.NONODE.intValue() || stat == null) + onPreviousNodeFail(); + + onProcessEnd(); + } + catch (Throwable e) { + onProcessError(e); + } + } + + /** + * + */ + abstract void onPreviousNodeFail(); + } + + /** + * + */ + private class ServerPreviousNodeWatcher extends PreviousNodeWatcher { + /** + * @param rtState Runtime state. + */ + ServerPreviousNodeWatcher(ZkRuntimeState rtState) { + super(rtState); + + assert !locNode.isClient() : locNode; + } + + /** {@inheritDoc} */ + @Override void onPreviousNodeFail() { + if (log.isInfoEnabled()) + log.info("Previous server node failed, check is node new coordinator [locId=" + locNode.id() + ']'); + + rtState.zkClient.getChildrenAsync(zkPaths.aliveNodesDir, null, new CheckCoordinatorCallback(rtState)); + } + } + + /** + * + */ + private class ClientPreviousNodeWatcher extends PreviousNodeWatcher { + /** + * @param rtState Runtime state. + */ + ClientPreviousNodeWatcher(ZkRuntimeState rtState) { + super(rtState); + + assert locNode.isClient() : locNode; + } + + /** {@inheritDoc} */ + @Override void onPreviousNodeFail() { + if (log.isInfoEnabled()) + log.info("Watched node failed, check if there are alive servers [locId=" + locNode.id() + ']'); + + rtState.zkClient.getChildrenAsync(zkPaths.aliveNodesDir, null, new CheckClientsStatusCallback(rtState)); + } + } + + /** + * + */ + class CheckCoordinatorCallback extends ZkAbstractChildrenCallback { + /** + * @param rtState Runtime state. + */ + CheckCoordinatorCallback(ZkRuntimeState rtState) { + super(rtState, ZookeeperDiscoveryImpl.this); + } + + /** {@inheritDoc} */ + @Override public void processResult0(int rc, String path, Object ctx, List children, Stat stat) + throws Exception + { + assert rc == 0 : KeeperException.Code.get(rc); + + checkIsCoordinator(children); + } + } + + /** + * + */ + class CheckClientsStatusCallback extends ZkAbstractChildrenCallback { + /** + * @param rtState Runtime state. + */ + CheckClientsStatusCallback(ZkRuntimeState rtState) { + super(rtState, ZookeeperDiscoveryImpl.this); + } + + /** {@inheritDoc} */ + @Override void processResult0(int rc, String path, Object ctx, List children, Stat stat) + throws Exception + { + assert rc == 0 : KeeperException.Code.get(rc); + + checkClientsStatus(children); + } + } + + /** + * + */ + private class PingFuture extends GridFutureAdapter implements IgniteSpiTimeoutObject { + /** */ + private final ZookeeperClusterNode node; + + /** */ + private final long endTime; + + /** */ + private final IgniteUuid id; + + /** */ + private final ZkRuntimeState rtState; + + /** + * @param rtState Runtime state. + * @param node Node. + */ + PingFuture(ZkRuntimeState rtState, ZookeeperClusterNode node) { + this.rtState = rtState; + this.node = node; + + id = IgniteUuid.fromUuid(node.id()); + + endTime = System.currentTimeMillis() + node.sessionTimeout() + 1000; + }; + + /** {@inheritDoc} */ + @Override public IgniteUuid id() { + return id; + } + + /** {@inheritDoc} */ + @Override public long endTime() { + return endTime; + } + + /** {@inheritDoc} */ + @Override public void onTimeout() { + if (checkNodeAndState()) { + runInWorkerThread(new ZkRunnable(rtState, ZookeeperDiscoveryImpl.this) { + @Override protected void run0() throws Exception { + if (checkNodeAndState()) { + try { + for (String path : rtState.zkClient.getChildren(zkPaths.aliveNodesDir)) { + if (node.internalId() == ZkIgnitePaths.aliveInternalId(path)) { + onDone(true); + + return; + } + } + + onDone(false); + } + catch (Exception e) { + onDone(e); + + throw e; + } + } + } + + @Override void onStartFailed() { + onDone(rtState.errForClose); + } + }); + } + } + + /** {@inheritDoc} */ + @Override public boolean onDone(@Nullable Boolean res, @Nullable Throwable err) { + if (super.onDone(res, err)) { + pingFuts.remove(node.order(), this); + + return true; + } + + return false; + } + + /** + * @return {@code False} if future was completed. + */ + boolean checkNodeAndState() { + if (isDone()) + return false; + + Exception err = rtState.errForClose; + + if (err != null) { + onDone(err); + + return false; + } + + ConnectionState connState = ZookeeperDiscoveryImpl.this.connState; + + if (connState == ConnectionState.DISCONNECTED) { + onDone(new IgniteClientDisconnectedException(null, "Client is disconnected.")); + + return false; + } + else if (connState == ConnectionState.STOPPED) { + onDone(new IgniteException("Node stopped.")); + + return false; + } + + if (node(node.id()) == null) { + onDone(false); + + return false; + } + + return true; + } + } + + /** + * + */ + enum ConnectionState { + /** */ + STARTED, + /** */ + DISCONNECTED, + /** */ + STOPPED + } +} diff --git a/modules/zookeeper/src/test/java/org/apache/ZookeeperNodeStart.java b/modules/zookeeper/src/test/java/org/apache/ZookeeperNodeStart.java new file mode 100644 index 0000000000000..ef4d5f47f51fa --- /dev/null +++ b/modules/zookeeper/src/test/java/org/apache/ZookeeperNodeStart.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache; + +import org.apache.ignite.Ignition; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi; + +/** + * + */ +public class ZookeeperNodeStart { + public static void main(String[] args) throws Exception { + try { + IgniteConfiguration cfg = new IgniteConfiguration(); + + ZookeeperDiscoverySpi spi = new ZookeeperDiscoverySpi(); + + spi.setZkConnectionString("localhost:2181"); + + cfg.setDiscoverySpi(spi); + + Ignition.start(cfg); + } + catch (Throwable e) { + e.printStackTrace(System.out); + + System.exit(1); + } + } +} diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/IgniteCacheEntryListenerWithZkDiscoAtomicTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/IgniteCacheEntryListenerWithZkDiscoAtomicTest.java new file mode 100644 index 0000000000000..754a6bf05b10a --- /dev/null +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/IgniteCacheEntryListenerWithZkDiscoAtomicTest.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ignite.spi.discovery.zk; + +import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerAtomicTest; + +/** + * Class is added to mute {@link #testConcurrentRegisterDeregister} test in ZooKeeper suite + * (see related ticket). + * + * When slow down is tracked down and fixed this class can be replaced back with its parent. + */ +public class IgniteCacheEntryListenerWithZkDiscoAtomicTest extends IgniteCacheEntryListenerAtomicTest { + /** {@inheritDoc} */ + @Override public void testConcurrentRegisterDeregister() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8109"); + } +} diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiAbstractTestSuite.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiAbstractTestSuite.java new file mode 100644 index 0000000000000..766635cb5580c --- /dev/null +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiAbstractTestSuite.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk; + +import java.io.File; +import java.util.ArrayList; +import java.util.List; +import junit.framework.TestSuite; +import org.apache.curator.test.InstanceSpec; +import org.apache.curator.test.TestingCluster; +import org.apache.ignite.IgniteException; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.spi.discovery.DiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.testframework.config.GridTestProperties; + +/** + * Allows to run regular Ignite tests with {@link org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi}. + */ +public abstract class ZookeeperDiscoverySpiAbstractTestSuite extends TestSuite { + /** */ + private static TestingCluster testingCluster; + + /** + * @throws Exception If failed. + */ + public static void initSuite() throws Exception { + System.setProperty("zookeeper.forceSync", "false"); + + testingCluster = createTestingCluster(3); + + testingCluster.start(); + + System.setProperty(GridTestProperties.IGNITE_CFG_PREPROCESSOR_CLS, ZookeeperDiscoverySpiAbstractTestSuite.class.getName()); + } + + /** + * Called via reflection by {@link org.apache.ignite.testframework.junits.GridAbstractTest}. + * + * @param cfg Configuration to change. + */ + public synchronized static void preprocessConfiguration(IgniteConfiguration cfg) { + if (testingCluster == null) + throw new IllegalStateException("Test Zookeeper cluster is not started."); + + ZookeeperDiscoverySpi zkSpi = new ZookeeperDiscoverySpi(); + + DiscoverySpi spi = cfg.getDiscoverySpi(); + + if (spi instanceof TcpDiscoverySpi) + zkSpi.setClientReconnectDisabled(((TcpDiscoverySpi)spi).isClientReconnectDisabled()); + + zkSpi.setSessionTimeout(30_000); + zkSpi.setZkConnectionString(testingCluster.getConnectString()); + + cfg.setDiscoverySpi(zkSpi); + } + + /** + * @param instances Number of instances in + * @return Test cluster. + */ + public static TestingCluster createTestingCluster(int instances) { + String tmpDir = System.getProperty("java.io.tmpdir"); + + List specs = new ArrayList<>(); + + for (int i = 0; i < instances; i++) { + File file = new File(tmpDir, "apacheIgniteTestZk-" + i); + + if (file.isDirectory()) + deleteRecursively0(file); + else { + if (!file.mkdirs()) + throw new IgniteException("Failed to create directory for test Zookeeper server: " + file.getAbsolutePath()); + } + + specs.add(new InstanceSpec(file, -1, -1, -1, true, -1, -1, 500)); + } + + return new TestingCluster(specs); + } + + /** + * @param file File or directory to delete. + */ + private static void deleteRecursively0(File file) { + File[] files = file.listFiles(); + + if (files == null) + return; + + for (File f : files) { + if (f.isDirectory()) + deleteRecursively0(f); + else { + if (!f.delete()) + throw new IgniteException("Failed to delete file: " + f.getAbsolutePath()); + } + } + } + +} diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite1.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite1.java new file mode 100644 index 0000000000000..860488b1a926f --- /dev/null +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite1.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk; + +import junit.framework.TestSuite; +import org.apache.ignite.spi.discovery.zk.internal.ZookeeperClientTest; +import org.apache.ignite.spi.discovery.zk.internal.ZookeeperDiscoverySpiSaslSuccessfulAuthTest; +import org.apache.ignite.spi.discovery.zk.internal.ZookeeperDiscoverySpiTest; + +/** + * + */ +public class ZookeeperDiscoverySpiTestSuite1 extends TestSuite { + /** + * @return Test suite. + * @throws Exception Thrown in case of the failure. + */ + public static TestSuite suite() throws Exception { + System.setProperty("zookeeper.forceSync", "false"); + + TestSuite suite = new TestSuite("ZookeeperDiscoverySpi Test Suite"); + + suite.addTestSuite(ZookeeperClientTest.class); + suite.addTestSuite(ZookeeperDiscoverySpiTest.class); + suite.addTestSuite(ZookeeperDiscoverySpiSaslSuccessfulAuthTest.class); + + return suite; + } +} diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite2.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite2.java new file mode 100644 index 0000000000000..3775aa1df9b3d --- /dev/null +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpiTestSuite2.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk; + +import junit.framework.TestSuite; +import org.apache.curator.test.TestingCluster; +import org.apache.ignite.internal.ClusterNodeMetricsUpdateTest; +import org.apache.ignite.internal.IgniteClientReconnectCacheTest; +import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerAtomicTest; +import org.apache.ignite.internal.processors.cache.datastructures.IgniteClientDataStructuresTest; +import org.apache.ignite.internal.processors.cache.datastructures.partitioned.GridCachePartitionedNodeRestartTxSelfTest; +import org.apache.ignite.internal.processors.cache.datastructures.partitioned.GridCachePartitionedSequenceApiSelfTest; +import org.apache.ignite.internal.processors.cache.datastructures.replicated.GridCacheReplicatedSequenceApiSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePutRetryAtomicSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePutRetryTransactionalSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicMultiNodeFullApiSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeFullApiSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNodeRestartTest; +import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedAtomicMultiNodeFullApiSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedMultiNodeFullApiSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedNodeRestartSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQuerySelfTest; +import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicMultiJvmFullApiSelfTest; +import org.apache.ignite.internal.processors.cache.multijvm.GridCachePartitionedMultiJvmFullApiSelfTest; +import org.apache.ignite.internal.processors.continuous.GridEventConsumeSelfTest; + +/** + * Regular Ignite tests executed with {@link org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi}. + */ +public class ZookeeperDiscoverySpiTestSuite2 extends ZookeeperDiscoverySpiAbstractTestSuite { + /** */ + private static TestingCluster testingCluster; + + /** + * @return Test suite. + * @throws Exception Thrown in case of the failure. + */ + public static TestSuite suite() throws Exception { + System.setProperty("H2_JDBC_CONNECTIONS", "500"); // For multi-jvm tests. + + initSuite(); + + TestSuite suite = new TestSuite("ZookeeperDiscoverySpi Test Suite"); + + suite.addTestSuite(ZookeeperDiscoverySuitePreprocessorTest.class); + + suite.addTestSuite(GridCacheReplicatedNodeRestartSelfTest.class); + suite.addTestSuite(GridCachePartitionedNodeRestartTest.class); + + suite.addTestSuite(IgniteCacheEntryListenerWithZkDiscoAtomicTest.class); + + suite.addTestSuite(GridEventConsumeSelfTest.class); + + suite.addTestSuite(IgniteClientReconnectCacheTest.class); + + suite.addTestSuite(IgniteCachePutRetryAtomicSelfTest.class); + suite.addTestSuite(IgniteCachePutRetryTransactionalSelfTest.class); + + suite.addTestSuite(ClusterNodeMetricsUpdateTest.class); + + suite.addTestSuite(GridCachePartitionedMultiNodeFullApiSelfTest.class); + suite.addTestSuite(GridCacheReplicatedMultiNodeFullApiSelfTest.class); + + suite.addTestSuite(GridCacheAtomicMultiNodeFullApiSelfTest.class); + suite.addTestSuite(GridCacheReplicatedAtomicMultiNodeFullApiSelfTest.class); + + suite.addTestSuite(GridCachePartitionedNodeRestartTxSelfTest.class); + suite.addTestSuite(IgniteClientDataStructuresTest.class); + suite.addTestSuite(GridCacheReplicatedSequenceApiSelfTest.class); + suite.addTestSuite(GridCachePartitionedSequenceApiSelfTest.class); + + suite.addTestSuite(IgniteCacheReplicatedQuerySelfTest.class); + + suite.addTestSuite(GridCacheAtomicMultiJvmFullApiSelfTest.class); + suite.addTestSuite(GridCachePartitionedMultiJvmFullApiSelfTest.class); + + return suite; + } +} diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySuitePreprocessorTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySuitePreprocessorTest.java new file mode 100644 index 0000000000000..28cf17f017c42 --- /dev/null +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySuitePreprocessorTest.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk; + +import java.util.List; +import org.apache.ignite.Ignite; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.spi.discovery.DiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.config.GridTestProperties; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Sanity test verifying that configuration callback specified via + * {@link GridTestProperties#IGNITE_CFG_PREPROCESSOR_CLS} really works. + *

    + * This test should be run as part of {@link ZookeeperDiscoverySpiTestSuite2}. + */ +public class ZookeeperDiscoverySuitePreprocessorTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + // Test sets TcpDiscoverySpi, but it should be automatically changed to ZookeeperDiscoverySpi. + TcpDiscoverySpi spi = new TcpDiscoverySpi(); + + spi.setIpFinder(IP_FINDER); + + cfg.setDiscoverySpi(spi); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + super.afterTest(); + } + + /** + * @throws Exception If failed. + */ + public void testSpiConfigurationIsChanged() throws Exception { + startGrid(0); + + checkDiscoverySpi(1); + + startGrid(1); + + checkDiscoverySpi(2); + + startGridsMultiThreaded(2, 2); + + checkDiscoverySpi(4); + + startGrid(); + + checkDiscoverySpi(5); + } + + /** + * @param expNodes Expected nodes number. + * @throws Exception If failed. + */ + private void checkDiscoverySpi(int expNodes) throws Exception { + List nodes = G.allGrids(); + + assertEquals(expNodes, nodes.size()); + + for (Ignite node : nodes) { + DiscoverySpi spi = node.configuration().getDiscoverySpi(); + + assertTrue("Node should be started with " + ZookeeperDiscoverySpi.class.getName(), + spi instanceof ZookeeperDiscoverySpi); + } + + waitForTopology(expNodes); + } +} diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java new file mode 100644 index 0000000000000..e7cb97a39434f --- /dev/null +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClientTest.java @@ -0,0 +1,495 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.curator.test.TestingCluster; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteRunnable; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.zookeeper.AsyncCallback; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.data.Stat; + +/** + * + */ +public class ZookeeperClientTest extends GridCommonAbstractTest { + /** */ + private static final int SES_TIMEOUT = 60_000; + + /** */ + private TestingCluster zkCluster; + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + closeZK(); + + super.afterTest(); + } + + /** + * @param sesTimeout Session timeout. + * @return Client. + * @throws Exception If failed. + */ + private ZookeeperClient createClient(int sesTimeout) throws Exception { + return new ZookeeperClient(log, zkCluster.getConnectString(), sesTimeout, null); + } + + /** + * @throws Exception If failed. + */ + public void testSaveLargeValue() throws Exception { + startZK(1); + + final ZookeeperClient client = createClient(SES_TIMEOUT); + + byte[] data = new byte[1024 * 1024]; + + String basePath = "/ignite"; + + assertTrue(client.needSplitNodeData(basePath, data, 2)); + + List parts = client.splitNodeData(basePath, data, 2); + + assertTrue(parts.size() > 1); + + ZooKeeper zk = client.zk(); + + for (int i = 0; i < parts.size(); i++) { + byte[] part = parts.get(i); + + assertTrue(part.length > 0); + + String path0 = basePath + ":" + i; + + zk.create(path0, part, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + } + } + + /** + * @throws Exception If failed. + */ + public void testClose() throws Exception { + startZK(1); + + final ZookeeperClient client = createClient(SES_TIMEOUT); + + client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT); + + client.zk().close(); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Void call() throws Exception { + client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT); + + return null; + } + }, ZookeeperClientFailedException.class, null); + } + + /** + * @throws Exception If failed. + */ + public void testCreateAll() throws Exception { + startZK(1); + + ZookeeperClient client = createClient(SES_TIMEOUT); + + client.createIfNeeded("/apacheIgnite", null, CreateMode.PERSISTENT); + + List paths = new ArrayList<>(); + + paths.add("/apacheIgnite/1"); + paths.add("/apacheIgnite/2"); + paths.add("/apacheIgnite/3"); + + client.createAll(paths, CreateMode.PERSISTENT); + + assertEquals(3, client.getChildren("/apacheIgnite").size()); + } + + /** + * @throws Exception If failed. + */ + public void testDeleteAll() throws Exception { + startZK(1); + + ZookeeperClient client = createClient(SES_TIMEOUT); + + client.createIfNeeded("/apacheIgnite", null, CreateMode.PERSISTENT); + client.createIfNeeded("/apacheIgnite/1", null, CreateMode.PERSISTENT); + client.createIfNeeded("/apacheIgnite/2", null, CreateMode.PERSISTENT); + + client.deleteAll("/apacheIgnite", Arrays.asList("1", "2"), -1); + + assertTrue(client.getChildren("/apacheIgnite").isEmpty()); + + client.createIfNeeded("/apacheIgnite/1", null, CreateMode.PERSISTENT); + client.deleteAll("/apacheIgnite", Collections.singletonList("1"), -1); + + assertTrue(client.getChildren("/apacheIgnite").isEmpty()); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionLoss1() throws Exception { + ZookeeperClient client = new ZookeeperClient(log, "localhost:2200", 3000, null); + + try { + client.createIfNeeded("/apacheIgnite", null, CreateMode.PERSISTENT); + + fail(); + } + catch (ZookeeperClientFailedException e) { + info("Expected error: " + e); + } + } + + /** + * @throws Exception If failed. + */ + public void testConnectionLoss2() throws Exception { + startZK(1); + + ZookeeperClient client = createClient(3000); + + client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT); + + closeZK(); + + try { + client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT); + + fail(); + } + catch (ZookeeperClientFailedException e) { + info("Expected error: " + e); + } + } + + /** + * @throws Exception If failed. + */ + public void testConnectionLoss3() throws Exception { + startZK(1); + + CallbackFuture cb = new CallbackFuture(); + + ZookeeperClient client = new ZookeeperClient(log, zkCluster.getConnectString(), 3000, cb); + + client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT); + + closeZK(); + + final AtomicBoolean res = new AtomicBoolean(); + + client.getChildrenAsync("/apacheIgnite1", null, new AsyncCallback.Children2Callback() { + @Override public void processResult(int rc, String path, Object ctx, List children, Stat stat) { + if (rc == 0) + res.set(true); + } + }); + + cb.get(60_000); + + assertFalse(res.get()); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionLoss4() throws Exception { + startZK(1); + + CallbackFuture cb = new CallbackFuture(); + + final ZookeeperClient client = new ZookeeperClient(log, zkCluster.getConnectString(), 3000, cb); + + client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT); + + final CountDownLatch l = new CountDownLatch(1); + + client.getChildrenAsync("/apacheIgnite1", null, new AsyncCallback.Children2Callback() { + @Override public void processResult(int rc, String path, Object ctx, List children, Stat stat) { + closeZK(); + + try { + client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT); + } + catch (ZookeeperClientFailedException e) { + info("Expected error: " + e); + + l.countDown(); + } + catch (Exception e) { + fail("Unexpected error: " + e); + } + } + }); + + assertTrue(l.await(10, TimeUnit.SECONDS)); + + cb.get(); + } + + /** + * @throws Exception If failed. + */ + public void testReconnect1() throws Exception { + startZK(1); + + ZookeeperClient client = createClient(SES_TIMEOUT); + + client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT); + + zkCluster.getServers().get(0).stop(); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + U.sleep(2000); + + info("Restart zookeeper server"); + + zkCluster.getServers().get(0).restart(); + + info("Zookeeper server restarted"); + + return null; + } + }, "start-zk"); + + client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT); + + fut.get(); + } + + /** + * @throws Exception If failed. + */ + public void testReconnect1_Callback() throws Exception { + startZK(1); + + ZookeeperClient client = createClient(SES_TIMEOUT); + + client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT); + + zkCluster.getServers().get(0).stop(); + + final CountDownLatch l = new CountDownLatch(1); + + client.getChildrenAsync("/apacheIgnite1", null, new AsyncCallback.Children2Callback() { + @Override public void processResult(int rc, String path, Object ctx, List children, Stat stat) { + info("Callback: " + rc); + + if (rc == 0) + l.countDown(); + } + }); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + U.sleep(2000); + + info("Restart zookeeper server"); + + zkCluster.getServers().get(0).restart(); + + info("Zookeeper server restarted"); + + return null; + } + }, "start-zk"); + + assertTrue(l.await(10, TimeUnit.SECONDS)); + + fut.get(); + } + + /** + * @throws Exception If failed. + */ + public void testReconnect1_InCallback() throws Exception { + startZK(1); + + final ZookeeperClient client = createClient(SES_TIMEOUT); + + client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT); + + final CountDownLatch l = new CountDownLatch(1); + + client.getChildrenAsync("/apacheIgnite1", null, new AsyncCallback.Children2Callback() { + @Override public void processResult(int rc, String path, Object ctx, List children, Stat stat) { + try { + zkCluster.getServers().get(0).stop(); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + U.sleep(2000); + + info("Restart zookeeper server"); + + zkCluster.getServers().get(0).restart(); + + info("Zookeeper server restarted"); + + return null; + } + }, "start-zk"); + + client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT); + + l.countDown(); + + fut.get(); + } + catch (Exception e) { + fail("Unexpected error: " + e); + } + } + }); + + assertTrue(l.await(10, TimeUnit.SECONDS)); + } + + /** + * @throws Exception If failed. + */ + public void testReconnect2() throws Exception { + startZK(1); + + ZookeeperClient client = createClient(SES_TIMEOUT); + + client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT); + + zkCluster.getServers().get(0).restart(); + + client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT); + } + + /** + * @throws Exception If failed. + */ + public void testReconnect3() throws Exception { + startZK(3); + + ZookeeperClient client = createClient(SES_TIMEOUT); + + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + for (int i = 0; i < 30; i++) { + info("Iteration: " + i); + + int idx = rnd.nextInt(3); + + zkCluster.getServers().get(idx).restart(); + + doSleep(rnd.nextLong(100) + 1); + + client.createIfNeeded("/apacheIgnite" + i, null, CreateMode.PERSISTENT); + } + } + + /** + * @throws Exception If failed. + */ + public void testReconnect4() throws Exception { + startZK(3); + + ZookeeperClient client = new ZookeeperClient(log, + zkCluster.getServers().get(2).getInstanceSpec().getConnectString(), + 60_000, + null); + + client.createIfNeeded("/apacheIgnite1", null, CreateMode.PERSISTENT); + + zkCluster.getServers().get(0).stop(); + zkCluster.getServers().get(1).stop(); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + U.sleep(2000); + + info("Restart zookeeper server"); + + zkCluster.getServers().get(0).restart(); + + info("Zookeeper server restarted"); + + return null; + } + }, "start-zk"); + + client.createIfNeeded("/apacheIgnite2", null, CreateMode.PERSISTENT); + + fut.get(); + } + + /** + * @param instances Number of servers in ZK ensemble. + * @throws Exception If failed. + */ + private void startZK(int instances) throws Exception { + assert zkCluster == null; + + zkCluster = new TestingCluster(instances); + + zkCluster.start(); + } + + /** + * + */ + private void closeZK() { + if (zkCluster != null) { + try { + zkCluster.close(); + } + catch (Exception e) { + U.error(log, "Failed to stop Zookeeper client: " + e, e); + } + + zkCluster = null; + } + } + + /** + * + */ + private static class CallbackFuture extends GridFutureAdapter implements IgniteRunnable { + /** {@inheritDoc} */ + @Override public void run() { + onDone(); + } + } +} diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslAuthAbstractTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslAuthAbstractTest.java new file mode 100644 index 0000000000000..ac94bf2dd88a8 --- /dev/null +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslAuthAbstractTest.java @@ -0,0 +1,247 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ignite.spi.discovery.zk.internal; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Paths; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.zookeeper.client.ZooKeeperSaslClient; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.apache.zookeeper.server.ZKDatabase; +import org.apache.zookeeper.server.ZooKeeperServer; + +import static org.apache.curator.test.DirectoryUtils.deleteRecursively; + +/** + * Implements methods to prepare SASL tests infrastructure: jaas.conf files, starting up ZooKeeper server, + * clean up procedures when the test has finished etc. + */ +public abstract class ZookeeperDiscoverySpiSaslAuthAbstractTest extends GridCommonAbstractTest { + /** */ + private File tmpDir = createTmpDir(); + + /** */ + private static final String JAAS_CONF_FILE = "jaas.conf"; + + /** */ + private static final String AUTH_PROVIDER = "zookeeper.authProvider.1"; + + /** */ + private static final String SASL_CONFIG = "java.security.auth.login.config"; + + /** */ + private long joinTimeout = 2_000; + + /** */ + private long sesTimeout = 10_000; + + /** */ + private ServerCnxnFactory serverFactory; + + /** */ + private String hostPort = "localhost:2181"; + + /** */ + private int maxCnxns; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String instanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(instanceName); + + ZookeeperDiscoverySpi zkSpi = new ZookeeperDiscoverySpi(); + + if (joinTimeout != 0) + zkSpi.setJoinTimeout(joinTimeout); + + zkSpi.setSessionTimeout(sesTimeout > 0 ? sesTimeout : 10_000); + + zkSpi.setZkConnectionString(hostPort); + + cfg.setDiscoverySpi(zkSpi); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + prepareJaasConfigFile(); + + prepareSaslSystemProperties(); + + startZooKeeperServer(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopZooKeeperServer(); + + stopAllGrids(); + + clearSaslSystemProperties(); + + clearTmpDir(); + } + + /** */ + private void clearTmpDir() throws Exception { + deleteRecursively(tmpDir); + } + + /** */ + protected void clearSaslSystemProperties() { + System.clearProperty(AUTH_PROVIDER); + + System.clearProperty(SASL_CONFIG); + + System.clearProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY); + } + + /** + * @throws Exception If failed. + */ + private void prepareJaasConfigFile() throws Exception { + U.ensureDirectory(tmpDir, "Temp directory for JAAS configuration file wasn't created", null); + + File saslConfFile = new File(tmpDir, JAAS_CONF_FILE); + + FileWriter fwriter = new FileWriter(saslConfFile); + + writeServerConfigSection(fwriter, "validPassword"); + + writeClientConfigSection(fwriter, "ValidZookeeperClient", "validPassword"); + + writeClientConfigSection(fwriter, "InvalidZookeeperClient", "invalidPassword"); + + fwriter.close(); + } + + /** */ + private void prepareSaslSystemProperties() { + System.setProperty(SASL_CONFIG, Paths.get(tmpDir.getPath().toString(), JAAS_CONF_FILE).toString()); + + System.setProperty(AUTH_PROVIDER, "org.apache.zookeeper.server.auth.SASLAuthenticationProvider"); + } + + /** */ + private void writeClientConfigSection(FileWriter fwriter, String clientName, String pass) throws IOException { + fwriter.write(clientName + "{\n" + + " org.apache.zookeeper.server.auth.DigestLoginModule required\n" + + " username=\"zkUser\"\n" + + " password=\"" + pass + "\";\n" + + "};" + "\n"); + } + + /** */ + private void writeServerConfigSection(FileWriter fwriter, String pass) throws IOException { + fwriter.write("Server {\n" + + " org.apache.zookeeper.server.auth.DigestLoginModule required\n" + + " user_zkUser=\"" + pass + "\";\n" + + "};\n"); + } + + /** */ + private File createTmpDir() { + File jaasConfDir = Paths.get(System.getProperty("java.io.tmpdir"), "zk_disco_spi_test").toFile(); + + try { + U.ensureDirectory(jaasConfDir, "", null); + } + catch (IgniteCheckedException e) { + // ignored + } + + return jaasConfDir; + } + + /** */ + private void stopZooKeeperServer() throws Exception { + shutdownServerInstance(serverFactory); + serverFactory = null; + } + + /** */ + private void shutdownServerInstance(ServerCnxnFactory factory) + { + if (factory != null) { + ZKDatabase zkDb = null; + { + ZooKeeperServer zs = getServer(factory); + if (zs != null) + zkDb = zs.getZKDatabase(); + } + factory.shutdown(); + try { + if (zkDb != null) + zkDb.close(); + } catch (IOException ie) { + // ignore + } + } + } + + /** */ + private ZooKeeperServer getServer(ServerCnxnFactory fac) { + ZooKeeperServer zs = U.field(fac, "zkServer"); + + return zs; + } + + /** */ + private void startZooKeeperServer() throws Exception { + serverFactory = createNewServerInstance(serverFactory, hostPort, + maxCnxns); + startServerInstance(tmpDir, serverFactory); + } + + /** */ + private ServerCnxnFactory createNewServerInstance( + ServerCnxnFactory factory, String hostPort, int maxCnxns) + throws IOException { + final int port = getPort(hostPort); + + if (factory == null) + factory = ServerCnxnFactory.createFactory(port, maxCnxns); + + return factory; + } + + /** */ + private void startServerInstance(File dataDir, + ServerCnxnFactory factory) throws IOException, + InterruptedException { + ZooKeeperServer zks = new ZooKeeperServer(dataDir, dataDir, 3000); + factory.startup(zks); + } + + /** */ + private int getPort(String hostPort) { + String[] split = hostPort.split(":"); + String portstr = split[split.length-1]; + String[] pc = portstr.split("/"); + + if (pc.length > 1) + portstr = pc[0]; + + return Integer.parseInt(portstr); + } +} diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslFailedAuthTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslFailedAuthTest.java new file mode 100644 index 0000000000000..864ac96d72a2c --- /dev/null +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslFailedAuthTest.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ignite.spi.discovery.zk.internal; + +import org.apache.zookeeper.client.ZooKeeperSaslClient; +import org.junit.Assert; + +/** + * + */ +public class ZookeeperDiscoverySpiSaslFailedAuthTest extends ZookeeperDiscoverySpiSaslAuthAbstractTest { + /** + * @throws Exception If failed. + */ + public void testIgniteNodeWithInvalidPasswordFailsToJoin() throws Exception { + System.setProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY, + "InvalidZookeeperClient"); + + System.setProperty("IGNITE_ZOOKEEPER_DISCOVERY_MAX_RETRY_COUNT", Integer.toString(1)); + + try { + startGrid(0); + + Assert.fail("Ignite node with invalid password should fail on join."); + } + catch (Exception e) { + //ignored + } + } +} diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslSuccessfulAuthTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslSuccessfulAuthTest.java new file mode 100644 index 0000000000000..5ee0a4300af7a --- /dev/null +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiSaslSuccessfulAuthTest.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ignite.spi.discovery.zk.internal; + +import org.apache.zookeeper.client.ZooKeeperSaslClient; + +/** + * + */ +public class ZookeeperDiscoverySpiSaslSuccessfulAuthTest extends ZookeeperDiscoverySpiSaslAuthAbstractTest { + /** + * @throws Exception If failed. + */ + public void testIgniteNodesWithValidPasswordSuccessfullyJoins() throws Exception { + System.setProperty(ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY, + "ValidZookeeperClient"); + + startGrids(3); + + waitForTopology(3); + } + + /** + * @throws Exception If failed. + */ + public void testIgniteNodeWithoutSaslConfigurationSuccessfullyJoins() throws Exception { + //clearing SASL-related system properties that were set in beforeTest + clearSaslSystemProperties(); + + startGrid(0); + + waitForTopology(1); + } +} diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiTest.java new file mode 100644 index 0000000000000..fb12c3a5e9801 --- /dev/null +++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoverySpiTest.java @@ -0,0 +1,4847 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.spi.discovery.zk.internal; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.BitSet; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.TreeMap; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.curator.test.TestingCluster; +import org.apache.curator.test.TestingZooKeeperServer; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.CommunicationFailureContext; +import org.apache.ignite.configuration.CommunicationFailureResolver; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.WALMode; +import org.apache.ignite.events.DiscoveryEvent; +import org.apache.ignite.events.Event; +import org.apache.ignite.events.EventType; +import org.apache.ignite.internal.DiscoverySpiTestListener; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.IgnitionEx; +import org.apache.ignite.internal.TestRecordingCommunicationSpi; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; +import org.apache.ignite.internal.managers.discovery.CustomEventListener; +import org.apache.ignite.internal.managers.discovery.DiscoCache; +import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; +import org.apache.ignite.internal.managers.discovery.DiscoveryLocalJoinData; +import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpiInternalListener; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.GridCacheAbstractFullApiSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.TestCacheNodeExcludingFilter; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage; +import org.apache.ignite.internal.processors.security.SecurityContext; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.internal.util.typedef.T3; +import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.lang.IgniteOutClosure; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.marshaller.jdk.JdkMarshaller; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.security.SecurityCredentials; +import org.apache.ignite.plugin.security.SecurityPermission; +import org.apache.ignite.plugin.security.SecuritySubject; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.resources.LoggerResource; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.discovery.DiscoverySpi; +import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; +import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator; +import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi; +import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpiTestSuite2; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZKUtil; +import org.apache.zookeeper.ZkTestClientCnxnSocketNIO; +import org.apache.zookeeper.ZooKeeper; +import org.jetbrains.annotations.Nullable; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED; +import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_RECONNECTED; +import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; +import static org.apache.ignite.events.EventType.EVT_NODE_JOINED; +import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGNITE_INSTANCE_NAME; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2; +import static org.apache.ignite.spi.discovery.zk.internal.ZookeeperDiscoveryImpl.IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD; +import static org.apache.zookeeper.ZooKeeper.ZOOKEEPER_CLIENT_CNXN_SOCKET; + +/** + * + */ +@SuppressWarnings("deprecation") +public class ZookeeperDiscoverySpiTest extends GridCommonAbstractTest { + /** */ + private static final String IGNITE_ZK_ROOT = ZookeeperDiscoverySpi.DFLT_ROOT_PATH; + + /** */ + private static final int ZK_SRVS = 3; + + /** */ + private static TestingCluster zkCluster; + + /** To run test with real local ZK. */ + private static final boolean USE_TEST_CLUSTER = true; + + /** */ + private boolean client; + + /** */ + private static ThreadLocal clientThreadLoc = new ThreadLocal<>(); + + /** */ + private static ConcurrentHashMap> evts = new ConcurrentHashMap<>(); + + /** */ + private static volatile boolean err; + + /** */ + private boolean testSockNio; + + /** */ + private boolean testCommSpi; + + /** */ + private long sesTimeout; + + /** */ + private long joinTimeout; + + /** */ + private boolean clientReconnectDisabled; + + /** */ + private ConcurrentHashMap spis = new ConcurrentHashMap<>(); + + /** */ + private Map userAttrs; + + /** */ + private boolean dfltConsistenId; + + /** */ + private UUID nodeId; + + /** */ + private boolean persistence; + + /** */ + private IgniteOutClosure commFailureRslvr; + + /** */ + private IgniteOutClosure auth; + + /** */ + private String zkRootPath; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(final String igniteInstanceName) throws Exception { + if (testSockNio) + System.setProperty(ZOOKEEPER_CLIENT_CNXN_SOCKET, ZkTestClientCnxnSocketNIO.class.getName()); + + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + if (nodeId != null) + cfg.setNodeId(nodeId); + + if (!dfltConsistenId) + cfg.setConsistentId(igniteInstanceName); + + ZookeeperDiscoverySpi zkSpi = new ZookeeperDiscoverySpi(); + + if (joinTimeout != 0) + zkSpi.setJoinTimeout(joinTimeout); + + zkSpi.setSessionTimeout(sesTimeout > 0 ? sesTimeout : 10_000); + + zkSpi.setClientReconnectDisabled(clientReconnectDisabled); + + // Set authenticator for basic sanity tests. + if (auth != null) { + zkSpi.setAuthenticator(auth.apply()); + + zkSpi.setInternalListener(new IgniteDiscoverySpiInternalListener() { + @Override public void beforeJoin(ClusterNode locNode, IgniteLogger log) { + ZookeeperClusterNode locNode0 = (ZookeeperClusterNode)locNode; + + Map attrs = new HashMap<>(locNode0.getAttributes()); + + attrs.put(ATTR_SECURITY_CREDENTIALS, new SecurityCredentials(null, null, igniteInstanceName)); + + locNode0.setAttributes(attrs); + } + + @Override public boolean beforeSendCustomEvent(DiscoverySpi spi, IgniteLogger log, DiscoverySpiCustomMessage msg) { + return false; + } + }); + } + + spis.put(igniteInstanceName, zkSpi); + + if (USE_TEST_CLUSTER) { + assert zkCluster != null; + + zkSpi.setZkConnectionString(zkCluster.getConnectString()); + + if (zkRootPath != null) + zkSpi.setZkRootPath(zkRootPath); + } + else + zkSpi.setZkConnectionString("localhost:2181"); + + cfg.setDiscoverySpi(zkSpi); + + CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME); + + ccfg.setWriteSynchronizationMode(FULL_SYNC); + + cfg.setCacheConfiguration(ccfg); + + Boolean clientMode = clientThreadLoc.get(); + + if (clientMode != null) + cfg.setClientMode(clientMode); + else + cfg.setClientMode(client); + + if (userAttrs != null) + cfg.setUserAttributes(userAttrs); + + Map, int[]> lsnrs = new HashMap<>(); + + lsnrs.put(new IgnitePredicate() { + /** */ + @IgniteInstanceResource + private Ignite ignite; + + @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter") + @Override public boolean apply(Event evt) { + try { + DiscoveryEvent discoveryEvt = (DiscoveryEvent)evt; + + UUID locId = ((IgniteKernal)ignite).context().localNodeId(); + + Map nodeEvts = evts.get(locId); + + if (nodeEvts == null) { + Object old = evts.put(locId, nodeEvts = new TreeMap<>()); + + assertNull(old); + + synchronized (nodeEvts) { + DiscoveryLocalJoinData locJoin = ((IgniteKernal)ignite).context().discovery().localJoin(); + + nodeEvts.put(locJoin.event().topologyVersion(), locJoin.event()); + } + } + + synchronized (nodeEvts) { + DiscoveryEvent old = nodeEvts.put(discoveryEvt.topologyVersion(), discoveryEvt); + + assertNull(old); + } + } + catch (Throwable e) { + error("Unexpected error [evt=" + evt + ", err=" + e + ']', e); + + err = true; + } + + return true; + } + }, new int[]{EVT_NODE_JOINED, EVT_NODE_FAILED, EVT_NODE_LEFT}); + + cfg.setLocalEventListeners(lsnrs); + + if (persistence) { + DataStorageConfiguration memCfg = new DataStorageConfiguration() + .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setMaxSize(100 * 1024 * 1024). + setPersistenceEnabled(true)) + .setPageSize(1024) + .setWalMode(WALMode.LOG_ONLY); + + cfg.setDataStorageConfiguration(memCfg); + } + + if (testCommSpi) + cfg.setCommunicationSpi(new ZkTestCommunicationSpi()); + + if (commFailureRslvr != null) + cfg.setCommunicationFailureResolver(commFailureRslvr.apply()); + + return cfg; + } + + /** + * @param clientMode Client mode flag for started nodes. + */ + private void clientMode(boolean clientMode) { + client = clientMode; + } + + /** + * @param clientMode Client mode flag for nodes started from current thread. + */ + private void clientModeThreadLocal(boolean clientMode) { + clientThreadLoc.set(clientMode); + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + System.setProperty(ZookeeperDiscoveryImpl.IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_TIMEOUT, "1000"); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopZkCluster(); + + System.clearProperty(ZookeeperDiscoveryImpl.IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_TIMEOUT); + + super.afterTestsStopped(); + } + + /** + * + */ + private void stopZkCluster() { + if (zkCluster != null) { + try { + zkCluster.close(); + } + catch (Exception e) { + U.error(log, "Failed to stop Zookeeper client: " + e, e); + } + + zkCluster = null; + } + } + + /** + * + */ + private static void ackEveryEventSystemProperty() { + System.setProperty(IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD, "1"); + } + + /** + * + */ + private void clearAckEveryEventSystemProperty() { + System.setProperty(IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD, "1"); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + if (USE_TEST_CLUSTER && zkCluster == null) { + zkCluster = ZookeeperDiscoverySpiTestSuite2.createTestingCluster(ZK_SRVS); + + zkCluster.start(); + } + + reset(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + clearAckEveryEventSystemProperty(); + + try { + assertFalse("Unexpected error, see log for details", err); + + checkEventsConsistency(); + + checkInternalStructuresCleanup(); + + //TODO uncomment when https://issues.apache.org/jira/browse/IGNITE-8193 is fixed +// checkZkNodesCleanup(); + } + finally { + reset(); + + stopAllGrids(); + } + } + + /** + * @throws Exception If failed. + */ + private void checkInternalStructuresCleanup() throws Exception { + for (Ignite node : G.allGrids()) { + final AtomicReference res = GridTestUtils.getFieldValue(spi(node), "impl", "commErrProcFut"); + + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return res.get() == null; + } + }, 30_000); + + assertNull(res.get()); + } + } + + /** + * @throws Exception If failed. + */ + public void testZkRootNotExists() throws Exception { + zkRootPath = "/a/b/c"; + + for (int i = 0; i < 3; i++) { + reset(); + + startGridsMultiThreaded(5); + + waitForTopology(5); + + stopAllGrids(); + + checkEventsConsistency(); + } + } + + /** + * @throws Exception If failed. + */ + public void testMetadataUpdate() throws Exception { + startGrid(0); + + GridTestUtils.runMultiThreaded(new Callable() { + @Override public Void call() throws Exception { + ignite(0).configuration().getMarshaller().marshal(new C1()); + ignite(0).configuration().getMarshaller().marshal(new C2()); + + return null; + } + }, 64, "marshal"); + } + + /** + * @throws Exception If failed. + */ + public void testNodeAddresses() throws Exception { + startGridsMultiThreaded(3); + + clientMode(true); + + startGridsMultiThreaded(3, 3); + + waitForTopology(6); + + for (Ignite node : G.allGrids()) { + ClusterNode locNode0 = node.cluster().localNode(); + + assertTrue(locNode0.addresses().size() > 0); + assertTrue(locNode0.hostNames().size() > 0); + + for (ClusterNode node0 : node.cluster().nodes()) { + assertTrue(node0.addresses().size() > 0); + assertTrue(node0.hostNames().size() > 0); + } + } + } + + /** + * @throws Exception If failed. + */ + public void testSetConsistentId() throws Exception { + startGridsMultiThreaded(3); + + clientMode(true); + + startGridsMultiThreaded(3, 3); + + waitForTopology(6); + + for (Ignite node : G.allGrids()) { + ClusterNode locNode0 = node.cluster().localNode(); + + assertEquals(locNode0.attribute(ATTR_IGNITE_INSTANCE_NAME), + locNode0.consistentId()); + + for (ClusterNode node0 : node.cluster().nodes()) { + assertEquals(node0.attribute(ATTR_IGNITE_INSTANCE_NAME), + node0.consistentId()); + } + } + } + + /** + * @throws Exception If failed. + */ + public void testDefaultConsistentId() throws Exception { + dfltConsistenId = true; + + startGridsMultiThreaded(3); + + clientMode(true); + + startGridsMultiThreaded(3, 3); + + waitForTopology(6); + + for (Ignite node : G.allGrids()) { + ClusterNode locNode0 = node.cluster().localNode(); + + assertNotNull(locNode0.consistentId()); + + for (ClusterNode node0 : node.cluster().nodes()) + assertNotNull(node0.consistentId()); + } + } + + /** + * @throws Exception If failed. + */ + public void testClientNodesStatus() throws Exception { + startGrid(0); + + for (Ignite node : G.allGrids()) { + assertEquals(0, node.cluster().forClients().nodes().size()); + assertEquals(1, node.cluster().forServers().nodes().size()); + } + + clientMode(true); + + startGrid(1); + + for (Ignite node : G.allGrids()) { + assertEquals(1, node.cluster().forClients().nodes().size()); + assertEquals(1, node.cluster().forServers().nodes().size()); + } + + clientMode(false); + + startGrid(2); + + clientMode(true); + + startGrid(3); + + for (Ignite node : G.allGrids()) { + assertEquals(2, node.cluster().forClients().nodes().size()); + assertEquals(2, node.cluster().forServers().nodes().size()); + } + + stopGrid(1); + + waitForTopology(3); + + for (Ignite node : G.allGrids()) { + assertEquals(1, node.cluster().forClients().nodes().size()); + assertEquals(2, node.cluster().forServers().nodes().size()); + } + + stopGrid(2); + + waitForTopology(2); + + for (Ignite node : G.allGrids()) { + assertEquals(1, node.cluster().forClients().nodes().size()); + assertEquals(1, node.cluster().forServers().nodes().size()); + } + } + + /** + * @throws Exception If failed. + */ + public void _testLocalAuthenticationFails() throws Exception { + auth = ZkTestNodeAuthenticator.factory(getTestIgniteInstanceName(0)); + + Throwable err = GridTestUtils.assertThrows(log, new Callable() { + @Override public Void call() throws Exception { + startGrid(0); + + return null; + } + }, IgniteCheckedException.class, null); + + IgniteSpiException spiErr = X.cause(err, IgniteSpiException.class); + + assertNotNull(spiErr); + assertTrue(spiErr.getMessage().contains("Authentication failed for local node")); + + startGrid(1); + startGrid(2); + + checkTestSecuritySubject(2); + } + + /** + * @throws Exception If failed. + */ + public void testAuthentication() throws Exception { + auth = ZkTestNodeAuthenticator.factory(getTestIgniteInstanceName(1), + getTestIgniteInstanceName(5)); + + startGrid(0); + + checkTestSecuritySubject(1); + + { + clientMode(false); + checkStartFail(1); + + clientMode(true); + checkStartFail(1); + + clientMode(false); + } + + startGrid(2); + + checkTestSecuritySubject(2); + + stopGrid(2); + + checkTestSecuritySubject(1); + + startGrid(2); + + checkTestSecuritySubject(2); + + stopGrid(0); + + checkTestSecuritySubject(1); + + checkStartFail(1); + + clientMode(false); + + startGrid(3); + + clientMode(true); + + startGrid(4); + + clientMode(false); + + startGrid(0); + + checkTestSecuritySubject(4); + + checkStartFail(1); + checkStartFail(5); + + clientMode(true); + + checkStartFail(1); + checkStartFail(5); + } + + /** + * @param nodeIdx Node index. + */ + private void checkStartFail(final int nodeIdx) { + Throwable err = GridTestUtils.assertThrows(log, new Callable() { + @Override public Void call() throws Exception { + startGrid(nodeIdx); + + return null; + } + }, IgniteCheckedException.class, null); + + IgniteSpiException spiErr = X.cause(err, IgniteSpiException.class); + + assertNotNull(spiErr); + assertTrue(spiErr.getMessage().contains("Authentication failed")); + } + + /** + * @param expNodes Expected nodes number. + * @throws Exception If failed. + */ + private void checkTestSecuritySubject(int expNodes) throws Exception { + waitForTopology(expNodes); + + List nodes = G.allGrids(); + + JdkMarshaller marsh = new JdkMarshaller(); + + for (Ignite ignite : nodes) { + Collection nodes0 = ignite.cluster().nodes(); + + assertEquals(nodes.size(), nodes0.size()); + + for (ClusterNode node : nodes0) { + byte[] secSubj = node.attribute(ATTR_SECURITY_SUBJECT_V2); + + assertNotNull(secSubj); + + ZkTestNodeAuthenticator.TestSecurityContext secCtx = marsh.unmarshal(secSubj, null); + + assertEquals(node.attribute(ATTR_IGNITE_INSTANCE_NAME), secCtx.nodeName); + } + } + } + + /** + * @throws Exception If failed. + */ + public void testStopNode_1() throws Exception { + startGrids(5); + + waitForTopology(5); + + stopGrid(3); + + waitForTopology(4); + + startGrid(3); + + waitForTopology(5); + } + + /** + * @throws Exception If failed. + */ + public void testCustomEventsSimple1_SingleNode() throws Exception { + ackEveryEventSystemProperty(); + + Ignite srv0 = startGrid(0); + + srv0.createCache(new CacheConfiguration<>("c1")); + + waitForEventsAcks(srv0); + } + + /** + * @throws Exception If failed. + */ + public void testCustomEventsSimple1_5_Nodes() throws Exception { + ackEveryEventSystemProperty(); + + Ignite srv0 = startGrids(5); + + srv0.createCache(new CacheConfiguration<>("c1")); + + awaitPartitionMapExchange(); + + waitForEventsAcks(srv0); + } + + /** + * @throws Exception If failed. + */ + public void testCustomEvents_FastStopProcess_1() throws Exception { + customEvents_FastStopProcess(1, 0); + } + + /** + * @throws Exception If failed. + */ + public void testCustomEvents_FastStopProcess_2() throws Exception { + customEvents_FastStopProcess(5, 5); + } + + /** + * @param srvs Servers number. + * @param clients Clients number. + * @throws Exception If failed. + */ + private void customEvents_FastStopProcess(int srvs, int clients) throws Exception { + ackEveryEventSystemProperty(); + + Map>> rcvdMsgs = + new ConcurrentHashMap<>(); + + Ignite crd = startGrid(0); + + UUID crdId = crd.cluster().localNode().id(); + + if (srvs > 1) + startGridsMultiThreaded(1, srvs - 1); + + if (clients > 0) { + client = true; + + startGridsMultiThreaded(srvs, clients); + } + + awaitPartitionMapExchange(); + + List nodes = G.allGrids(); + + assertEquals(srvs + clients, nodes.size()); + + for (Ignite node : nodes) + registerTestEventListeners(node, rcvdMsgs); + + int payload = 0; + + AffinityTopologyVersion topVer = ((IgniteKernal)crd).context().discovery().topologyVersionEx(); + + for (Ignite node : nodes) { + UUID sndId = node.cluster().localNode().id(); + + info("Send from node: " + sndId); + + GridDiscoveryManager discoveryMgr = ((IgniteKernal)node).context().discovery(); + + { + List> expCrdMsgs = new ArrayList<>(); + List> expNodesMsgs = Collections.emptyList(); + + TestFastStopProcessCustomMessage msg = new TestFastStopProcessCustomMessage(false, payload++); + + expCrdMsgs.add(new T3(topVer, sndId, msg)); + + discoveryMgr.sendCustomEvent(msg); + + doSleep(200); // Wait some time to check extra messages are not received. + + checkEvents(crd, rcvdMsgs, expCrdMsgs); + + for (Ignite node0 : nodes) { + if (node0 != crd) + checkEvents(node0, rcvdMsgs, expNodesMsgs); + } + + rcvdMsgs.clear(); + } + { + List> expCrdMsgs = new ArrayList<>(); + List> expNodesMsgs = new ArrayList<>(); + + TestFastStopProcessCustomMessage msg = new TestFastStopProcessCustomMessage(true, payload++); + + expCrdMsgs.add(new T3(topVer, sndId, msg)); + + discoveryMgr.sendCustomEvent(msg); + + TestFastStopProcessCustomMessageAck ackMsg = new TestFastStopProcessCustomMessageAck(msg.payload); + + expCrdMsgs.add(new T3(topVer, crdId, ackMsg)); + expNodesMsgs.add(new T3(topVer, crdId, ackMsg)); + + doSleep(200); // Wait some time to check extra messages are not received. + + checkEvents(crd, rcvdMsgs, expCrdMsgs); + + for (Ignite node0 : nodes) { + if (node0 != crd) + checkEvents(node0, rcvdMsgs, expNodesMsgs); + } + + rcvdMsgs.clear(); + } + + waitForEventsAcks(crd); + } + } + + /** + * @param node Node to check. + * @param rcvdMsgs Received messages. + * @param expMsgs Expected messages. + * @throws Exception If failed. + */ + private void checkEvents( + Ignite node, + final Map>> rcvdMsgs, + final List> expMsgs) throws Exception { + final UUID nodeId = node.cluster().localNode().id(); + + assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + List> msgs = rcvdMsgs.get(nodeId); + + int size = msgs == null ? 0 : msgs.size(); + + return size >= expMsgs.size(); + } + }, 5000)); + + List> msgs = rcvdMsgs.get(nodeId); + + if (msgs == null) + msgs = Collections.emptyList(); + + assertEqualsCollections(expMsgs, msgs); + } + + /** + * @param node Node. + * @param rcvdMsgs Map to store received events. + */ + private void registerTestEventListeners(Ignite node, + final Map>> rcvdMsgs) { + GridDiscoveryManager discoveryMgr = ((IgniteKernal)node).context().discovery(); + + final UUID nodeId = node.cluster().localNode().id(); + + discoveryMgr.setCustomEventListener(TestFastStopProcessCustomMessage.class, + new CustomEventListener() { + @Override public void onCustomEvent(AffinityTopologyVersion topVer, ClusterNode snd, TestFastStopProcessCustomMessage msg) { + List> list = rcvdMsgs.get(nodeId); + + if (list == null) + rcvdMsgs.put(nodeId, list = new ArrayList<>()); + + list.add(new T3<>(topVer, snd.id(), (DiscoveryCustomMessage)msg)); + } + } + ); + discoveryMgr.setCustomEventListener(TestFastStopProcessCustomMessageAck.class, + new CustomEventListener() { + @Override public void onCustomEvent(AffinityTopologyVersion topVer, ClusterNode snd, TestFastStopProcessCustomMessageAck msg) { + List> list = rcvdMsgs.get(nodeId); + + if (list == null) + rcvdMsgs.put(nodeId, list = new ArrayList<>()); + + list.add(new T3<>(topVer, snd.id(), (DiscoveryCustomMessage)msg)); + } + } + ); + } + + /** + * @throws Exception If failed. + */ + public void testSegmentation1() throws Exception { + sesTimeout = 2000; + testSockNio = true; + + Ignite node0 = startGrid(0); + + final CountDownLatch l = new CountDownLatch(1); + + node0.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event evt) { + l.countDown(); + + return false; + } + }, EventType.EVT_NODE_SEGMENTED); + + ZkTestClientCnxnSocketNIO c0 = ZkTestClientCnxnSocketNIO.forNode(node0); + + c0.closeSocket(true); + + for (int i = 0; i < 10; i++) { + Thread.sleep(1_000); + + if (l.getCount() == 0) + break; + } + + info("Allow connect"); + + c0.allowConnect(); + + assertTrue(l.await(10, TimeUnit.SECONDS)); + } + + /** + * @throws Exception If failed. + */ + public void testSegmentation2() throws Exception { + sesTimeout = 2000; + + Ignite node0 = startGrid(0); + + final CountDownLatch l = new CountDownLatch(1); + + node0.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event evt) { + l.countDown(); + + return false; + } + }, EventType.EVT_NODE_SEGMENTED); + + try { + zkCluster.close(); + + assertTrue(l.await(10, TimeUnit.SECONDS)); + } + finally { + zkCluster = ZookeeperDiscoverySpiTestSuite2.createTestingCluster(ZK_SRVS); + + zkCluster.start(); + } + } + + /** + * @throws Exception If failed. + */ + public void testSegmentation3() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8183"); + + sesTimeout = 5000; + + Ignite node0 = startGrid(0); + + final CountDownLatch l = new CountDownLatch(1); + + node0.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event evt) { + l.countDown(); + + return false; + } + }, EventType.EVT_NODE_SEGMENTED); + + List srvs = zkCluster.getServers(); + + assertEquals(3, srvs.size()); + + try { + srvs.get(0).stop(); + srvs.get(1).stop(); + + assertTrue(l.await(20, TimeUnit.SECONDS)); + } + finally { + zkCluster.close(); + + zkCluster = ZookeeperDiscoverySpiTestSuite2.createTestingCluster(ZK_SRVS); + + zkCluster.start(); + } + } + + /** + * @throws Exception If failed. + */ + public void testQuorumRestore() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8180"); + + sesTimeout = 15_000; + + startGrids(3); + + waitForTopology(3); + + List srvs = zkCluster.getServers(); + + assertEquals(3, srvs.size()); + + try { + srvs.get(0).stop(); + srvs.get(1).stop(); + + U.sleep(2000); + + srvs.get(1).restart(); + + U.sleep(4000); + + startGrid(4); + + waitForTopology(4); + } + finally { + zkCluster.close(); + + zkCluster = ZookeeperDiscoverySpiTestSuite2.createTestingCluster(ZK_SRVS); + + zkCluster.start(); + } + } + + /** + * @throws Exception If failed. + */ + public void testConnectionRestore1() throws Exception { + testSockNio = true; + + Ignite node0 = startGrid(0); + + ZkTestClientCnxnSocketNIO c0 = ZkTestClientCnxnSocketNIO.forNode(node0); + + c0.closeSocket(false); + + startGrid(1); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionRestore2() throws Exception { + testSockNio = true; + + Ignite node0 = startGrid(0); + + ZkTestClientCnxnSocketNIO c0 = ZkTestClientCnxnSocketNIO.forNode(node0); + + c0.closeSocket(false); + + startGridsMultiThreaded(1, 5); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionRestore_NonCoordinator1() throws Exception { + connectionRestore_NonCoordinator(false); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionRestore_NonCoordinator2() throws Exception { + connectionRestore_NonCoordinator(true); + } + + /** + * @param failWhenDisconnected {@code True} if fail node while another node is disconnected. + * @throws Exception If failed. + */ + private void connectionRestore_NonCoordinator(boolean failWhenDisconnected) throws Exception { + testSockNio = true; + + Ignite node0 = startGrid(0); + Ignite node1 = startGrid(1); + + ZkTestClientCnxnSocketNIO c1 = ZkTestClientCnxnSocketNIO.forNode(node1); + + c1.closeSocket(true); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Void call() { + try { + startGrid(2); + } + catch (Exception e) { + info("Start error: " + e); + } + + return null; + } + }, "start-node"); + + checkEvents(node0, joinEvent(3)); + + if (failWhenDisconnected) { + ZookeeperDiscoverySpi spi = spis.get(getTestIgniteInstanceName(2)); + + closeZkClient(spi); + + checkEvents(node0, failEvent(4)); + } + + c1.allowConnect(); + + checkEvents(ignite(1), joinEvent(3)); + + if (failWhenDisconnected) { + checkEvents(ignite(1), failEvent(4)); + + IgnitionEx.stop(getTestIgniteInstanceName(2), true, true); + } + + fut.get(); + + waitForTopology(failWhenDisconnected ? 2 : 3); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionRestore_Coordinator1() throws Exception { + connectionRestore_Coordinator(1, 1, 0); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionRestore_Coordinator1_1() throws Exception { + connectionRestore_Coordinator(1, 1, 1); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionRestore_Coordinator2() throws Exception { + connectionRestore_Coordinator(1, 3, 0); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionRestore_Coordinator3() throws Exception { + connectionRestore_Coordinator(3, 3, 0); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionRestore_Coordinator4() throws Exception { + connectionRestore_Coordinator(3, 3, 1); + } + + /** + * @param initNodes Number of initially started nodes. + * @param startNodes Number of nodes to start after coordinator loose connection. + * @param failCnt Number of nodes to stop after coordinator loose connection. + * @throws Exception If failed. + */ + private void connectionRestore_Coordinator(final int initNodes, int startNodes, int failCnt) throws Exception { + sesTimeout = 30_000; + testSockNio = true; + + Ignite node0 = startGrids(initNodes); + + ZkTestClientCnxnSocketNIO c0 = ZkTestClientCnxnSocketNIO.forNode(node0); + + c0.closeSocket(true); + + final AtomicInteger nodeIdx = new AtomicInteger(initNodes); + + IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Callable() { + @Override public Void call() { + try { + startGrid(nodeIdx.getAndIncrement()); + } + catch (Exception e) { + error("Start failed: " + e); + } + + return null; + } + }, startNodes, "start-node"); + + int cnt = 0; + + DiscoveryEvent[] expEvts = new DiscoveryEvent[startNodes - failCnt]; + + int expEvtCnt = 0; + + sesTimeout = 1000; + + List blockedC = new ArrayList<>(); + + final List failedZkNodes = new ArrayList<>(failCnt); + + for (int i = initNodes; i < initNodes + startNodes; i++) { + final ZookeeperDiscoverySpi spi = waitSpi(getTestIgniteInstanceName(i)); + + assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + Object spiImpl = GridTestUtils.getFieldValue(spi, "impl"); + + if (spiImpl == null) + return false; + + long internalOrder = GridTestUtils.getFieldValue(spiImpl, "rtState", "internalOrder"); + + return internalOrder > 0; + } + }, 10_000)); + + if (cnt++ < failCnt) { + ZkTestClientCnxnSocketNIO c = ZkTestClientCnxnSocketNIO.forNode(getTestIgniteInstanceName(i)); + + c.closeSocket(true); + + blockedC.add(c); + + failedZkNodes.add(aliveZkNodePath(spi)); + } + else { + expEvts[expEvtCnt] = joinEvent(initNodes + expEvtCnt + 1); + + expEvtCnt++; + } + } + + waitNoAliveZkNodes(log, zkCluster.getConnectString(), failedZkNodes, 30_000); + + c0.allowConnect(); + + for (ZkTestClientCnxnSocketNIO c : blockedC) + c.allowConnect(); + + if (expEvts.length > 0) { + for (int i = 0; i < initNodes; i++) + checkEvents(ignite(i), expEvts); + } + + fut.get(); + + waitForTopology(initNodes + startNodes - failCnt); + } + + /** + * @param node Node. + * @return Corresponding znode. + */ + private static String aliveZkNodePath(Ignite node) { + return aliveZkNodePath(node.configuration().getDiscoverySpi()); + } + + /** + * @param spi SPI. + * @return Znode related to given SPI. + */ + private static String aliveZkNodePath(DiscoverySpi spi) { + String path = GridTestUtils.getFieldValue(spi, "impl", "rtState", "locNodeZkPath"); + + return path.substring(path.lastIndexOf('/') + 1); + } + + /** + * @param log Logger. + * @param connectString Zookeeper connect string. + * @param failedZkNodes Znodes which should be removed. + * @param timeout Timeout. + * @throws Exception If failed. + */ + private static void waitNoAliveZkNodes(final IgniteLogger log, + String connectString, + final List failedZkNodes, + long timeout) + throws Exception + { + final ZookeeperClient zkClient = new ZookeeperClient(log, connectString, 10_000, null); + + try { + assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + try { + List c = zkClient.getChildren(IGNITE_ZK_ROOT + "/" + ZkIgnitePaths.ALIVE_NODES_DIR); + + for (String failedZkNode : failedZkNodes) { + if (c.contains(failedZkNode)) { + log.info("Alive node is not removed [node=" + failedZkNode + ", all=" + c + ']'); + + return false; + } + } + + return true; + } + catch (Exception e) { + e.printStackTrace(); + + fail(); + + return true; + } + } + }, timeout)); + } + finally { + zkClient.close(); + } + } + + /** + * @throws Exception If failed. + */ + public void testConcurrentStartWithClient() throws Exception { + final int NODES = 20; + + for (int i = 0; i < 3; i++) { + info("Iteration: " + i); + + final int srvIdx = ThreadLocalRandom.current().nextInt(NODES); + + final AtomicInteger idx = new AtomicInteger(); + + GridTestUtils.runMultiThreaded(new Callable() { + @Override public Void call() throws Exception { + int threadIdx = idx.getAndIncrement(); + + clientModeThreadLocal(threadIdx == srvIdx || ThreadLocalRandom.current().nextBoolean()); + + startGrid(threadIdx); + + return null; + } + }, NODES, "start-node"); + + waitForTopology(NODES); + + stopAllGrids(); + + checkEventsConsistency(); + + evts.clear(); + } + } + + /** + * @throws Exception If failed. + */ + public void testConcurrentStart() throws Exception { + final int NODES = 20; + + for (int i = 0; i < 3; i++) { + info("Iteration: " + i); + + final AtomicInteger idx = new AtomicInteger(); + + final CyclicBarrier b = new CyclicBarrier(NODES); + + GridTestUtils.runMultiThreaded(new Callable() { + @Override public Void call() throws Exception { + b.await(); + + int threadIdx = idx.getAndIncrement(); + + startGrid(threadIdx); + + return null; + } + }, NODES, "start-node"); + + waitForTopology(NODES); + + stopAllGrids(); + + checkEventsConsistency(); + + evts.clear(); + } + } + + /** + * @throws Exception If failed. + */ + public void testConcurrentStartStop1() throws Exception { + concurrentStartStop(1); + } + + /** + * @throws Exception If failed. + */ + public void testConcurrentStartStop2() throws Exception { + concurrentStartStop(5); + } + + /** + * @throws Exception If failed. + */ + public void testConcurrentStartStop2_EventsThrottle() throws Exception { + System.setProperty(ZookeeperDiscoveryImpl.IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS, "1"); + + try { + concurrentStartStop(5); + } + finally { + System.clearProperty(ZookeeperDiscoveryImpl.IGNITE_ZOOKEEPER_DISCOVERY_SPI_MAX_EVTS); + } + } + + /** + * @param initNodes Number of initially started nnodes. + * @throws Exception If failed. + */ + private void concurrentStartStop(final int initNodes) throws Exception { + startGrids(initNodes); + + final int NODES = 5; + + long topVer = initNodes; + + for (int i = 0; i < 10; i++) { + info("Iteration: " + i); + + DiscoveryEvent[] expEvts = new DiscoveryEvent[NODES]; + + startGridsMultiThreaded(initNodes, NODES); + + for (int j = 0; j < NODES; j++) + expEvts[j] = joinEvent(++topVer); + + checkEvents(ignite(0), expEvts); + + checkEventsConsistency(); + + final CyclicBarrier b = new CyclicBarrier(NODES); + + GridTestUtils.runMultiThreaded(new IgniteInClosure() { + @Override public void apply(Integer idx) { + try { + b.await(); + + stopGrid(initNodes + idx); + } + catch (Exception e) { + e.printStackTrace(); + + fail(); + } + } + }, NODES, "stop-node"); + + for (int j = 0; j < NODES; j++) + expEvts[j] = failEvent(++topVer); + + checkEventsConsistency(); + } + } + + /** + * @throws Exception If failed. + */ + public void testClusterRestart() throws Exception { + startGridsMultiThreaded(3, false); + + stopAllGrids(); + + evts.clear(); + + startGridsMultiThreaded(3, false); + + waitForTopology(3); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionRestore4() throws Exception { + testSockNio = true; + + Ignite node0 = startGrid(0); + + ZkTestClientCnxnSocketNIO c0 = ZkTestClientCnxnSocketNIO.forNode(node0); + + c0.closeSocket(false); + + startGrid(1); + } + + /** + * @throws Exception If failed. + */ + public void testStartStop_1_Node() throws Exception { + startGrid(0); + + waitForTopology(1); + + stopGrid(0); + } + + /** + * @throws Exception If failed. + */ + public void testRestarts_2_Nodes() throws Exception { + startGrid(0); + + for (int i = 0; i < 10; i++) { + info("Iteration: " + i); + + startGrid(1); + + waitForTopology(2); + + stopGrid(1); + } + } + + /** + * @throws Exception If failed. + */ + public void testStartStop_2_Nodes_WithCache() throws Exception { + startGrids(2); + + for (Ignite node : G.allGrids()) { + IgniteCache cache = node.cache(DEFAULT_CACHE_NAME); + + assertNotNull(cache); + + for (int i = 0; i < 100; i++) { + cache.put(i, node.name()); + + assertEquals(node.name(), cache.get(i)); + } + } + + awaitPartitionMapExchange(); + } + + /** + * @throws Exception If failed. + */ + public void testStartStop_2_Nodes() throws Exception { + ackEveryEventSystemProperty(); + + startGrid(0); + + waitForTopology(1); + + startGrid(1); + + waitForTopology(2); + + for (Ignite node : G.allGrids()) + node.compute().broadcast(new DummyCallable(null)); + + awaitPartitionMapExchange(); + + waitForEventsAcks(ignite(0)); + } + + /** + * @throws Exception If failed. + */ + public void testMultipleClusters() throws Exception { + Ignite c0 = startGrid(0); + + zkRootPath = "/cluster2"; + + Ignite c1 = startGridsMultiThreaded(1, 5); + + zkRootPath = "/cluster3"; + + Ignite c2 = startGridsMultiThreaded(6, 3); + + checkNodesNumber(c0, 1); + checkNodesNumber(c1, 5); + checkNodesNumber(c2, 3); + + stopGrid(2); + + checkNodesNumber(c0, 1); + checkNodesNumber(c1, 4); + checkNodesNumber(c2, 3); + + for (int i = 0; i < 3; i++) + stopGrid(i + 6); + + checkNodesNumber(c0, 1); + checkNodesNumber(c1, 4); + + c2 = startGridsMultiThreaded(6, 2); + + checkNodesNumber(c0, 1); + checkNodesNumber(c1, 4); + checkNodesNumber(c2, 2); + + evts.clear(); + } + + /** + * @param node Node. + * @param expNodes Expected node in cluster. + * @throws Exception If failed. + */ + private void checkNodesNumber(final Ignite node, final int expNodes) throws Exception { + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return node.cluster().nodes().size() == expNodes; + } + }, 5000); + + assertEquals(expNodes, node.cluster().nodes().size()); + } + + /** + * @throws Exception If failed. + */ + public void testStartStop1() throws Exception { + ackEveryEventSystemProperty(); + + startGridsMultiThreaded(5, false); + + waitForTopology(5); + + awaitPartitionMapExchange(); + + waitForEventsAcks(ignite(0)); + + stopGrid(0); + + waitForTopology(4); + + for (Ignite node : G.allGrids()) + node.compute().broadcast(new DummyCallable(null)); + + startGrid(0); + + waitForTopology(5); + + awaitPartitionMapExchange(); + + waitForEventsAcks(grid(CU.oldest(ignite(1).cluster().nodes()))); + } + + /** + * @throws Exception If failed. + */ + public void testStartStop3() throws Exception { + startGrids(4); + + awaitPartitionMapExchange(); + + stopGrid(0); + + startGrid(5); + + awaitPartitionMapExchange(); + } + + /** + * @throws Exception If failed. + */ + public void testStartStop4() throws Exception { + startGrids(6); + + awaitPartitionMapExchange(); + + stopGrid(2); + + if (ThreadLocalRandom.current().nextBoolean()) + awaitPartitionMapExchange(); + + stopGrid(1); + + if (ThreadLocalRandom.current().nextBoolean()) + awaitPartitionMapExchange(); + + stopGrid(0); + + if (ThreadLocalRandom.current().nextBoolean()) + awaitPartitionMapExchange(); + + startGrid(7); + + awaitPartitionMapExchange(); + } + + /** + * @throws Exception If failed. + */ + public void testStartStop2() throws Exception { + startGridsMultiThreaded(10, false); + + GridTestUtils.runMultiThreaded(new IgniteInClosure() { + @Override public void apply(Integer idx) { + stopGrid(idx); + } + }, 3, "stop-node-thread"); + + waitForTopology(7); + + startGridsMultiThreaded(0, 3); + + waitForTopology(10); + } + + /** + * @throws Exception If failed. + */ + public void testStartStopWithClients() throws Exception { + final int SRVS = 3; + + startGrids(SRVS); + + clientMode(true); + + final int THREADS = 30; + + for (int i = 0; i < 5; i++) { + info("Iteration: " + i); + + startGridsMultiThreaded(SRVS, THREADS); + + waitForTopology(SRVS + THREADS); + + GridTestUtils.runMultiThreaded(new IgniteInClosure() { + @Override public void apply(Integer idx) { + stopGrid(idx + SRVS); + } + }, THREADS, "stop-node"); + + waitForTopology(SRVS); + + checkEventsConsistency(); + } + } + + /** + * @throws Exception If failed. + */ + public void testTopologyChangeMultithreaded() throws Exception { + topologyChangeWithRestarts(false, false); + } + + /** + * @throws Exception If failed. + */ + public void testTopologyChangeMultithreaded_RestartZk() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8184"); + + try { + topologyChangeWithRestarts(true, false); + } + finally { + zkCluster.stop(); + + zkCluster = null; + } + } + + /** + * @throws Exception If failed. + */ + public void testTopologyChangeMultithreaded_RestartZk_CloseClients() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8184"); + + try { + topologyChangeWithRestarts(true, true); + } + finally { + zkCluster.stop(); + + zkCluster = null; + } + } + + /** + * @param restartZk If {@code true} in background restarts on of ZK servers. + * @param closeClientSock If {@code true} in background closes zk clients' sockets. + * @throws Exception If failed. + */ + private void topologyChangeWithRestarts(boolean restartZk, boolean closeClientSock) throws Exception { + sesTimeout = 30_000; + + if (closeClientSock) + testSockNio = true; + + long stopTime = System.currentTimeMillis() + 60_000; + + AtomicBoolean stop = new AtomicBoolean(); + + IgniteInternalFuture fut1 = null; + + IgniteInternalFuture fut2 = null; + + try { + fut1 = restartZk ? startRestartZkServers(stopTime, stop) : null; + fut2 = closeClientSock ? startCloseZkClientSocket(stopTime, stop) : null; + + int INIT_NODES = 10; + + startGridsMultiThreaded(INIT_NODES); + + final int MAX_NODES = 20; + + final List startedNodes = new ArrayList<>(); + + for (int i = 0; i < INIT_NODES; i++) + startedNodes.add(i); + + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + final AtomicInteger startIdx = new AtomicInteger(INIT_NODES); + + while (System.currentTimeMillis() < stopTime) { + if (startedNodes.size() >= MAX_NODES) { + int stopNodes = rnd.nextInt(5) + 1; + + log.info("Next, stop nodes: " + stopNodes); + + final List idxs = new ArrayList<>(); + + while (idxs.size() < stopNodes) { + Integer stopIdx = rnd.nextInt(startedNodes.size()); + + if (!idxs.contains(stopIdx)) + idxs.add(startedNodes.get(stopIdx)); + } + + GridTestUtils.runMultiThreaded(new IgniteInClosure() { + @Override public void apply(Integer threadIdx) { + int stopNodeIdx = idxs.get(threadIdx); + + info("Stop node: " + stopNodeIdx); + + stopGrid(stopNodeIdx); + } + }, stopNodes, "stop-node"); + + startedNodes.removeAll(idxs); + } + else { + int startNodes = rnd.nextInt(5) + 1; + + log.info("Next, start nodes: " + startNodes); + + GridTestUtils.runMultiThreaded(new Callable() { + @Override public Void call() throws Exception { + int idx = startIdx.incrementAndGet(); + + log.info("Start node: " + idx); + + startGrid(idx); + + synchronized (startedNodes) { + startedNodes.add(idx); + } + + return null; + } + }, startNodes, "start-node"); + } + + U.sleep(rnd.nextInt(100) + 1); + } + } + finally { + stop.set(true); + } + + if (fut1 != null) + fut1.get(); + + if (fut2 != null) + fut2.get(); + } + + /** + * @throws Exception If failed. + */ + public void testRandomTopologyChanges() throws Exception { + randomTopologyChanges(false, false); + } + + /** + * @throws Exception If failed. + */ + private void checkZkNodesCleanup() throws Exception { + final ZookeeperClient zkClient = new ZookeeperClient(getTestResources().getLogger(), + zkCluster.getConnectString(), + 30_000, + null); + + final String basePath = IGNITE_ZK_ROOT + "/"; + + final String aliveDir = basePath + ZkIgnitePaths.ALIVE_NODES_DIR + "/"; + + try { + List znodes = listSubTree(zkClient.zk(), IGNITE_ZK_ROOT); + + boolean foundAlive = false; + + for (String znode : znodes) { + if (znode.startsWith(aliveDir)) { + foundAlive = true; + + break; + } + } + + assertTrue(foundAlive); // Sanity check to make sure we check correct directory. + + assertTrue("Failed to wait for unused znodes cleanup", GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + try { + List znodes = listSubTree(zkClient.zk(), IGNITE_ZK_ROOT); + + for (String znode : znodes) { + if (znode.startsWith(aliveDir) || znode.length() < basePath.length()) + continue; + + znode = znode.substring(basePath.length()); + + if (!znode.contains("/")) // Ignore roots. + continue; + + // TODO ZK: https://issues.apache.org/jira/browse/IGNITE-8193 + if (znode.startsWith("jd/")) + continue; + + log.info("Found unexpected znode: " + znode); + + return false; + } + + return true; + } + catch (Exception e) { + error("Unexpected error: " + e, e); + + fail("Unexpected error: " + e); + } + + return false; + } + }, 10_000)); + } + finally { + zkClient.close(); + } + } + + /** + * @throws Exception If failed. + */ + public void testRandomTopologyChanges_RestartZk() throws Exception { + randomTopologyChanges(true, false); + } + + /** + * @throws Exception If failed. + */ + public void testRandomTopologyChanges_CloseClients() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8182"); + + randomTopologyChanges(false, true); + } + + /** + * @throws Exception If failed. + */ + public void testDeployService1() throws Exception { + startGridsMultiThreaded(3); + + grid(0).services(grid(0).cluster()).deployNodeSingleton("test", new GridCacheAbstractFullApiSelfTest.DummyServiceImpl()); + } + + /** + * @throws Exception If failed. + */ + public void testDeployService2() throws Exception { + clientMode(false); + + startGrid(0); + + clientMode(true); + + startGrid(1); + + grid(0).services(grid(0).cluster()).deployNodeSingleton("test", new GridCacheAbstractFullApiSelfTest.DummyServiceImpl()); + } + + /** + * @throws Exception If failed. + */ + public void testDeployService3() throws Exception { + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + clientModeThreadLocal(true); + + startGrid(0); + + return null; + } + }, "start-node"); + + clientModeThreadLocal(false); + + startGrid(1); + + fut.get(); + + grid(0).services(grid(0).cluster()).deployNodeSingleton("test", new GridCacheAbstractFullApiSelfTest.DummyServiceImpl()); + } + + /** + * @throws Exception If failed. + */ + public void testLargeUserAttribute1() throws Exception { + initLargeAttribute(); + + startGrid(0); + + checkZkNodesCleanup(); + + userAttrs = null; + + startGrid(1); + + waitForEventsAcks(ignite(0)); + + waitForTopology(2); + } + + /** + * @throws Exception If failed. + */ + public void testLargeUserAttribute2() throws Exception { + startGrid(0); + + initLargeAttribute(); + + startGrid(1); + + waitForEventsAcks(ignite(0)); + + checkZkNodesCleanup(); + } + + /** + * @throws Exception If failed. + */ + public void testLargeUserAttribute3() throws Exception { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + long stopTime = System.currentTimeMillis() + 60_000; + + int nodes = 0; + + for (int i = 0; i < 25; i++) { + info("Iteration: " + i); + + if (rnd.nextBoolean()) + initLargeAttribute(); + else + userAttrs = null; + + clientMode(i > 5); + + startGrid(i); + + nodes++; + + if (System.currentTimeMillis() >= stopTime) + break; + } + + waitForTopology(nodes); + } + + /** + * + */ + private void initLargeAttribute() { + userAttrs = new HashMap<>(); + + int[] attr = new int[1024 * 1024 + ThreadLocalRandom.current().nextInt(1024)]; + + for (int i = 0; i < attr.length; i++) + attr[i] = i; + + userAttrs.put("testAttr", attr); + } + + /** + * @throws Exception If failed. + */ + public void testLargeCustomEvent() throws Exception { + Ignite srv0 = startGrid(0); + + // Send large message, single node in topology. + IgniteCache cache = srv0.createCache(largeCacheConfiguration("c1")); + + for (int i = 0; i < 100; i++) + cache.put(i, i); + + assertEquals(1, cache.get(1)); + + waitForEventsAcks(ignite(0)); + + startGridsMultiThreaded(1, 3); + + srv0.destroyCache("c1"); + + // Send large message, multiple nodes in topology. + cache = srv0.createCache(largeCacheConfiguration("c1")); + + for (int i = 0; i < 100; i++) + cache.put(i, i); + + waitForTopology(4); + + ignite(3).createCache(largeCacheConfiguration("c2")); + } + + /** + * @throws Exception If failed. + */ + public void testClientReconnectSessionExpire1_1() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8131"); + + clientReconnectSessionExpire(false); + } + + /** + * @throws Exception If failed. + */ + public void testClientReconnectSessionExpire1_2() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8131"); + + clientReconnectSessionExpire(true); + } + + /** + * @param closeSock Test mode flag. + * @throws Exception If failed. + */ + private void clientReconnectSessionExpire(boolean closeSock) throws Exception { + startGrid(0); + + sesTimeout = 2000; + clientMode(true); + testSockNio = true; + + Ignite client = startGrid(1); + + client.cache(DEFAULT_CACHE_NAME).put(1, 1); + + reconnectClientNodes(log, Collections.singletonList(client), closeSock); + + assertEquals(1, client.cache(DEFAULT_CACHE_NAME).get(1)); + + client.compute().broadcast(new DummyCallable(null)); + } + + /** + * @throws Exception If failed. + */ + public void testForceClientReconnect() throws Exception { + final int SRVS = 3; + + startGrids(SRVS); + + clientMode(true); + + startGrid(SRVS); + + reconnectClientNodes(Collections.singletonList(ignite(SRVS)), new Callable() { + @Override public Void call() throws Exception { + ZookeeperDiscoverySpi spi = waitSpi(getTestIgniteInstanceName(SRVS)); + + spi.clientReconnect(); + + return null; + } + }); + + waitForTopology(SRVS + 1); + } + + /** + * @throws Exception If failed. + */ + public void testForcibleClientFail() throws Exception { + final int SRVS = 3; + + startGrids(SRVS); + + clientMode(true); + + startGrid(SRVS); + + reconnectClientNodes(Collections.singletonList(ignite(SRVS)), new Callable() { + @Override public Void call() throws Exception { + ZookeeperDiscoverySpi spi = waitSpi(getTestIgniteInstanceName(0)); + + spi.failNode(ignite(SRVS).cluster().localNode().id(), "Test forcible node fail"); + + return null; + } + }); + + waitForTopology(SRVS + 1); + } + + /** + * @throws Exception If failed. + */ + public void testDuplicatedNodeId() throws Exception { + UUID nodeId0 = nodeId = UUID.randomUUID(); + + startGrid(0); + + int failingNodeIdx = 100; + + for (int i = 0; i < 5; i++) { + final int idx = failingNodeIdx++; + + nodeId = nodeId0; + + info("Start node with duplicated ID [iter=" + i + ", nodeId=" + nodeId + ']'); + + Throwable err = GridTestUtils.assertThrows(log, new Callable() { + @Override public Void call() throws Exception { + startGrid(idx); + + return null; + } + }, IgniteCheckedException.class, null); + + IgniteSpiException spiErr = X.cause(err, IgniteSpiException.class); + + assertNotNull(spiErr); + assertTrue(spiErr.getMessage().contains("Node with the same ID already exists")); + + nodeId = null; + + info("Start node with unique ID [iter=" + i + ']'); + + Ignite ignite = startGrid(idx); + + nodeId0 = ignite.cluster().localNode().id(); + + waitForTopology(i + 2); + } + } + + /** + * @throws Exception If failed. + */ + public void testPing() throws Exception { + sesTimeout = 5000; + + startGrids(3); + + final ZookeeperDiscoverySpi spi = waitSpi(getTestIgniteInstanceName(1)); + + final UUID nodeId = ignite(2).cluster().localNode().id(); + + IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Runnable() { + @Override public void run() { + assertTrue(spi.pingNode(nodeId)); + } + }, 32, "ping"); + + fut.get(); + + fut = GridTestUtils.runMultiThreadedAsync(new Runnable() { + @Override public void run() { + spi.pingNode(nodeId); + } + }, 32, "ping"); + + U.sleep(100); + + stopGrid(2); + + fut.get(); + + fut = GridTestUtils.runMultiThreadedAsync(new Runnable() { + @Override public void run() { + assertFalse(spi.pingNode(nodeId)); + } + }, 32, "ping"); + + fut.get(); + } + + /** + * @throws Exception If failed. + */ + public void testWithPersistence1() throws Exception { + startWithPersistence(false); + } + + /** + * @throws Exception If failed. + */ + public void testWithPersistence2() throws Exception { + startWithPersistence(true); + } + + /** + * @throws Exception If failed. + */ + public void testNoOpCommunicationFailureResolve_1() throws Exception { + communicationFailureResolve_Simple(2); + } + + /** + * @throws Exception If failed. + */ + public void testNoOpCommunicationErrorResolve_2() throws Exception { + communicationFailureResolve_Simple(10); + } + + /** + * @param nodes Nodes number. + * @throws Exception If failed. + */ + private void communicationFailureResolve_Simple(int nodes) throws Exception { + assert nodes > 1; + + sesTimeout = 2000; + commFailureRslvr = NoOpCommunicationFailureResolver.FACTORY; + + startGridsMultiThreaded(nodes); + + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + for (int i = 0; i < 3; i++) { + info("Iteration: " + i); + + int idx1 = rnd.nextInt(nodes); + + int idx2; + + do { + idx2 = rnd.nextInt(nodes); + } + while (idx1 == idx2); + + ZookeeperDiscoverySpi spi = spi(ignite(idx1)); + + spi.resolveCommunicationFailure(ignite(idx2).cluster().localNode(), new Exception("test")); + + checkInternalStructuresCleanup(); + } + } + + /** + * Tests case when one node fails before sending communication status. + * + * @throws Exception If failed. + */ + public void testNoOpCommunicationErrorResolve_3() throws Exception { + sesTimeout = 2000; + commFailureRslvr = NoOpCommunicationFailureResolver.FACTORY; + + startGridsMultiThreaded(3); + + sesTimeout = 10_000; + + testSockNio = true; + sesTimeout = 5000; + + startGrid(3); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() { + ZookeeperDiscoverySpi spi = spi(ignite(0)); + + spi.resolveCommunicationFailure(ignite(1).cluster().localNode(), new Exception("test")); + + return null; + } + }); + + U.sleep(1000); + + ZkTestClientCnxnSocketNIO nio = ZkTestClientCnxnSocketNIO.forNode(ignite(3)); + + nio.closeSocket(true); + + try { + stopGrid(3); + + fut.get(); + } + finally { + nio.allowConnect(); + } + + waitForTopology(3); + } + + /** + * Tests case when Coordinator fails while resolve process is in progress. + * + * @throws Exception If failed. + */ + public void testNoOpCommunicationErrorResolve_4() throws Exception { + testCommSpi = true; + + sesTimeout = 2000; + commFailureRslvr = NoOpCommunicationFailureResolver.FACTORY; + + startGrid(0); + + startGridsMultiThreaded(1, 3); + + ZkTestCommunicationSpi commSpi = ZkTestCommunicationSpi.testSpi(ignite(3)); + + commSpi.pingLatch = new CountDownLatch(1); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() { + ZookeeperDiscoverySpi spi = spi(ignite(1)); + + spi.resolveCommunicationFailure(ignite(2).cluster().localNode(), new Exception("test")); + + return null; + } + }); + + U.sleep(1000); + + assertFalse(fut.isDone()); + + stopGrid(0); + + commSpi.pingLatch.countDown(); + + fut.get(); + + waitForTopology(3); + } + + /** + * Tests that nodes join is delayed while resolve is in progress. + * + * @throws Exception If failed. + */ + public void testNoOpCommunicationErrorResolve_5() throws Exception { + testCommSpi = true; + + sesTimeout = 2000; + commFailureRslvr = NoOpCommunicationFailureResolver.FACTORY; + + startGrid(0); + + startGridsMultiThreaded(1, 3); + + ZkTestCommunicationSpi commSpi = ZkTestCommunicationSpi.testSpi(ignite(3)); + + commSpi.pingStartLatch = new CountDownLatch(1); + commSpi.pingLatch = new CountDownLatch(1); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() { + ZookeeperDiscoverySpi spi = spi(ignite(1)); + + spi.resolveCommunicationFailure(ignite(2).cluster().localNode(), new Exception("test")); + + return null; + } + }); + + assertTrue(commSpi.pingStartLatch.await(10, SECONDS)); + + try { + assertFalse(fut.isDone()); + + final AtomicInteger nodeIdx = new AtomicInteger(3); + + IgniteInternalFuture startFut = GridTestUtils.runMultiThreadedAsync(new Callable() { + @Override public Void call() throws Exception { + startGrid(nodeIdx.incrementAndGet()); + + return null; + } + }, 3, "start-node"); + + U.sleep(1000); + + assertFalse(startFut.isDone()); + + assertEquals(4, ignite(0).cluster().nodes().size()); + + commSpi.pingLatch.countDown(); + + startFut.get(); + fut.get(); + + waitForTopology(7); + } + finally { + commSpi.pingLatch.countDown(); + } + } + + /** + * @throws Exception If failed. + */ + public void testCommunicationErrorResolve_KillNode_1() throws Exception { + communicationFailureResolve_KillNodes(2, Collections.singleton(2L)); + } + + /** + * @throws Exception If failed. + */ + public void testCommunicationErrorResolve_KillNode_2() throws Exception { + communicationFailureResolve_KillNodes(3, Collections.singleton(2L)); + } + + /** + * @throws Exception If failed. + */ + public void testCommunicationErrorResolve_KillNode_3() throws Exception { + communicationFailureResolve_KillNodes(10, Arrays.asList(2L, 4L, 6L)); + } + + /** + * @throws Exception If failed. + */ + public void testCommunicationErrorResolve_KillCoordinator_1() throws Exception { + communicationFailureResolve_KillNodes(2, Collections.singleton(1L)); + } + + /** + * @throws Exception If failed. + */ + public void testCommunicationErrorResolve_KillCoordinator_2() throws Exception { + communicationFailureResolve_KillNodes(3, Collections.singleton(1L)); + } + + /** + * @throws Exception If failed. + */ + public void testCommunicationErrorResolve_KillCoordinator_3() throws Exception { + communicationFailureResolve_KillNodes(10, Arrays.asList(1L, 4L, 6L)); + } + + /** + * @throws Exception If failed. + */ + public void testCommunicationErrorResolve_KillCoordinator_4() throws Exception { + communicationFailureResolve_KillNodes(10, Arrays.asList(1L, 2L, 3L)); + } + + /** + * @param startNodes Number of nodes to start. + * @param killNodes Nodes to kill by resolve process. + * @throws Exception If failed. + */ + private void communicationFailureResolve_KillNodes(int startNodes, Collection killNodes) throws Exception { + testCommSpi = true; + + commFailureRslvr = TestNodeKillCommunicationFailureResolver.factory(killNodes); + + startGrids(startNodes); + + ZkTestCommunicationSpi commSpi = ZkTestCommunicationSpi.testSpi(ignite(0)); + + commSpi.checkRes = new BitSet(startNodes); + + ZookeeperDiscoverySpi spi = null; + UUID killNodeId = null; + + for (Ignite node : G.allGrids()) { + ZookeeperDiscoverySpi spi0 = spi(node); + + if (!killNodes.contains(node.cluster().localNode().order())) + spi = spi0; + else + killNodeId = node.cluster().localNode().id(); + } + + assertNotNull(spi); + assertNotNull(killNodeId); + + try { + spi.resolveCommunicationFailure(spi.getNode(killNodeId), new Exception("test")); + + fail("Exception is not thrown"); + } + catch (IgniteSpiException e) { + assertTrue("Unexpected exception: " + e, e.getCause() instanceof ClusterTopologyCheckedException); + } + + int expNodes = startNodes - killNodes.size(); + + waitForTopology(expNodes); + + for (Ignite node : G.allGrids()) + assertFalse(killNodes.contains(node.cluster().localNode().order())); + + startGrid(startNodes); + + waitForTopology(expNodes + 1); + } + + /** + * @throws Exception If failed. + */ + public void testCommunicationFailureResolve_KillCoordinator_5() throws Exception { + sesTimeout = 2000; + + testCommSpi = true; + commFailureRslvr = KillCoordinatorCommunicationFailureResolver.FACTORY; + + startGrids(10); + + int crd = 0; + + int nodeIdx = 10; + + for (int i = 0; i < 10; i++) { + info("Iteration: " + i); + + for (Ignite node : G.allGrids()) + ZkTestCommunicationSpi.testSpi(node).initCheckResult(10); + + UUID crdId = ignite(crd).cluster().localNode().id(); + + ZookeeperDiscoverySpi spi = spi(ignite(crd + 1)); + + try { + spi.resolveCommunicationFailure(spi.getNode(crdId), new Exception("test")); + + fail("Exception is not thrown"); + } + catch (IgniteSpiException e) { + assertTrue("Unexpected exception: " + e, e.getCause() instanceof ClusterTopologyCheckedException); + } + + waitForTopology(9); + + startGrid(nodeIdx++); + + waitForTopology(10); + + crd++; + } + } + + /** + * @throws Exception If failed. + */ + public void testCommunicationFailureResolve_KillRandom() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8179"); + + sesTimeout = 2000; + + testCommSpi = true; + commFailureRslvr = KillRandomCommunicationFailureResolver.FACTORY; + + startGridsMultiThreaded(10); + + clientMode(true); + + startGridsMultiThreaded(10, 5); + + int nodeIdx = 15; + + for (int i = 0; i < 10; i++) { + info("Iteration: " + i); + + ZookeeperDiscoverySpi spi = null; + + for (Ignite node : G.allGrids()) { + ZkTestCommunicationSpi.testSpi(node).initCheckResult(100); + + spi = spi(node); + } + + assert spi != null; + + try { + spi.resolveCommunicationFailure(spi.getRemoteNodes().iterator().next(), new Exception("test")); + } + catch (IgniteSpiException ignore) { + // No-op. + } + + clientMode(ThreadLocalRandom.current().nextBoolean()); + + startGrid(nodeIdx++); + + awaitPartitionMapExchange(); + } + } + + /** + * @throws Exception If failed. + */ + public void testDefaultCommunicationFailureResolver1() throws Exception { + testCommSpi = true; + sesTimeout = 5000; + + startGrids(3); + + ZkTestCommunicationSpi.testSpi(ignite(0)).initCheckResult(3, 0, 1); + ZkTestCommunicationSpi.testSpi(ignite(1)).initCheckResult(3, 0, 1); + ZkTestCommunicationSpi.testSpi(ignite(2)).initCheckResult(3, 2); + + UUID killedId = nodeId(2); + + assertNotNull(ignite(0).cluster().node(killedId)); + + ZookeeperDiscoverySpi spi = spi(ignite(0)); + + spi.resolveCommunicationFailure(spi.getNode(ignite(1).cluster().localNode().id()), new Exception("test")); + + waitForTopology(2); + + assertNull(ignite(0).cluster().node(killedId)); + } + + /** + * @throws Exception If failed. + */ + public void testDefaultCommunicationFailureResolver2() throws Exception { + testCommSpi = true; + sesTimeout = 5000; + + startGrids(3); + + clientMode(true); + + startGridsMultiThreaded(3, 2); + + ZkTestCommunicationSpi.testSpi(ignite(0)).initCheckResult(5, 0, 1); + ZkTestCommunicationSpi.testSpi(ignite(1)).initCheckResult(5, 0, 1); + ZkTestCommunicationSpi.testSpi(ignite(2)).initCheckResult(5, 2, 3, 4); + ZkTestCommunicationSpi.testSpi(ignite(3)).initCheckResult(5, 2, 3, 4); + ZkTestCommunicationSpi.testSpi(ignite(4)).initCheckResult(5, 2, 3, 4); + + ZookeeperDiscoverySpi spi = spi(ignite(0)); + + spi.resolveCommunicationFailure(spi.getNode(ignite(1).cluster().localNode().id()), new Exception("test")); + + waitForTopology(2); + } + + /** + * @throws Exception If failed. + */ + public void testDefaultCommunicationFailureResolver3() throws Exception { + defaultCommunicationFailureResolver_BreakCommunication(3, 1); + } + + /** + * @throws Exception If failed. + */ + public void testDefaultCommunicationFailureResolver4() throws Exception { + defaultCommunicationFailureResolver_BreakCommunication(3, 0); + } + + /** + * @throws Exception If failed. + */ + public void testDefaultCommunicationFailureResolver5() throws Exception { + defaultCommunicationFailureResolver_BreakCommunication(10, 1, 3, 6); + } + + /** + * @param startNodes Initial nodes number. + * @param breakNodes Node indices where communication server is closed. + * @throws Exception If failed. + */ + private void defaultCommunicationFailureResolver_BreakCommunication(int startNodes, final int...breakNodes) throws Exception { + sesTimeout = 5000; + + startGridsMultiThreaded(startNodes); + + final CyclicBarrier b = new CyclicBarrier(breakNodes.length); + + GridTestUtils.runMultiThreaded(new IgniteInClosure() { + @Override public void apply(Integer threadIdx) { + try { + b.await(); + + int nodeIdx = breakNodes[threadIdx]; + + info("Close communication: " + nodeIdx); + + ((TcpCommunicationSpi)ignite(nodeIdx).configuration().getCommunicationSpi()).simulateNodeFailure(); + } + catch (Exception e) { + fail("Unexpected error: " + e); + } + } + }, breakNodes.length, "break-communication"); + + waitForTopology(startNodes - breakNodes.length); + } + + /** + * @throws Exception If failed. + */ + public void testCommunicationFailureResolve_CachesInfo1() throws Exception { + testCommSpi = true; + sesTimeout = 5000; + + final CacheInfoCommunicationFailureResolver rslvr = new CacheInfoCommunicationFailureResolver(); + + commFailureRslvr = new IgniteOutClosure() { + @Override public CommunicationFailureResolver apply() { + return rslvr; + } + }; + + startGrids(2); + + awaitPartitionMapExchange(); + + Map> expCaches = new HashMap<>(); + + expCaches.put(DEFAULT_CACHE_NAME, new T3<>(RendezvousAffinityFunction.DFLT_PARTITION_COUNT, 0, 1)); + + checkResolverCachesInfo(ignite(0), expCaches); + + List caches = new ArrayList<>(); + + CacheConfiguration c1 = new CacheConfiguration("c1"); + c1.setBackups(1); + c1.setAffinity(new RendezvousAffinityFunction(false, 64)); + caches.add(c1); + + CacheConfiguration c2 = new CacheConfiguration("c2"); + c2.setBackups(2); + c2.setAffinity(new RendezvousAffinityFunction(false, 128)); + caches.add(c2); + + CacheConfiguration c3 = new CacheConfiguration("c3"); + c3.setCacheMode(CacheMode.REPLICATED); + c3.setAffinity(new RendezvousAffinityFunction(false, 256)); + caches.add(c3); + + ignite(0).createCaches(caches); + + expCaches.put("c1", new T3<>(64, 1, 2)); + expCaches.put("c2", new T3<>(128, 2, 2)); + expCaches.put("c3", new T3<>(256, 1, 2)); + + checkResolverCachesInfo(ignite(0), expCaches); + + startGrid(2); + startGrid(3); + + awaitPartitionMapExchange(); + + expCaches.put("c2", new T3<>(128, 2, 3)); + expCaches.put("c3", new T3<>(256, 1, 4)); + + checkResolverCachesInfo(ignite(0), expCaches); + + CacheConfiguration c4 = new CacheConfiguration("c4"); + c4.setCacheMode(CacheMode.PARTITIONED); + c4.setBackups(0); + c4.setAffinity(new RendezvousAffinityFunction(false, 256)); + c4.setNodeFilter(new TestCacheNodeExcludingFilter(getTestIgniteInstanceName(0), getTestIgniteInstanceName(1))); + + ignite(2).createCache(c4); + + expCaches.put("c4", new T3<>(256, 0, 1)); + + checkResolverCachesInfo(ignite(0), expCaches); + + stopGrid(0); // Stop current coordinator, check new coordinator will initialize required caches information. + + awaitPartitionMapExchange(); + + expCaches.put("c3", new T3<>(256, 1, 3)); + + checkResolverCachesInfo(ignite(1), expCaches); + + startGrid(0); + + expCaches.put("c3", new T3<>(256, 1, 4)); + + checkResolverCachesInfo(ignite(1), expCaches); + + stopGrid(1); + + expCaches.put("c3", new T3<>(256, 1, 3)); + + checkResolverCachesInfo(ignite(3), expCaches); + } + + /** + * @throws Exception If failed. + */ + public void testCommunicationFailureResolve_CachesInfo2() throws Exception { + testCommSpi = true; + sesTimeout = 5000; + + final CacheInfoCommunicationFailureResolver rslvr = new CacheInfoCommunicationFailureResolver(); + + commFailureRslvr = new IgniteOutClosure() { + @Override public CommunicationFailureResolver apply() { + return rslvr; + } + }; + + Ignite srv0 = startGrid(0); + + CacheConfiguration ccfg = new CacheConfiguration("c1"); + ccfg.setBackups(1); + + srv0.createCache(ccfg); + + // Block rebalance to make sure node0 will be the only owner. + TestRecordingCommunicationSpi.spi(srv0).blockMessages(new IgniteBiPredicate() { + @Override public boolean apply(ClusterNode node, Message msg) { + return msg instanceof GridDhtPartitionSupplyMessage && + ((GridDhtPartitionSupplyMessage) msg).groupId() == CU.cacheId("c1"); + } + }); + + startGrid(1); + + U.sleep(1000); + + ZookeeperDiscoverySpi spi = spi(srv0); + + rslvr.latch = new CountDownLatch(1); + + ZkTestCommunicationSpi.testSpi(srv0).initCheckResult(2, 0); + + spi.resolveCommunicationFailure(spi.getRemoteNodes().iterator().next(), new Exception("test")); + + assertTrue(rslvr.latch.await(10, SECONDS)); + + List> cacheOwners = rslvr.ownersMap.get("c1"); + + ClusterNode node0 = srv0.cluster().localNode(); + + for (int p = 0; p < RendezvousAffinityFunction.DFLT_PARTITION_COUNT; p++) { + List owners = cacheOwners.get(p); + + assertEquals(1, owners.size()); + assertEquals(node0, owners.get(0)); + } + + TestRecordingCommunicationSpi.spi(srv0).stopBlock(); + + awaitPartitionMapExchange(); + + Map> expCaches = new HashMap<>(); + + expCaches.put(DEFAULT_CACHE_NAME, new T3<>(RendezvousAffinityFunction.DFLT_PARTITION_COUNT, 0, 1)); + expCaches.put("c1", new T3<>(RendezvousAffinityFunction.DFLT_PARTITION_COUNT, 1, 2)); + + checkResolverCachesInfo(srv0, expCaches); + } + + /** + * @param crd Coordinator node. + * @param expCaches Expected caches info. + * @throws Exception If failed. + */ + private void checkResolverCachesInfo(Ignite crd, Map> expCaches) + throws Exception + { + CacheInfoCommunicationFailureResolver rslvr = + (CacheInfoCommunicationFailureResolver)crd.configuration().getCommunicationFailureResolver(); + + assertNotNull(rslvr); + + ZookeeperDiscoverySpi spi = spi(crd); + + rslvr.latch = new CountDownLatch(1); + + ZkTestCommunicationSpi.testSpi(crd).initCheckResult(crd.cluster().nodes().size(), 0); + + spi.resolveCommunicationFailure(spi.getRemoteNodes().iterator().next(), new Exception("test")); + + assertTrue(rslvr.latch.await(10, SECONDS)); + + rslvr.checkCachesInfo(expCaches); + + rslvr.reset(); + } + + /** + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + public void testCommunicationFailureResolve_ConcurrentDiscoveyEvents() throws Exception { + sesTimeout = 5000; + + commFailureRslvr = NoOpCommunicationFailureResolver.FACTORY; + + final int INITIAL_NODES = 5; + + startGridsMultiThreaded(INITIAL_NODES); + + final CyclicBarrier b = new CyclicBarrier(4); + + GridCompoundFuture fut = new GridCompoundFuture<>(); + + final AtomicBoolean stop = new AtomicBoolean(); + + fut.add((IgniteInternalFuture)GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + b.await(); + + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + for (int i = 0; i < 10; i++) { + startGrid(i + INITIAL_NODES); + + Thread.sleep(rnd.nextLong(1000) + 10); + + if (stop.get()) + break; + } + + return null; + } + }, "test-node-start")); + + fut.add((IgniteInternalFuture)GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + b.await(); + + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + while (!stop.get()) { + startGrid(100); + + Thread.sleep(rnd.nextLong(1000) + 10); + + stopGrid(100); + + Thread.sleep(rnd.nextLong(1000) + 10); + } + + return null; + } + }, "test-node-restart")); + + fut.add((IgniteInternalFuture)GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + b.await(); + + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + int idx = 0; + + while (!stop.get()) { + CacheConfiguration ccfg = new CacheConfiguration("c-" + idx++); + ccfg.setBackups(rnd.nextInt(5)); + + ignite(rnd.nextInt(INITIAL_NODES)).createCache(ccfg); + + Thread.sleep(rnd.nextLong(1000) + 10); + + ignite(rnd.nextInt(INITIAL_NODES)).destroyCache(ccfg.getName()); + + Thread.sleep(rnd.nextLong(1000) + 10); + } + + return null; + } + }, "test-create-cache")); + + fut.add((IgniteInternalFuture)GridTestUtils.runMultiThreadedAsync(new Callable() { + @Override public Void call() throws Exception { + try { + b.await(); + + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + for (int i = 0; i < 5; i++) { + info("resolveCommunicationFailure: " + i); + + ZookeeperDiscoverySpi spi = spi(ignite(rnd.nextInt(INITIAL_NODES))); + + spi.resolveCommunicationFailure(ignite(rnd.nextInt(INITIAL_NODES)).cluster().localNode(), + new Exception("test")); + } + + return null; + } + finally { + stop.set(true); + } + } + }, 5, "test-resolve-failure")); + + fut.markInitialized(); + + fut.get(); + } + + /** + * @throws Exception If failed. + */ + public void testCommunicationFailureResolve_ConcurrentMultinode() throws Exception { + sesTimeout = 5000; + + commFailureRslvr = NoOpCommunicationFailureResolver.FACTORY; + + startGridsMultiThreaded(5); + + client = true; + + startGridsMultiThreaded(5, 5); + + final int NODES = 10; + + GridTestUtils.runMultiThreaded(new Callable() { + @Override public Void call() throws Exception { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + for (int i = 0; i < 5; i++) { + info("resolveCommunicationFailure: " + i); + + ZookeeperDiscoverySpi spi = spi(ignite(rnd.nextInt(NODES))); + + spi.resolveCommunicationFailure(spi.getRemoteNodes().iterator().next(), new Exception("test")); + } + + return null; + } + }, 30, "test-resolve-failure"); + } + + /** + * @throws Exception If failed. + */ + public void testConnectionCheck() throws Exception { + final int NODES = 5; + + startGridsMultiThreaded(NODES); + + for (int i = 0; i < NODES; i++) { + Ignite node = ignite(i); + + TcpCommunicationSpi spi = (TcpCommunicationSpi)node.configuration().getCommunicationSpi(); + + List nodes = new ArrayList<>(node.cluster().nodes()); + + BitSet res = spi.checkConnection(nodes).get(); + + for (int j = 0; j < NODES; j++) + assertTrue(res.get(j)); + } + } + + /** + * @throws Exception If failed. + */ + public void testReconnectDisabled_ConnectionLost() throws Exception { + clientReconnectDisabled = true; + + startGrid(0); + + sesTimeout = 3000; + testSockNio = true; + client = true; + + Ignite client = startGrid(1); + + final CountDownLatch latch = new CountDownLatch(1); + + client.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event evt) { + latch.countDown(); + + return false; + } + }, EventType.EVT_NODE_SEGMENTED); + + ZkTestClientCnxnSocketNIO nio = ZkTestClientCnxnSocketNIO.forNode(client); + + nio.closeSocket(true); + + try { + waitNoAliveZkNodes(log, + zkCluster.getConnectString(), + Collections.singletonList(aliveZkNodePath(client)), + 10_000); + } + finally { + nio.allowConnect(); + } + + assertTrue(latch.await(10, SECONDS)); + } + + /** + * @throws Exception If failed. + */ + public void testServersLeft_FailOnTimeout() throws Exception { + startGrid(0); + + final int CLIENTS = 5; + + joinTimeout = 3000; + + clientMode(true); + + startGridsMultiThreaded(1, CLIENTS); + + waitForTopology(CLIENTS + 1); + + final CountDownLatch latch = new CountDownLatch(CLIENTS); + + for (int i = 0; i < CLIENTS; i++) { + Ignite node = ignite(i + 1); + + node.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event evt) { + latch.countDown(); + + return false; + } + }, EventType.EVT_NODE_SEGMENTED); + } + + stopGrid(getTestIgniteInstanceName(0), true, false); + + assertTrue(latch.await(10, SECONDS)); + + evts.clear(); + } + + /** + * + */ + public void testStartNoServers_FailOnTimeout() { + joinTimeout = 3000; + + clientMode(true); + + long start = System.currentTimeMillis(); + + Throwable err = GridTestUtils.assertThrows(log, new Callable() { + @Override public Void call() throws Exception { + startGrid(0); + + return null; + } + }, IgniteCheckedException.class, null); + + assertTrue(System.currentTimeMillis() >= start + joinTimeout); + + IgniteSpiException spiErr = X.cause(err, IgniteSpiException.class); + + assertNotNull(spiErr); + assertTrue(spiErr.getMessage().contains("Failed to connect to cluster within configured timeout")); + } + + /** + * @throws Exception If failed. + */ + public void testStartNoServer_WaitForServers1() throws Exception { + startNoServer_WaitForServers(0); + } + + /** + * @throws Exception If failed. + */ + public void testStartNoServer_WaitForServers2() throws Exception { + startNoServer_WaitForServers(10_000); + } + + /** + * @param joinTimeout Join timeout. + * @throws Exception If failed. + */ + private void startNoServer_WaitForServers(long joinTimeout) throws Exception { + this.joinTimeout = joinTimeout; + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + clientModeThreadLocal(true); + + startGrid(0); + + return null; + } + }); + + U.sleep(3000); + + waitSpi(getTestIgniteInstanceName(0)); + + clientModeThreadLocal(false); + + startGrid(1); + + fut.get(); + + waitForTopology(2); + } + + /** + * @throws Exception If failed. + */ + public void testDisconnectOnServersLeft_1() throws Exception { + disconnectOnServersLeft(1, 1); + } + + /** + * @throws Exception If failed. + */ + public void testDisconnectOnServersLeft_2() throws Exception { + disconnectOnServersLeft(5, 1); + } + + /** + * @throws Exception If failed. + */ + public void testDisconnectOnServersLeft_3() throws Exception { + disconnectOnServersLeft(1, 10); + } + + /** + * @throws Exception If failed. + */ + public void testDisconnectOnServersLeft_4() throws Exception { + disconnectOnServersLeft(5, 10); + } + + /** + * @throws Exception If failed. + */ + public void testDisconnectOnServersLeft_5() throws Exception { + joinTimeout = 10_000; + + disconnectOnServersLeft(5, 10); + } + + /** + * @param srvs Number of servers. + * @param clients Number of clients. + * @throws Exception If failed. + */ + private void disconnectOnServersLeft(int srvs, int clients) throws Exception { + startGridsMultiThreaded(srvs); + + clientMode(true); + + startGridsMultiThreaded(srvs, clients); + + for (int i = 0; i < 5; i++) { + info("Iteration: " + i); + + final CountDownLatch disconnectLatch = new CountDownLatch(clients); + final CountDownLatch reconnectLatch = new CountDownLatch(clients); + + IgnitePredicate p = new IgnitePredicate() { + @Override public boolean apply(Event evt) { + if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) { + log.info("Disconnected: " + evt); + + disconnectLatch.countDown(); + } + else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) { + log.info("Reconnected: " + evt); + + reconnectLatch.countDown(); + + return false; + } + + return true; + } + }; + + for (int c = 0; c < clients; c++) { + Ignite client = ignite(srvs + c); + + assertTrue(client.configuration().isClientMode()); + + client.events().localListen(p, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED); + } + + log.info("Stop all servers."); + + GridTestUtils.runMultiThreaded(new IgniteInClosure() { + @Override public void apply(Integer threadIdx) { + stopGrid(getTestIgniteInstanceName(threadIdx), true, false); + } + }, srvs, "stop-server"); + + waitReconnectEvent(log, disconnectLatch); + + evts.clear(); + + clientMode(false); + + log.info("Restart servers."); + + startGridsMultiThreaded(0, srvs); + + waitReconnectEvent(log, reconnectLatch); + + waitForTopology(srvs + clients); + + log.info("Reconnect finished."); + } + } + + /** + * @throws Exception If failed. + */ + public void testReconnectServersRestart_1() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8178"); + + reconnectServersRestart(1); + } + + /** + * @throws Exception If failed. + */ + public void testReconnectServersRestart_2() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8178"); + + reconnectServersRestart(3); + } + + /** + * @param srvs Number of server nodes in test. + * @throws Exception If failed. + */ + private void reconnectServersRestart(int srvs) throws Exception { + startGridsMultiThreaded(srvs); + + clientMode(true); + + final int CLIENTS = 10; + + startGridsMultiThreaded(srvs, CLIENTS); + + clientMode(false); + + long stopTime = System.currentTimeMillis() + 30_000; + + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + final int NODES = srvs + CLIENTS; + + int iter = 0; + + while (System.currentTimeMillis() < stopTime) { + int restarts = rnd.nextInt(10) + 1; + + info("Test iteration [iter=" + iter++ + ", restarts=" + restarts + ']'); + + for (int i = 0; i < restarts; i++) { + GridTestUtils.runMultiThreaded(new IgniteInClosure() { + @Override public void apply(Integer threadIdx) { + stopGrid(getTestIgniteInstanceName(threadIdx), true, false); + } + }, srvs, "stop-server"); + + startGridsMultiThreaded(0, srvs); + } + + final Ignite srv = ignite(0); + + assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return srv.cluster().nodes().size() == NODES; + } + }, 30_000)); + + waitForTopology(NODES); + + awaitPartitionMapExchange(); + } + + evts.clear(); + } + + /** + * @throws Exception If failed. + */ + public void testReconnectServersRestart_3() throws Exception { + startGrid(0); + + clientMode(true); + + startGridsMultiThreaded(10, 10); + + stopGrid(getTestIgniteInstanceName(0), true, false); + + final int srvIdx = ThreadLocalRandom.current().nextInt(10); + + final AtomicInteger idx = new AtomicInteger(); + + info("Restart nodes."); + + // Test concurrent start when there are disconnected nodes from previous cluster. + GridTestUtils.runMultiThreaded(new Callable() { + @Override public Void call() throws Exception { + int threadIdx = idx.getAndIncrement(); + + clientModeThreadLocal(threadIdx == srvIdx || ThreadLocalRandom.current().nextBoolean()); + + startGrid(threadIdx); + + return null; + } + }, 10, "start-node"); + + waitForTopology(20); + + evts.clear(); + } + + /** + * @throws Exception If failed. + */ + public void testStartNoZk() throws Exception { + stopZkCluster(); + + sesTimeout = 30_000; + + zkCluster = ZookeeperDiscoverySpiTestSuite2.createTestingCluster(3); + + try { + final AtomicInteger idx = new AtomicInteger(); + + IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Callable() { + @Override public Void call() throws Exception { + startGrid(idx.getAndIncrement()); + + return null; + } + }, 5, "start-node"); + + U.sleep(5000); + + assertFalse(fut.isDone()); + + zkCluster.start(); + + fut.get(); + + waitForTopology(5); + } + finally { + zkCluster.start(); + } + } + + /** + * @param dfltConsistenId Default consistent ID flag. + * @throws Exception If failed. + */ + private void startWithPersistence(boolean dfltConsistenId) throws Exception { + this.dfltConsistenId = dfltConsistenId; + + persistence = true; + + for (int i = 0; i < 3; i++) { + info("Iteration: " + i); + + clientMode(false); + + startGridsMultiThreaded(4, i == 0); + + clientMode(true); + + startGridsMultiThreaded(4, 3); + + waitForTopology(7); + + stopGrid(1); + + waitForTopology(6); + + stopGrid(4); + + waitForTopology(5); + + stopGrid(0); + + waitForTopology(4); + + checkEventsConsistency(); + + stopAllGrids(); + + evts.clear(); + } + } + + /** + * @param clients Clients. + * @param c Closure to run. + * @throws Exception If failed. + */ + private void reconnectClientNodes(List clients, Callable c) + throws Exception { + final CountDownLatch disconnectLatch = new CountDownLatch(clients.size()); + final CountDownLatch reconnectLatch = new CountDownLatch(clients.size()); + + IgnitePredicate p = new IgnitePredicate() { + @Override public boolean apply(Event evt) { + if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) { + log.info("Disconnected: " + evt); + + disconnectLatch.countDown(); + } + else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) { + log.info("Reconnected: " + evt); + + reconnectLatch.countDown(); + } + + return true; + } + }; + + for (Ignite client : clients) + client.events().localListen(p, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED); + + c.call(); + + waitReconnectEvent(log, disconnectLatch); + + waitReconnectEvent(log, reconnectLatch); + + for (Ignite client : clients) + client.events().stopLocalListen(p); + } + + /** + * @param restartZk If {@code true} in background restarts on of ZK servers. + * @param closeClientSock If {@code true} in background closes zk clients' sockets. + * @throws Exception If failed. + */ + private void randomTopologyChanges(boolean restartZk, boolean closeClientSock) throws Exception { + sesTimeout = 30_000; + + if (closeClientSock) + testSockNio = true; + + List startedNodes = new ArrayList<>(); + List startedCaches = new ArrayList<>(); + + int nextNodeIdx = 0; + int nextCacheIdx = 0; + + long stopTime = System.currentTimeMillis() + 60_000; + + int MAX_NODES = 20; + int MAX_CACHES = 10; + + AtomicBoolean stop = new AtomicBoolean(); + + IgniteInternalFuture fut1 = restartZk ? startRestartZkServers(stopTime, stop) : null; + IgniteInternalFuture fut2 = closeClientSock ? startCloseZkClientSocket(stopTime, stop) : null; + + try { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + while (System.currentTimeMillis() < stopTime) { + if (startedNodes.size() > 0 && rnd.nextInt(10) == 0) { + boolean startCache = startedCaches.size() < 2 || + (startedCaches.size() < MAX_CACHES && rnd.nextInt(5) != 0); + + int nodeIdx = startedNodes.get(rnd.nextInt(startedNodes.size())); + + if (startCache) { + String cacheName = "cache-" + nextCacheIdx++; + + log.info("Next, start new cache [cacheName=" + cacheName + + ", node=" + nodeIdx + + ", crd=" + (startedNodes.isEmpty() ? null : Collections.min(startedNodes)) + + ", curCaches=" + startedCaches.size() + ']'); + + ignite(nodeIdx).createCache(new CacheConfiguration<>(cacheName)); + + startedCaches.add(cacheName); + } + else { + if (startedCaches.size() > 1) { + String cacheName = startedCaches.get(rnd.nextInt(startedCaches.size())); + + log.info("Next, stop cache [nodeIdx=" + nodeIdx + + ", node=" + nodeIdx + + ", crd=" + (startedNodes.isEmpty() ? null : Collections.min(startedNodes)) + + ", cacheName=" + startedCaches.size() + ']'); + + ignite(nodeIdx).destroyCache(cacheName); + + assertTrue(startedCaches.remove(cacheName)); + } + } + } + else { + boolean startNode = startedNodes.size() < 2 || + (startedNodes.size() < MAX_NODES && rnd.nextInt(5) != 0); + + if (startNode) { + int nodeIdx = nextNodeIdx++; + + log.info("Next, start new node [nodeIdx=" + nodeIdx + + ", crd=" + (startedNodes.isEmpty() ? null : Collections.min(startedNodes)) + + ", curNodes=" + startedNodes.size() + ']'); + + startGrid(nodeIdx); + + assertTrue(startedNodes.add(nodeIdx)); + } + else { + if (startedNodes.size() > 1) { + int nodeIdx = startedNodes.get(rnd.nextInt(startedNodes.size())); + + log.info("Next, stop [nodeIdx=" + nodeIdx + + ", crd=" + (startedNodes.isEmpty() ? null : Collections.min(startedNodes)) + + ", curNodes=" + startedNodes.size() + ']'); + + stopGrid(nodeIdx); + + assertTrue(startedNodes.remove((Integer)nodeIdx)); + } + } + } + + U.sleep(rnd.nextInt(100) + 1); + } + } + finally { + stop.set(true); + } + + if (fut1 != null) + fut1.get(); + + if (fut2 != null) + fut2.get(); + } + + /** + * + */ + private void reset() { + System.clearProperty(ZOOKEEPER_CLIENT_CNXN_SOCKET); + + ZkTestClientCnxnSocketNIO.reset(); + + System.clearProperty(ZOOKEEPER_CLIENT_CNXN_SOCKET); + + err = false; + + evts.clear(); + + try { + cleanPersistenceDir(); + } + catch (Exception e) { + error("Failed to delete DB files: " + e, e); + } + + clientThreadLoc.set(null); + } + + /** + * @param stopTime Stop time. + * @param stop Stop flag. + * @return Future. + */ + private IgniteInternalFuture startRestartZkServers(final long stopTime, final AtomicBoolean stop) { + return GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + while (!stop.get() && System.currentTimeMillis() < stopTime) { + U.sleep(rnd.nextLong(500) + 500); + + int idx = rnd.nextInt(ZK_SRVS); + + log.info("Restart ZK server: " + idx); + + zkCluster.getServers().get(idx).restart(); + + } + + return null; + } + }, "zk-restart-thread"); + } + + /** + * @param stopTime Stop time. + * @param stop Stop flag. + * @return Future. + */ + private IgniteInternalFuture startCloseZkClientSocket(final long stopTime, final AtomicBoolean stop) { + assert testSockNio; + + return GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + while (!stop.get() && System.currentTimeMillis() < stopTime) { + U.sleep(rnd.nextLong(100) + 50); + + List nodes = G.allGrids(); + + if (nodes.size() > 0) { + Ignite node = nodes.get(rnd.nextInt(nodes.size())); + + ZkTestClientCnxnSocketNIO nio = ZkTestClientCnxnSocketNIO.forNode(node); + + if (nio != null) { + info("Close zk client socket for node: " + node.name()); + + try { + nio.closeSocket(false); + } + catch (Exception e) { + info("Failed to close zk client socket for node: " + node.name()); + } + } + } + } + + return null; + } + }, "zk-restart-thread"); + } + + /** + * @param node Node. + * @throws Exception If failed. + */ + private void waitForEventsAcks(final Ignite node) throws Exception { + assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + Map evts = GridTestUtils.getFieldValue(node.configuration().getDiscoverySpi(), + "impl", "rtState", "evtsData", "evts"); + + if (!evts.isEmpty()) { + info("Unacked events: " + evts); + + return false; + } + + return true; + } + }, 10_000)); + } + + /** + * + */ + @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter") + private void checkEventsConsistency() { + for (Map.Entry> nodeEvtEntry : evts.entrySet()) { + UUID nodeId = nodeEvtEntry.getKey(); + Map nodeEvts = nodeEvtEntry.getValue(); + + for (Map.Entry> nodeEvtEntry0 : evts.entrySet()) { + if (!nodeId.equals(nodeEvtEntry0.getKey())) { + Map nodeEvts0 = nodeEvtEntry0.getValue(); + + synchronized (nodeEvts) { + synchronized (nodeEvts0) { + checkEventsConsistency(nodeEvts, nodeEvts0); + } + } + } + } + } + } + + /** + * @param evts1 Received events. + * @param evts2 Received events. + */ + private void checkEventsConsistency(Map evts1, Map evts2) { + for (Map.Entry e1 : evts1.entrySet()) { + DiscoveryEvent evt1 = e1.getValue(); + DiscoveryEvent evt2 = evts2.get(e1.getKey()); + + if (evt2 != null) { + assertEquals(evt1.topologyVersion(), evt2.topologyVersion()); + assertEquals(evt1.eventNode(), evt2.eventNode()); + assertEquals(evt1.topologyNodes(), evt2.topologyNodes()); + } + } + } + + /** + * @param node Node. + * @return Node's discovery SPI. + */ + private static ZookeeperDiscoverySpi spi(Ignite node) { + return (ZookeeperDiscoverySpi)node.configuration().getDiscoverySpi(); + } + + /** + * @param nodeName Node name. + * @return Node's discovery SPI. + * @throws Exception If failed. + */ + private ZookeeperDiscoverySpi waitSpi(final String nodeName) throws Exception { + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + ZookeeperDiscoverySpi spi = spis.get(nodeName); + + return spi != null && GridTestUtils.getFieldValue(spi, "impl") != null; + + } + }, 5000); + + ZookeeperDiscoverySpi spi = spis.get(nodeName); + + assertNotNull("Failed to get SPI for node: " + nodeName, spi); + + return spi; + } + + /** + * @param topVer Topology version. + * @return Expected event instance. + */ + private static DiscoveryEvent joinEvent(long topVer) { + DiscoveryEvent expEvt = new DiscoveryEvent(null, null, EventType.EVT_NODE_JOINED, null); + + expEvt.topologySnapshot(topVer, null); + + return expEvt; + } + + /** + * @param topVer Topology version. + * @return Expected event instance. + */ + private static DiscoveryEvent failEvent(long topVer) { + DiscoveryEvent expEvt = new DiscoveryEvent(null, null, EventType.EVT_NODE_FAILED, null); + + expEvt.topologySnapshot(topVer, null); + + return expEvt; + } + + /** + * @param node Node. + * @param expEvts Expected events. + * @throws Exception If fialed. + */ + private void checkEvents(final Ignite node, final DiscoveryEvent...expEvts) throws Exception { + checkEvents(node.cluster().localNode().id(), expEvts); + } + + /** + * @param nodeId Node ID. + * @param expEvts Expected events. + * @throws Exception If failed. + */ + private void checkEvents(final UUID nodeId, final DiscoveryEvent...expEvts) throws Exception { + assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() { + @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter") + @Override public boolean apply() { + Map nodeEvts = evts.get(nodeId); + + if (nodeEvts == null) { + info("No events for node: " + nodeId); + + return false; + } + + synchronized (nodeEvts) { + for (DiscoveryEvent expEvt : expEvts) { + DiscoveryEvent evt0 = nodeEvts.get(expEvt.topologyVersion()); + + if (evt0 == null) { + info("No event for version: " + expEvt.topologyVersion()); + + return false; + } + + assertEquals("Unexpected event [topVer=" + expEvt.topologyVersion() + + ", exp=" + U.gridEventName(expEvt.type()) + + ", evt=" + evt0 + ']', expEvt.type(), evt0.type()); + } + } + + return true; + } + }, 30000)); + } + + /** + * @param spi Spi instance. + */ + private static void closeZkClient(ZookeeperDiscoverySpi spi) { + ZooKeeper zk = zkClient(spi); + + try { + zk.close(); + } + catch (Exception e) { + fail("Unexpected error: " + e); + } + } + + /** + * @param spi Spi instance. + * @return Zookeeper client. + */ + private static ZooKeeper zkClient(ZookeeperDiscoverySpi spi) { + return GridTestUtils.getFieldValue(spi, "impl", "rtState", "zkClient", "zk"); + } + + /** + * Reconnect client node. + * + * @param log Logger. + * @param clients Clients. + * @param closeSock {@code True} to simulate reconnect by closing zk client's socket. + * @throws Exception If failed. + */ + private static void reconnectClientNodes(final IgniteLogger log, + List clients, + boolean closeSock) + throws Exception { + final CountDownLatch disconnectLatch = new CountDownLatch(clients.size()); + final CountDownLatch reconnectLatch = new CountDownLatch(clients.size()); + + IgnitePredicate p = new IgnitePredicate() { + @Override public boolean apply(Event evt) { + if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) { + log.info("Disconnected: " + evt); + + disconnectLatch.countDown(); + } + else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) { + log.info("Reconnected: " + evt); + + reconnectLatch.countDown(); + } + + return true; + } + }; + + List zkNodes = new ArrayList<>(); + + for (Ignite client : clients) { + client.events().localListen(p, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED); + + zkNodes.add(aliveZkNodePath(client)); + } + + long timeout = 15_000; + + if (closeSock) { + for (Ignite client : clients) { + ZookeeperDiscoverySpi spi = (ZookeeperDiscoverySpi)client.configuration().getDiscoverySpi(); + + ZkTestClientCnxnSocketNIO.forNode(client.name()).closeSocket(true); + + timeout = Math.max(timeout, (long)(spi.getSessionTimeout() * 1.5f)); + } + } + else { + /* + * Use hack to simulate session expire without waiting session timeout: + * create and close ZooKeeper with the same session ID as ignite node's ZooKeeper. + */ + List dummyClients = new ArrayList<>(); + + for (Ignite client : clients) { + ZookeeperDiscoverySpi spi = (ZookeeperDiscoverySpi)client.configuration().getDiscoverySpi(); + + ZooKeeper zk = zkClient(spi); + + ZooKeeper dummyZk = new ZooKeeper( + spi.getZkConnectionString(), + 10_000, + null, + zk.getSessionId(), + zk.getSessionPasswd()); + + dummyZk.exists("/a", false); + + dummyClients.add(dummyZk); + } + + for (ZooKeeper zk : dummyClients) + zk.close(); + } + + waitNoAliveZkNodes(log, + ((ZookeeperDiscoverySpi)clients.get(0).configuration().getDiscoverySpi()).getZkConnectionString(), + zkNodes, + timeout); + + if (closeSock) { + for (Ignite client : clients) + ZkTestClientCnxnSocketNIO.forNode(client.name()).allowConnect(); + } + + waitReconnectEvent(log, disconnectLatch); + + waitReconnectEvent(log, reconnectLatch); + + for (Ignite client : clients) + client.events().stopLocalListen(p); + } + + /** + * @param zk ZooKeeper client. + * @param root Root path. + * @return All children znodes for given path. + * @throws Exception If failed/ + */ + private List listSubTree(ZooKeeper zk, String root) throws Exception { + for (int i = 0; i < 30; i++) { + try { + return ZKUtil.listSubTreeBFS(zk, root); + } + catch (KeeperException.NoNodeException e) { + info("NoNodeException when get znodes, will retry: " + e); + } + } + + throw new Exception("Failed to get znodes: " + root); + } + + /** + * @param log Logger. + * @param latch Latch. + * @throws Exception If failed. + */ + private static void waitReconnectEvent(IgniteLogger log, CountDownLatch latch) throws Exception { + if (!latch.await(30_000, MILLISECONDS)) { + log.error("Failed to wait for reconnect event, will dump threads, latch count: " + latch.getCount()); + + U.dumpThreads(log); + + fail("Failed to wait for disconnect/reconnect event."); + } + } + + /** + * @param cacheName Cache name. + * @return Configuration. + */ + private CacheConfiguration largeCacheConfiguration(String cacheName) { + CacheConfiguration ccfg = new CacheConfiguration<>(cacheName); + + ccfg.setAffinity(new TestAffinityFunction(1024 * 1024)); + ccfg.setWriteSynchronizationMode(FULL_SYNC); + + return ccfg; + } + + /** {@inheritDoc} */ + @Override protected void waitForTopology(int expSize) throws Exception { + super.waitForTopology(expSize); + + // checkZkNodesCleanup(); + } + + /** + * + */ + @SuppressWarnings("MismatchedReadAndWriteOfArray") + static class TestAffinityFunction extends RendezvousAffinityFunction { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private int[] dummyData; + + /** + * @param dataSize Dummy data size. + */ + TestAffinityFunction(int dataSize) { + dummyData = new int[dataSize]; + + for (int i = 0; i < dataSize; i++) + dummyData[i] = i; + } + } + + /** + * + */ + private static class DummyCallable implements IgniteCallable { + /** */ + private byte[] data; + + /** + * @param data Data. + */ + DummyCallable(byte[] data) { + this.data = data; + } + + /** {@inheritDoc} */ + @Override public Object call() throws Exception { + return data; + } + } + + /** + * + */ + private static class C1 implements Serializable { + // No-op. + } + + /** + * + */ + private static class C2 implements Serializable { + // No-op. + } + + /** + * + */ + static class ZkTestNodeAuthenticator implements DiscoverySpiNodeAuthenticator { + /** + * @param failAuthNodes Node names which should not pass authentication. + * @return Factory. + */ + static IgniteOutClosure factory(final String...failAuthNodes) { + return new IgniteOutClosure() { + @Override public DiscoverySpiNodeAuthenticator apply() { + return new ZkTestNodeAuthenticator(Arrays.asList(failAuthNodes)); + } + }; + } + + /** */ + private final Collection failAuthNodes; + + /** + * @param failAuthNodes Node names which should not pass authentication. + */ + ZkTestNodeAuthenticator(Collection failAuthNodes) { + this.failAuthNodes = failAuthNodes; + } + + /** {@inheritDoc} */ + @Override public SecurityContext authenticateNode(ClusterNode node, SecurityCredentials cred) { + assertNotNull(cred); + + String nodeName = node.attribute(ATTR_IGNITE_INSTANCE_NAME); + + assertEquals(nodeName, cred.getUserObject()); + + boolean auth = !failAuthNodes.contains(nodeName); + + System.out.println(Thread.currentThread().getName() + " authenticateNode [node=" + node.id() + ", res=" + auth + ']'); + + return auth ? new TestSecurityContext(nodeName) : null; + } + + /** {@inheritDoc} */ + @Override public boolean isGlobalNodeAuthentication() { + return false; + } + + /** + * + */ + private static class TestSecurityContext implements SecurityContext, Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** */ + final String nodeName; + + /** + * @param nodeName Authenticated node name. + */ + TestSecurityContext(String nodeName) { + this.nodeName = nodeName; + } + + /** {@inheritDoc} */ + @Override public SecuritySubject subject() { + return null; + } + + /** {@inheritDoc} */ + @Override public boolean taskOperationAllowed(String taskClsName, SecurityPermission perm) { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean cacheOperationAllowed(String cacheName, SecurityPermission perm) { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean serviceOperationAllowed(String srvcName, SecurityPermission perm) { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean systemOperationAllowed(SecurityPermission perm) { + return true; + } + } + } + + /** + * + */ + static class CacheInfoCommunicationFailureResolver implements CommunicationFailureResolver { + /** */ + @LoggerResource + private IgniteLogger log; + + /** */ + Map> caches; + + /** */ + Map>> affMap; + + /** */ + Map>> ownersMap; + + /** */ + volatile CountDownLatch latch; + + /** {@inheritDoc} */ + @Override public void resolve(CommunicationFailureContext ctx) { + assert latch != null; + assert latch.getCount() == 1L : latch.getCount(); + + caches = ctx.startedCaches(); + + log.info("Resolver called, started caches: " + caches.keySet()); + + assertNotNull(caches); + + affMap = new HashMap<>(); + ownersMap = new HashMap<>(); + + for (String cache : caches.keySet()) { + affMap.put(cache, ctx.cacheAffinity(cache)); + ownersMap.put(cache, ctx.cachePartitionOwners(cache)); + } + + latch.countDown(); + } + + /** + * @param expCaches Expected caches information (when late assignment doen and rebalance finished). + */ + void checkCachesInfo(Map> expCaches) { + assertNotNull(caches); + assertNotNull(affMap); + assertNotNull(ownersMap); + + for (Map.Entry> e : expCaches.entrySet()) { + String cacheName = e.getKey(); + + int parts = e.getValue().get1(); + int backups = e.getValue().get2(); + int expNodes = e.getValue().get3(); + + assertTrue(cacheName, caches.containsKey(cacheName)); + + CacheConfiguration ccfg = caches.get(cacheName); + + assertEquals(cacheName, ccfg.getName()); + + if (ccfg.getCacheMode() == CacheMode.REPLICATED) + assertEquals(Integer.MAX_VALUE, ccfg.getBackups()); + else + assertEquals(backups, ccfg.getBackups()); + + assertEquals(parts, ccfg.getAffinity().partitions()); + + List> aff = affMap.get(cacheName); + + assertNotNull(cacheName, aff); + assertEquals(parts, aff.size()); + + List> owners = ownersMap.get(cacheName); + + assertNotNull(cacheName, owners); + assertEquals(parts, owners.size()); + + for (int i = 0; i < parts; i++) { + List partAff = aff.get(i); + + assertEquals(cacheName, expNodes, partAff.size()); + + List partOwners = owners.get(i); + + assertEquals(cacheName, expNodes, partOwners.size()); + + assertTrue(cacheName, partAff.containsAll(partOwners)); + assertTrue(cacheName, partOwners.containsAll(partAff)); + } + } + } + + /** + * + */ + void reset() { + caches = null; + affMap = null; + ownersMap = null; + } + } + + /** + * + */ + static class NoOpCommunicationFailureResolver implements CommunicationFailureResolver { + /** */ + static final IgniteOutClosure FACTORY = new IgniteOutClosure() { + @Override public CommunicationFailureResolver apply() { + return new NoOpCommunicationFailureResolver(); + } + }; + + /** {@inheritDoc} */ + @Override public void resolve(CommunicationFailureContext ctx) { + // No-op. + } + } + + /** + * + */ + static class KillCoordinatorCommunicationFailureResolver implements CommunicationFailureResolver { + /** */ + static final IgniteOutClosure FACTORY = new IgniteOutClosure() { + @Override public CommunicationFailureResolver apply() { + return new KillCoordinatorCommunicationFailureResolver(); + } + }; + + /** */ + @LoggerResource + private IgniteLogger log; + + /** {@inheritDoc} */ + @Override public void resolve(CommunicationFailureContext ctx) { + List nodes = ctx.topologySnapshot(); + + ClusterNode node = nodes.get(0); + + log.info("Resolver kills node: " + node.id()); + + ctx.killNode(node); + } + } + + /** + * + */ + static class KillRandomCommunicationFailureResolver implements CommunicationFailureResolver { + /** */ + static final IgniteOutClosure FACTORY = new IgniteOutClosure() { + @Override public CommunicationFailureResolver apply() { + return new KillRandomCommunicationFailureResolver(); + } + }; + + /** */ + @LoggerResource + private IgniteLogger log; + + /** {@inheritDoc} */ + @Override public void resolve(CommunicationFailureContext ctx) { + List nodes = ctx.topologySnapshot(); + + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + int killNodes = rnd.nextInt(nodes.size() / 2); + + log.info("Resolver kills nodes [total=" + nodes.size() + ", kill=" + killNodes + ']'); + + Set idxs = new HashSet<>(); + + while (idxs.size() < killNodes) + idxs.add(rnd.nextInt(nodes.size())); + + for (int idx : idxs) { + ClusterNode node = nodes.get(idx); + + log.info("Resolver kills node: " + node.id()); + + ctx.killNode(node); + } + } + } + + /** + * + */ + static class TestNodeKillCommunicationFailureResolver implements CommunicationFailureResolver { + /** + * @param killOrders Killed nodes order. + * @return Factory. + */ + static IgniteOutClosure factory(final Collection killOrders) { + return new IgniteOutClosure() { + @Override public CommunicationFailureResolver apply() { + return new TestNodeKillCommunicationFailureResolver(killOrders); + } + }; + } + + /** */ + final Collection killNodeOrders; + + /** + * @param killOrders Killed nodes order. + */ + TestNodeKillCommunicationFailureResolver(Collection killOrders) { + this.killNodeOrders = killOrders; + } + + /** {@inheritDoc} */ + @Override public void resolve(CommunicationFailureContext ctx) { + List nodes = ctx.topologySnapshot(); + + assertTrue(nodes.size() > 0); + + for (ClusterNode node : nodes) { + if (killNodeOrders.contains(node.order())) + ctx.killNode(node); + } + } + } + + /** + * + */ + static class ZkTestCommunicationSpi extends TestRecordingCommunicationSpi { + /** */ + private volatile CountDownLatch pingStartLatch; + + /** */ + private volatile CountDownLatch pingLatch; + + /** */ + private volatile BitSet checkRes; + + /** + * @param ignite Node. + * @return Node's communication SPI. + */ + static ZkTestCommunicationSpi testSpi(Ignite ignite) { + return (ZkTestCommunicationSpi)ignite.configuration().getCommunicationSpi(); + } + + /** + * @param nodes Number of nodes. + * @param setBitIdxs Bits indexes to set in check result. + */ + void initCheckResult(int nodes, Integer... setBitIdxs) { + checkRes = new BitSet(nodes); + + for (Integer bitIdx : setBitIdxs) + checkRes.set(bitIdx); + } + + /** {@inheritDoc} */ + @Override public IgniteFuture checkConnection(List nodes) { + CountDownLatch pingStartLatch = this.pingStartLatch; + + if (pingStartLatch != null) + pingStartLatch.countDown(); + + CountDownLatch pingLatch = this.pingLatch; + + try { + if (pingLatch != null) + pingLatch.await(); + } + catch (InterruptedException e) { + throw new IgniteException(e); + } + + BitSet checkRes = this.checkRes; + + if (checkRes != null) { + this.checkRes = null; + + return new IgniteFinishedFutureImpl<>(checkRes); + } + + return super.checkConnection(nodes); + } + } + + /** + * + */ + static class TestFastStopProcessCustomMessage implements DiscoveryCustomMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private final IgniteUuid id = IgniteUuid.randomUuid(); + + /** */ + private final boolean createAck; + + /** */ + private final int payload; + + /** + * @param createAck Create ack message flag. + * @param payload Payload. + */ + TestFastStopProcessCustomMessage(boolean createAck, int payload) { + this.createAck = createAck; + this.payload = payload; + + } + + /** {@inheritDoc} */ + @Override public IgniteUuid id() { + return id; + } + + /** {@inheritDoc} */ + @Nullable @Override public DiscoveryCustomMessage ackMessage() { + return createAck ? new TestFastStopProcessCustomMessageAck(payload) : null; + } + + /** {@inheritDoc} */ + @Override public boolean isMutable() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return true; + } + + /** {@inheritDoc} */ + @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, + AffinityTopologyVersion topVer, + DiscoCache discoCache) { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + TestFastStopProcessCustomMessage that = (TestFastStopProcessCustomMessage)o; + + return createAck == that.createAck && payload == that.payload; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return Objects.hash(createAck, payload); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(TestFastStopProcessCustomMessage.class, this); + } + } + + /** + * + */ + static class TestFastStopProcessCustomMessageAck implements DiscoveryCustomMessage { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private final IgniteUuid id = IgniteUuid.randomUuid(); + + /** */ + private final int payload; + + /** + * @param payload Payload. + */ + TestFastStopProcessCustomMessageAck(int payload) { + this.payload = payload; + } + + /** {@inheritDoc} */ + @Override public IgniteUuid id() { + return id; + } + + /** {@inheritDoc} */ + @Nullable @Override public DiscoveryCustomMessage ackMessage() { + return null; + } + + /** {@inheritDoc} */ + @Override public boolean isMutable() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean stopProcess() { + return true; + } + + /** {@inheritDoc} */ + @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, + AffinityTopologyVersion topVer, + DiscoCache discoCache) { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + TestFastStopProcessCustomMessageAck that = (TestFastStopProcessCustomMessageAck)o; + return payload == that.payload; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return Objects.hash(payload); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(TestFastStopProcessCustomMessageAck.class, this); + } + } +} diff --git a/modules/zookeeper/src/test/java/org/apache/zookeeper/ZkTestClientCnxnSocketNIO.java b/modules/zookeeper/src/test/java/org/apache/zookeeper/ZkTestClientCnxnSocketNIO.java new file mode 100644 index 0000000000000..7892b5e7f2bd9 --- /dev/null +++ b/modules/zookeeper/src/test/java/org/apache/zookeeper/ZkTestClientCnxnSocketNIO.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.channels.SelectionKey; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.logger.java.JavaLogger; +import org.apache.ignite.testframework.GridTestUtils; + +/** + * + */ +public class ZkTestClientCnxnSocketNIO extends ClientCnxnSocketNIO { + /** */ + public static final IgniteLogger log = new JavaLogger().getLogger(ZkTestClientCnxnSocketNIO.class); + + /** */ + public static volatile boolean DEBUG = false; + + /** */ + public volatile CountDownLatch blockConnectLatch; + + /** */ + public static ConcurrentHashMap clients = new ConcurrentHashMap<>(); + + /** */ + private final String nodeName; + + /** + * + */ + public static void reset() { + clients.clear(); + } + + /** + * @param node Node. + * @return ZK client. + */ + public static ZkTestClientCnxnSocketNIO forNode(Ignite node) { + return clients.get(node.name()); + } + + /** + * @param instanceName Ignite instance name. + * @return ZK client. + */ + public static ZkTestClientCnxnSocketNIO forNode(String instanceName) { + return clients.get(instanceName); + } + + /** + * @throws IOException If failed. + */ + public ZkTestClientCnxnSocketNIO() throws IOException { + super(); + + String threadName = Thread.currentThread().getName(); + + nodeName = threadName.substring(threadName.indexOf('-') + 1); + + if (DEBUG) + log.info("ZkTestClientCnxnSocketNIO created for node: " + nodeName); + } + + /** {@inheritDoc} */ + @Override void connect(InetSocketAddress addr) throws IOException { + CountDownLatch blockConnect = this.blockConnectLatch; + + if (DEBUG) + log.info("ZkTestClientCnxnSocketNIO connect [node=" + nodeName + ", addr=" + addr + ']'); + + if (blockConnect != null && blockConnect.getCount() > 0) { + try { + log.info("ZkTestClientCnxnSocketNIO block connect"); + + blockConnect.await(60, TimeUnit.SECONDS); + + log.info("ZkTestClientCnxnSocketNIO finish block connect"); + } + catch (Exception e) { + log.error("Error in ZkTestClientCnxnSocketNIO: " + e, e); + } + } + + super.connect(addr); + + clients.put(nodeName, this); + } + + /** + * + */ + public void allowConnect() { + assert blockConnectLatch != null && blockConnectLatch.getCount() == 1 : blockConnectLatch; + + log.info("ZkTestClientCnxnSocketNIO allowConnect [node=" + nodeName + ']'); + + blockConnectLatch.countDown(); + } + + /** + * @param blockConnect {@code True} to block client reconnect. + * @throws Exception If failed. + */ + public void closeSocket(boolean blockConnect) throws Exception { + if (blockConnect) + blockConnectLatch = new CountDownLatch(1); + + log.info("ZkTestClientCnxnSocketNIO closeSocket [node=" + nodeName + ", block=" + blockConnect + ']'); + + SelectionKey k = GridTestUtils.getFieldValue(this, ClientCnxnSocketNIO.class, "sockKey"); + + k.channel().close(); + } +} From b096a463c338565a7661f8a853a257518d872997 Mon Sep 17 00:00:00 2001 From: Stanislav Lukyanov Date: Mon, 9 Apr 2018 14:33:13 +0300 Subject: [PATCH 0010/1463] IGNITE-7904: Changed IgniteUtils::cast not to trim exception chains. This closes #3683. --- .../ignite/compute/ComputeTaskAdapter.java | 2 +- .../processors/cache/GridCacheUtils.java | 5 +- .../cache/IgniteCacheProxyImpl.java | 3 + .../processors/igfs/IgfsMetaManager.java | 30 ++- .../processors/job/GridJobWorker.java | 2 +- .../platform/services/PlatformServices.java | 8 +- .../processors/service/GridServiceProxy.java | 27 ++- .../ignite/internal/util/IgniteUtils.java | 32 +-- .../IgniteComputeResultExceptionTest.java | 186 ++++++++++++++++++ .../GridCacheAbstractFullApiSelfTest.java | 9 +- .../closure/GridClosureSerializationTest.java | 2 +- .../GridServiceProcessorProxySelfTest.java | 12 +- .../IgniteComputeGridTestSuite.java | 2 + ...teCacheLockPartitionOnAffinityRunTest.java | 46 ++--- .../CacheJdbcPojoStoreFactorySelfTest.java | 11 +- .../GridServiceInjectionSelfTest.java | 64 +++--- .../GridSpringResourceInjectionSelfTest.java | 58 +++--- 17 files changed, 339 insertions(+), 160 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeResultExceptionTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java index c5352aacb41ff..fc55ad9f30f8f 100644 --- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java @@ -99,7 +99,7 @@ public abstract class ComputeTaskAdapter implements ComputeTask { return ComputeJobResultPolicy.FAILOVER; throw new IgniteException("Remote job threw user exception (override or implement ComputeTask.result(..) " + - "method if you would like to have automatic failover for this exception).", e); + "method if you would like to have automatic failover for this exception): " + e.getMessage(), e); } // Wait for all job responses. 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 6d4c1f21cb273..a5169d26dcc13 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 @@ -1283,8 +1283,9 @@ else if (e instanceof ClusterTopologyServerNotFoundException) else if (e instanceof SchemaOperationException) return new CacheException(e.getMessage(), e); - if (e.getCause() instanceof CacheException) - return (CacheException)e.getCause(); + CacheException ce = X.cause(e, CacheException.class); + if (ce != null) + return ce; if (e.getCause() instanceof NullPointerException) return (NullPointerException)e.getCause(); 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 c5d68b58f1efc..be4b0dbffaf5a 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 @@ -1727,6 +1727,9 @@ private RuntimeException cacheException(Exception e) { ctx.name(), e); } + if (e instanceof IgniteException && X.hasCause(e, CacheException.class)) + e = X.cause(e, CacheException.class); + if (e instanceof IgniteCheckedException) return CU.convertToCacheException((IgniteCheckedException) e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java index a26239caa40b5..e821b80641b4d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java @@ -89,6 +89,7 @@ import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T1; +import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.transactions.TransactionConcurrency; @@ -253,29 +254,20 @@ boolean isClient() { * @return Result. */ T runClientTask(IgfsClientAbstractCallable task) { - try { - return runClientTask(IgfsUtils.ROOT_ID, task); - } - catch (ClusterTopologyException e) { - throw new IgfsException("Failed to execute operation because there are no IGFS metadata nodes." , e); - } - } - - /** - * Run client task. - * - * @param affinityFileId Affinity fileId. - * @param task Task. - * @return Result. - */ - T runClientTask(IgniteUuid affinityFileId, IgfsClientAbstractCallable task) { try { return (cfg.isColocateMetadata()) ? - clientCompute().affinityCall(metaCacheName, affinityFileId, task) : + clientCompute().affinityCall(metaCacheName, IgfsUtils.ROOT_ID, task) : clientCompute().call(task); } - catch (ClusterTopologyException e) { - throw new IgfsException("Failed to execute operation because there are no IGFS metadata nodes." , e); + catch (Exception e) { + if (X.hasCause(e, ClusterTopologyException.class)) + throw new IgfsException("Failed to execute operation because there are no IGFS metadata nodes." , e); + + IgfsException igfsEx = X.cause(e, IgfsException.class); + if (igfsEx != null) + throw igfsEx; + + throw e; } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java index cce1077d42425..6d2e621c3b9af 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java @@ -678,7 +678,7 @@ else if (sysStopping && X.hasCause(e, InterruptedException.class, IgniteInterrup if (msg == null) { msg = "Failed to execute job due to unexpected runtime exception [jobId=" + ses.getJobId() + - ", ses=" + ses + ']'; + ", ses=" + ses + ", err=" + e.getMessage() + ']'; ex = new ComputeUserUndeclaredException(msg, e); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java index ccb04d4d6c0d6..4ae59b99929e3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java @@ -33,6 +33,7 @@ import org.apache.ignite.internal.processors.platform.utils.PlatformWriterBiClosure; import org.apache.ignite.internal.processors.platform.utils.PlatformWriterClosure; import org.apache.ignite.internal.processors.service.GridServiceProxy; +import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.typedef.T3; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgnitePredicate; @@ -581,7 +582,12 @@ public Object invoke(String mthdName, boolean srvKeepBinary, Object[] args) Method mtd = getMethod(serviceClass, mthdName, args); - return ((GridServiceProxy)proxy).invokeMethod(mtd, args); + try { + return ((GridServiceProxy)proxy).invokeMethod(mtd, args); + } + catch (Throwable t) { + throw IgniteUtils.cast(t); + } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java index e55c2e5d4099b..c5a2ceea19f54 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java @@ -47,6 +47,7 @@ import org.apache.ignite.internal.managers.communication.GridIoPolicy; import org.apache.ignite.internal.util.tostring.GridToStringExclude; 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.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteCallable; @@ -150,7 +151,7 @@ private boolean hasLocalNode(ClusterGroup prj) { * @return Result. */ @SuppressWarnings("BusyWait") - public Object invokeMethod(final Method mtd, final Object[] args) { + public Object invokeMethod(final Method mtd, final Object[] args) throws Throwable { if (U.isHashCodeMethod(mtd)) return System.identityHashCode(proxy); else if (U.isEqualsMethod(mtd)) @@ -205,6 +206,12 @@ else if (U.isToStringMethod(mtd)) throw e; } catch (IgniteCheckedException e) { + // Rethrow original service method exception so that calling user code can handle it correctly. + ServiceProxyException svcProxyE = X.cause(e, ServiceProxyException.class); + + if (svcProxyE != null) + throw svcProxyE.getCause(); + throw U.convertException(e); } catch (Exception e) { @@ -352,7 +359,7 @@ private class ProxyInvocationHandler implements InvocationHandler { /** {@inheritDoc} */ @SuppressWarnings("BusyWait") - @Override public Object invoke(Object proxy, final Method mtd, final Object[] args) { + @Override public Object invoke(Object proxy, final Method mtd, final Object[] args) throws Throwable { return invokeMethod(mtd, args); } } @@ -418,8 +425,7 @@ private ServiceProxyCallable(String mtdName, String svcName, Class[] argTypes, O return mtd.invoke(svcCtx.service(), args); } catch (InvocationTargetException e) { - // Get error message. - throw new IgniteCheckedException(e.getCause().getMessage(), e); + throw new ServiceProxyException(e.getCause()); } } @@ -444,4 +450,17 @@ private ServiceProxyCallable(String mtdName, String svcName, Class[] argTypes, O return S.toString(ServiceProxyCallable.class, this); } } + + /** + * Exception class that wraps an exception thrown by the service implementation. + */ + private static class ServiceProxyException extends RuntimeException { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + ServiceProxyException(Throwable cause) { + super(cause); + } + } } 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 18e182dde22c8..93f4fb4a39f2e 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 @@ -7285,35 +7285,23 @@ public static Exception unwrap(Throwable t) { } /** - * Casts this throwable as {@link IgniteCheckedException}. Creates wrapping - * {@link IgniteCheckedException}, if needed. + * Casts the passed {@code Throwable t} to {@link IgniteCheckedException}.
    + * If {@code t} is a {@link GridClosureException}, it is unwrapped and then cast to {@link IgniteCheckedException}. + * If {@code t} is an {@link IgniteCheckedException}, it is returned. + * If {@code t} is not a {@link IgniteCheckedException}, a new {@link IgniteCheckedException} caused by {@code t} + * is returned. * * @param t Throwable to cast. - * @return Grid exception. + * @return {@code t} cast to {@link IgniteCheckedException}. */ public static IgniteCheckedException cast(Throwable t) { assert t != null; - while (true) { - if (t instanceof Error) - throw (Error)t; + t = unwrap(t); - if (t instanceof GridClosureException) { - t = ((GridClosureException)t).unwrap(); - - continue; - } - - if (t instanceof IgniteCheckedException) - return (IgniteCheckedException)t; - - if (!(t instanceof IgniteException) || t.getCause() == null) - return new IgniteCheckedException(t); - - assert t.getCause() != null; // ...and it is IgniteException. - - t = t.getCause(); - } + return t instanceof IgniteCheckedException + ? (IgniteCheckedException)t + : new IgniteCheckedException(t); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeResultExceptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeResultExceptionTest.java new file mode 100644 index 0000000000000..fab5de6d25229 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeResultExceptionTest.java @@ -0,0 +1,186 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCompute; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.compute.ComputeJob; +import org.apache.ignite.compute.ComputeJobResult; +import org.apache.ignite.compute.ComputeJobResultPolicy; +import org.apache.ignite.compute.ComputeTask; +import org.apache.ignite.compute.ComputeTaskFuture; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.Nullable; + +/** + * Testing that if {@link ComputeTask#result(ComputeJobResult, List)} throws an {@link IgniteException} + * then that exception is thrown as the execution result. + */ +public class IgniteComputeResultExceptionTest extends GridCommonAbstractTest { + /** */ + public void testIgniteExceptionExecute() throws Exception { + checkExecuteException(new IgniteException()); + } + + /** */ + public void testIgniteExceptionWithCauseExecute() throws Exception { + checkExecuteException(new IgniteException(new Exception())); + } + + /** */ + public void testIgniteExceptionWithCauseChainExecute() throws Exception { + checkExecuteException(new IgniteException(new Exception(new Throwable()))); + } + + /** */ + public void testCustomExceptionExecute() throws Exception { + checkExecuteException(new TaskException()); + } + + /** */ + public void testCustomExceptionWithCauseExecute() throws Exception { + checkExecuteException(new TaskException(new Exception())); + } + + /** */ + public void testCustomExceptionWithCauseChainExecute() throws Exception { + checkExecuteException(new TaskException(new Exception(new Throwable()))); + } + + /** */ + private void checkExecuteException(IgniteException resE) throws Exception { + try (Ignite ignite = startGrid()) { + IgniteCompute compute = ignite.compute(); + try { + compute.execute(new ResultExceptionTask(resE), null); + } catch (IgniteException e) { + assertSame(resE, e); + } + } + } + + /** */ + public void testIgniteExceptionExecuteAsync() throws Exception { + checkExecuteAsyncException(new IgniteException()); + } + + /** */ + public void testIgniteExceptionWithCauseExecuteAsync() throws Exception { + checkExecuteAsyncException(new IgniteException(new Exception())); + } + + /** */ + public void testIgniteExceptionWithCauseChainExecuteAsync() throws Exception { + checkExecuteAsyncException(new IgniteException(new Exception(new Throwable()))); + } + + + /** */ + public void testCustomExceptionExecuteAsync() throws Exception { + checkExecuteAsyncException(new TaskException()); + } + + /** */ + public void testCustomExceptionWithCauseExecuteAsync() throws Exception { + checkExecuteAsyncException(new TaskException(new Exception())); + } + + /** */ + public void testCustomExceptionWithCauseChainExecuteAsync() throws Exception { + checkExecuteAsyncException(new TaskException(new Exception(new Throwable()))); + } + + /** */ + private void checkExecuteAsyncException(IgniteException resE) throws Exception { + try (Ignite ignite = startGrid()) { + IgniteCompute compute = ignite.compute(); + ComputeTaskFuture fut = compute.executeAsync(new ResultExceptionTask(resE), null); + try { + fut.get(); + } catch (IgniteException e) { + assertSame(resE, e); + } + } + } + + /** */ + private static class TaskException extends IgniteException { + /** */ + public TaskException() { + // No-op. + } + + /** */ + public TaskException(Throwable cause) { + super(cause); + } + } + + /** */ + private static class NoopJob implements ComputeJob { + /** */ + @Override public void cancel() { + // No-op. + } + + /** */ + @Override public Object execute() throws IgniteException { + return null; + } + } + + /** */ + private static class ResultExceptionTask implements ComputeTask { + /** */ + private final IgniteException resE; + + /** + * @param resE Exception to be rethrown by the + */ + ResultExceptionTask(IgniteException resE) { + this.resE = resE; + } + + /** */ + @Override public Map map(List subgrid, + @Nullable Object arg) throws IgniteException { + Map jobs = new HashMap<>(); + + for (ClusterNode node : subgrid) + jobs.put(new NoopJob(), node); + + return jobs; + } + + /** */ + @Override + public ComputeJobResultPolicy result(ComputeJobResult res, List rcvd) throws IgniteException { + throw resE; + } + + /** */ + @Nullable @Override public Object reduce(List results) throws IgniteException { + return null; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java index 2e6a19cb5903a..e5df2c83d6af8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java @@ -82,6 +82,7 @@ import org.apache.ignite.internal.util.typedef.CIX1; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.PA; +import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; @@ -5041,12 +5042,16 @@ private void waitForIteratorsCleared(IgniteCache cache, int sec checkIteratorsCleared(); } - catch (AssertionFailedError e) { + catch (Throwable t) { + // If AssertionFailedError is in the chain, assume we need to wait and retry. + if (!X.hasCause(t, AssertionFailedError.class)) + throw t; + if (i == 9) { for (int j = 0; j < gridCount(); j++) executeOnLocalOrRemoteJvm(j, new PrintIteratorStateTask()); - throw e; + throw t; } log.info("Iterators not cleared, will wait"); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureSerializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureSerializationTest.java index 8bcbd817574e3..324a9e9c99252 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureSerializationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureSerializationTest.java @@ -68,7 +68,7 @@ public void testSerializationFailure() throws Exception { final IgniteEx ignite0 = grid(0); final IgniteEx ignite1 = grid(1); - GridTestUtils.assertThrows(null, new Callable() { + GridTestUtils.assertThrowsAnyCause(log, new Callable() { @Override public Object call() throws Exception { ignite1.compute(ignite1.cluster().forNode(ignite0.localNode())).call(new IgniteCallable() { @Override public Object call() throws Exception { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java index d1c5294d48604..97d5f05e9e52b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java @@ -89,7 +89,7 @@ public void testException() throws Exception { return null; } - }, IgniteException.class, "Test exception"); + }, ErrorServiceException.class, "Test exception"); } @@ -450,9 +450,15 @@ protected class ErrorServiceImpl implements ErrorService { /** {@inheritDoc} */ @Override public void go() throws Exception { - throw new Exception("Test exception"); + throw new ErrorServiceException("Test exception"); } } - + /** */ + private static class ErrorServiceException extends Exception { + /** */ + ErrorServiceException(String msg) { + super(msg); + } + } } diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java index be99adf3b169d..14eb296424a47 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java @@ -68,6 +68,7 @@ import org.apache.ignite.internal.GridTaskTimeoutSelfTest; import org.apache.ignite.internal.IgniteComputeEmptyClusterGroupTest; import org.apache.ignite.internal.IgniteComputeJobOneThreadTest; +import org.apache.ignite.internal.IgniteComputeResultExceptionTest; import org.apache.ignite.internal.IgniteComputeTopologyExceptionTest; import org.apache.ignite.internal.IgniteExecutorServiceTest; import org.apache.ignite.internal.IgniteExplicitImplicitDeploymentSelfTest; @@ -158,6 +159,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridMultinodeRedeployIsolatedModeSelfTest.class); suite.addTestSuite(IgniteComputeEmptyClusterGroupTest.class); suite.addTestSuite(IgniteComputeTopologyExceptionTest.class); + suite.addTestSuite(IgniteComputeResultExceptionTest.class); suite.addTestSuite(GridTaskFailoverAffinityRunTest.class); suite.addTestSuite(TaskNodeRestartTest.class); suite.addTestSuite(IgniteRoundRobinErrorAfterClientReconnectTest.class); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLockPartitionOnAffinityRunTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLockPartitionOnAffinityRunTest.java index 598212bd04e3a..7bddafc88231a 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLockPartitionOnAffinityRunTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheLockPartitionOnAffinityRunTest.java @@ -23,6 +23,7 @@ import java.io.ObjectOutput; import java.util.Arrays; import java.util.List; +import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteException; @@ -404,39 +405,20 @@ public void testMultipleCaches() throws Exception { public void testCheckReservePartitionException() throws Exception { int orgId = primaryKey(grid(1).cache(Organization.class.getSimpleName())); - try { - grid(0).compute().affinityRun( - Arrays.asList(Organization.class.getSimpleName(), OTHER_CACHE_NAME), - new Integer(orgId), - new IgniteRunnable() { - @Override public void run() { - // No-op. - } - }); - - fail("Exception is expected"); - } - catch (Exception e) { - assertTrue(e.getMessage() - .startsWith("Failed partition reservation. Partition is not primary on the node.")); - } - - try { - grid(0).compute().affinityCall( - Arrays.asList(Organization.class.getSimpleName(), OTHER_CACHE_NAME), - new Integer(orgId), - new IgniteCallable() { - @Override public Object call() throws Exception { - return null; - } - }); + GridTestUtils.assertThrowsAnyCause(log, new Callable() { + @Override public Void call() throws Exception { + grid(0).compute().affinityRun( + Arrays.asList(Organization.class.getSimpleName(), OTHER_CACHE_NAME), + new Integer(orgId), + new IgniteRunnable() { + @Override public void run() { + // No-op. + } + }); - fail("Exception is expected"); - } - catch (Exception e) { - assertTrue(e.getMessage() - .startsWith("Failed partition reservation. Partition is not primary on the node.")); - } + return null; + } + }, IgniteException.class, "Failed partition reservation. Partition is not primary on the node."); } /** diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java index 0fd16f0cd94e7..0d6d6a28279fe 100644 --- a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java +++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java @@ -21,7 +21,7 @@ import java.util.concurrent.Callable; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.Ignition; import org.apache.ignite.cache.store.jdbc.dialect.H2Dialect; import org.apache.ignite.cache.store.jdbc.dialect.JdbcDialect; @@ -69,15 +69,14 @@ public void testSerializable() throws Exception { * @throws Exception If failed. */ public void testIncorrectBeanConfiguration() throws Exception { - GridTestUtils.assertThrows(log, new Callable() { + GridTestUtils.assertThrowsAnyCause(log, new Callable() { @Override public Object call() throws Exception { - try(Ignite ignite = Ignition.start("modules/spring/src/test/config/pojo-incorrect-store-cache.xml")) { - ignite.cache(CACHE_NAME).getConfiguration(CacheConfiguration.class). - getCacheStoreFactory().create(); + try (Ignite ignored = Ignition.start("modules/spring/src/test/config/pojo-incorrect-store-cache.xml")) { + // No-op. } return null; } - }, IgniteException.class, "Spring bean with provided name doesn't exist"); + }, IgniteCheckedException.class, "Spring bean with provided name doesn't exist"); } /** diff --git a/modules/spring/src/test/java/org/apache/ignite/internal/processors/resource/GridServiceInjectionSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/internal/processors/resource/GridServiceInjectionSelfTest.java index b59bf2457f4de..52581b6d699ac 100644 --- a/modules/spring/src/test/java/org/apache/ignite/internal/processors/resource/GridServiceInjectionSelfTest.java +++ b/modules/spring/src/test/java/org/apache/ignite/internal/processors/resource/GridServiceInjectionSelfTest.java @@ -18,11 +18,13 @@ package org.apache.ignite.internal.processors.resource; import java.io.Serializable; -import org.apache.ignite.IgniteException; +import java.util.concurrent.Callable; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.lang.IgniteCallable; import org.apache.ignite.resources.ServiceResource; import org.apache.ignite.services.Service; import org.apache.ignite.services.ServiceContext; +import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; /** @@ -124,23 +126,22 @@ public void testClosureFieldLocalProxy() throws Exception { * @throws Exception If failed. */ public void testClosureFieldWithIncorrectType() throws Exception { - try { - grid(0).compute().call(new IgniteCallable() { - @ServiceResource(serviceName = SERVICE_NAME1) - private String svcName; + GridTestUtils.assertThrowsAnyCause(log, new Callable() { + @Override public Void call() { + grid(0).compute().call(new IgniteCallable() { + @ServiceResource(serviceName = SERVICE_NAME1) + private String svcName; - @Override public Object call() throws Exception { - fail(); + @Override public Object call() throws Exception { + fail(); - return null; - } - }); + return null; + } + }); - fail(); - } - catch (IgniteException e) { - assertTrue(e.getMessage().startsWith("Resource field is not assignable from the resource")); - } + return null; + } + }, IgniteCheckedException.class, "Resource field is not assignable from the resource"); } /** @@ -221,23 +222,22 @@ private void service(DummyService svc) { * @throws Exception If failed. */ public void testClosureMethodWithIncorrectType() throws Exception { - try { - grid(0).compute().call(new IgniteCallable() { - @ServiceResource(serviceName = SERVICE_NAME1) - private void service(String svcs) { - fail(); - } - - @Override public Object call() throws Exception { - return null; - } - }); - - fail(); - } - catch (IgniteException e) { - assertTrue(e.getMessage().startsWith("Setter does not have single parameter of required type")); - } + GridTestUtils.assertThrowsAnyCause(log, new Callable() { + @Override public Void call() { + grid(0).compute().call(new IgniteCallable() { + @ServiceResource(serviceName = SERVICE_NAME1) + private void service(String svcs) { + fail(); + } + + @Override public Object call() throws Exception { + return null; + } + }); + + return null; + } + }, IgniteCheckedException.class, "Setter does not have single parameter of required type"); } /** diff --git a/modules/spring/src/test/java/org/apache/ignite/internal/processors/resource/GridSpringResourceInjectionSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/internal/processors/resource/GridSpringResourceInjectionSelfTest.java index 827dd72efc2dd..08fe69da55852 100644 --- a/modules/spring/src/test/java/org/apache/ignite/internal/processors/resource/GridSpringResourceInjectionSelfTest.java +++ b/modules/spring/src/test/java/org/apache/ignite/internal/processors/resource/GridSpringResourceInjectionSelfTest.java @@ -27,6 +27,8 @@ import org.apache.ignite.resources.SpringResource; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.springframework.beans.factory.NoSuchBeanDefinitionException; +import org.springframework.beans.factory.NoUniqueBeanDefinitionException; import org.springframework.context.support.ClassPathXmlApplicationContext; /** @@ -93,7 +95,7 @@ public void testClosureFieldByResourceClassWithMultipleBeans() throws Exception Ignite anotherGrid = IgniteSpring.start(anotherCfg, new ClassPathXmlApplicationContext( "/org/apache/ignite/internal/processors/resource/spring-resource-with-duplicate-beans.xml")); - Throwable err = assertError(new IgniteCallable() { + assertError(new IgniteCallable() { @SpringResource(resourceClass = DummyResourceBean.class) private transient DummyResourceBean dummyRsrcBean; @@ -102,11 +104,9 @@ public void testClosureFieldByResourceClassWithMultipleBeans() throws Exception return null; } - }, anotherGrid, null); - - assertTrue("Unexpected message: " + err.getMessage(), err.getMessage().startsWith("No qualifying bean of type " + + }, anotherGrid, NoUniqueBeanDefinitionException.class, "No qualifying bean of type " + "'org.apache.ignite.internal.processors.resource.GridSpringResourceInjectionSelfTest$DummyResourceBean'" + - " available: expected single matching bean but found 2:")); + " available: expected single matching bean but found 2:"); G.stop("anotherGrid", false); } @@ -124,7 +124,7 @@ public void testClosureFieldWithWrongResourceName() { return null; } - }, "No bean named 'nonExistentResource' available"); + }, grid, NoSuchBeanDefinitionException.class, "No bean named 'nonExistentResource' available"); } /** @@ -140,7 +140,7 @@ public void testClosureFieldWithWrongResourceClass() { return null; } - }, "No qualifying bean of type 'org.apache.ignite.internal.processors.resource." + + }, grid, NoSuchBeanDefinitionException.class, "No qualifying bean of type 'org.apache.ignite.internal.processors.resource." + "GridSpringResourceInjectionSelfTest$AnotherDummyResourceBean' available"); } @@ -157,7 +157,7 @@ public void testClosureFieldByResourceClassAndName() { return null; } - }, "Either bean name or its class must be specified in @SpringResource, but not both"); + }, grid, IgniteException.class, "Either bean name or its class must be specified in @SpringResource, but not both"); } /** @@ -173,7 +173,7 @@ public void testClosureFieldWithNoParams() { return null; } - }, "Either bean name or its class must be specified in @SpringResource, but not both"); + }, grid, IgniteException.class, "Either bean name or its class must be specified in @SpringResource, but not both"); } /** @@ -232,7 +232,7 @@ public void testClosureMethodWithResourceClassWithMultipleBeans() throws Excepti "/org/apache/ignite/internal/processors/resource/spring-resource-with-duplicate-beans.xml")); try { - Throwable err = assertError(new IgniteCallable() { + assertError(new IgniteCallable() { private DummyResourceBean dummyRsrcBean; @SpringResource(resourceClass = DummyResourceBean.class) @@ -247,11 +247,9 @@ private void setDummyResourceBean(DummyResourceBean dummyRsrcBean) { return null; } - }, anotherGrid, null); - - assertTrue("Unexpected message: " + err.getMessage(), err.getMessage().startsWith("No qualifying bean of type " + + }, anotherGrid, NoUniqueBeanDefinitionException.class, "No qualifying bean of type " + "'org.apache.ignite.internal.processors.resource.GridSpringResourceInjectionSelfTest$DummyResourceBean'" + - " available: expected single matching bean but found 2:")); + " available: expected single matching bean but found 2:"); } finally { G.stop("anotherGrid", false); @@ -275,7 +273,7 @@ private void setDummyResourceBean(DummyResourceBean dummyRsrcBean) { return null; } - }, "No bean named 'nonExistentResource' available"); + }, grid, NoSuchBeanDefinitionException.class, "No bean named 'nonExistentResource' available"); } /** @@ -295,7 +293,7 @@ private void setDummyResourceBean(AnotherDummyResourceBean dummyRsrcBean) { return null; } - }, "No qualifying bean of type 'org.apache.ignite.internal.processors.resource" + + }, grid, NoSuchBeanDefinitionException.class,"No qualifying bean of type 'org.apache.ignite.internal.processors.resource" + ".GridSpringResourceInjectionSelfTest$AnotherDummyResourceBean' available"); } @@ -312,7 +310,7 @@ public void testClosureMethodByResourceClassAndName() { return null; } - }, "Either bean name or its class must be specified in @SpringResource, but not both"); + }, grid, IgniteException.class, "Either bean name or its class must be specified in @SpringResource, but not both"); } /** @@ -328,32 +326,24 @@ public void testClosureMethodWithNoParams() { return null; } - }, "Either bean name or its class must be specified in @SpringResource, but not both"); + }, grid, IgniteException.class, "Either bean name or its class must be specified in @SpringResource, but not both"); } /** - * @param job {@link IgniteCallable} to be run - * @param grid Node. - * @param expEMsg Message that {@link IgniteException} thrown from job should bear - * @return Thrown error. + * @param job {@link IgniteCallable} to be run. + * @param grid Node to run the job on. + * @param expE Expected exception type. + * @param expEMsg Expected exception message. */ @SuppressWarnings("ThrowableResultOfMethodCallIgnored") - private Throwable assertError(final IgniteCallable job, final Ignite grid, String expEMsg) { - return GridTestUtils.assertThrows(log, new Callable() { + private void assertError(final IgniteCallable job, final Ignite grid, Class expE, + String expEMsg) { + GridTestUtils.assertThrowsAnyCause(log, new Callable() { @Override public Object call() throws Exception { grid.compute(grid.cluster().forLocal()).call(job); return null; } - }, IgniteException.class, expEMsg); - } - - /** - * @param job {@link IgniteCallable} to be run - * @param expEMsg Message that {@link IgniteException} thrown from job should bear - */ - @SuppressWarnings("ThrowableResultOfMethodCallIgnored") - private void assertError(final IgniteCallable job, String expEMsg) { - assertError(job, grid, expEMsg); + }, expE, expEMsg); } /** From 82a4c024fe06ef8c8deeaf762f0cc20a8e481252 Mon Sep 17 00:00:00 2001 From: Roman Guseinov Date: Mon, 9 Apr 2018 14:45:44 +0300 Subject: [PATCH 0011/1463] IGNITE-7944: Disconnected client node tries to send JOB_CANCEL message. Applied fix: - Skip sending message if client disconnected; - Throw IgniteCheckedException if a client node is disconnected and communication client is null. This closes #3737. --- .../processors/task/GridTaskProcessor.java | 2 +- .../tcp/TcpCommunicationSpi.java | 26 +- ...cpCommunicationSpiSkipMessageSendTest.java | 414 ++++++++++++++++++ .../IgniteSpiCommunicationSelfTestSuite.java | 3 + 4 files changed, 442 insertions(+), 3 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiSkipMessageSendTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java index d27e11680e538..2f0aa7b858940 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 @@ -171,7 +171,7 @@ public GridTaskProcessor(GridKernalContext ctx) { IgniteClientDisconnectedCheckedException err = disconnectedError(reconnectFut); for (GridTaskWorker worker : tasks.values()) - worker.finishTask(null, err); + worker.finishTask(null, err, false); } /** 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 a3fccbcad5da2..4a0710e790f48 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 @@ -65,6 +65,7 @@ import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; @@ -2656,6 +2657,11 @@ private void sendMessage0(ClusterNode node, Message msg, IgniteInClosure() { + @Override public Integer call() throws Exception { + COMPUTE_JOB_STARTED.countDown(); + + // Simulate long-running job. + new CountDownLatch(1).await(); + + return null; + } + }); + } + catch (Exception e) { + e.printStackTrace(); + } + } + }).start(); + } + + /** + * Create Communication Spi instance. + * + * @param client Is a client node. + * @return Communication Spi. + */ + private TcpCommunicationSpi getCommunicationSpi(boolean client) { + TcpCommunicationSpi spi = new CustomCommunicationSpi(client); + + spi.setName("CustomCommunicationSpi"); + + return spi; + } + + /** + * Create Discovery Spi instance. + * + * @return Discovery Spi. + */ + private TcpDiscoverySpi getDiscoverySpi() { + TcpDiscoverySpi spi = new CustomDiscoverySpi(); + + spi.setName("CustomDiscoverySpi"); + + spi.setIpFinder(LOCAL_IP_FINDER); + + return spi; + } + + /** + * Create Ignite configuration. + * + * @param clientMode Client mode. + * @return Ignite configuration. + */ + private IgniteConfiguration getConfig(boolean clientMode) { + IgniteConfiguration cfg = new IgniteConfiguration(); + + cfg.setIgniteInstanceName(clientMode ? "client-node" : "server-node"); + + cfg.setClientMode(clientMode); + + cfg.setCommunicationSpi(getCommunicationSpi(clientMode)); + + if (!clientMode) { + cfg.setDiscoverySpi(getDiscoverySpi()); + + FifoQueueCollisionSpi collisionSpi = new FifoQueueCollisionSpi(); + + collisionSpi.setParallelJobsNumber(1); + + cfg.setCollisionSpi(collisionSpi); + } + else { + cfg.setFailureDetectionTimeout(FAILURE_DETECTION_TIMEOUT); + + cfg.setDiscoverySpi(getDiscoverySpi().setJoinTimeout(JOIN_TIMEOUT)); + } + + return cfg; + } + + /** + * Start client node. + * + * @param clientDisconnected Client is disconnected. + * @param clientSegmented Client is segmented. + * @return Ignite instance. + */ + private Ignite startClient(final CountDownLatch clientDisconnected, final CountDownLatch clientSegmented) { + Ignite ignite = Ignition.start(getConfig(true)); + + IgnitePredicate locLsnr = new IgnitePredicate() { + @Override public boolean apply(Event event) { + log.info("Client node received event: " + event.name()); + + if (event.type() == EventType.EVT_CLIENT_NODE_DISCONNECTED) + clientDisconnected.countDown(); + + if (event.type() == EventType.EVT_NODE_SEGMENTED) + clientSegmented.countDown(); + + return true; + } + }; + + ignite.events().localListen(locLsnr, + EventType.EVT_NODE_SEGMENTED, + EventType.EVT_CLIENT_NODE_DISCONNECTED); + + return ignite; + } + + /** + * Communication Spi that emulates connection troubles. + */ + class CustomCommunicationSpi extends TcpCommunicationSpi { + /** Network is disabled. */ + private volatile boolean networkDisabled = false; + + /** Additional logging is enabled. */ + private final boolean logEnabled; + + /** + * @param enableLogs Enable additional logging. + */ + CustomCommunicationSpi(boolean enableLogs) { + super(); + this.logEnabled = enableLogs; + } + + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, + IgniteInClosure ackC) throws IgniteSpiException { + String message = msg.toString(); + + if (logEnabled) + log.info("CustomCommunicationSpi.sendMessage: " + message); + + if (message.contains("TOPIC_JOB_CANCEL")) + closeTcpConnections(); + + super.sendMessage(node, msg, ackC); + } + + /** {@inheritDoc} */ + @Override protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) throws IgniteCheckedException { + if (logEnabled) + log.info(String.format("CustomCommunicationSpi.createTcpClient [networkDisabled=%s, node=%s]", networkDisabled, node)); + + if (networkDisabled) { + IgniteSpiOperationTimeoutHelper timeoutHelper = new IgniteSpiOperationTimeoutHelper(this, !node.isClient()); + + long timeout = timeoutHelper.nextTimeoutChunk(getConnectTimeout()); + + if (logEnabled) + log.info("CustomCommunicationSpi.createTcpClient [timeoutHelper.nextTimeoutChunk=" + timeout + "]"); + + sleep(timeout); + + return null; + } + else + return super.createTcpClient(node, connIdx); + } + + /** + * Simulate network disabling. + */ + void disableNetwork() { + networkDisabled = true; + } + + /** + * Close communication clients. It will lead that sendMessage method will be trying to create new ones. + */ + private void closeTcpConnections() { + final ConcurrentMap clients = U.field(this, "clients"); + + Set ids = clients.keySet(); + + if (ids.size() > 0) { + log.info("Close TCP clients: " + ids); + + for (UUID nodeId : ids) { + GridCommunicationClient[] clients0 = clients.remove(nodeId); + + if (clients0 != null) { + for (GridCommunicationClient client : clients0) { + if (client != null) + client.forceClose(); + } + } + } + + log.info("TCP clients are closed."); + } + } + } + + /** + * Discovery Spi that emulates connection troubles. + */ + class CustomDiscoverySpi extends TcpDiscoverySpi { + /** Network is disabled. */ + private volatile boolean networkDisabled = false; + + /** */ + private final CountDownLatch networkDisabledLatch = new CountDownLatch(1); + + /** */ + CustomDiscoverySpi() { + super(); + + setName("CustomDiscoverySpi"); + } + + /** {@inheritDoc} */ + @Override protected T readMessage(Socket sock, @Nullable InputStream in, + long timeout) throws IOException, IgniteCheckedException { + if (networkDisabled) { + sleep(timeout); + + return null; + } + else + return super.readMessage(sock, in, timeout); + } + + /** {@inheritDoc} */ + @Override protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg, + long timeout) throws IOException, IgniteCheckedException { + if (networkDisabled) { + sleep(timeout); + + networkDisabledLatch.countDown(); + + throw new IgniteCheckedException("CustomDiscoverySpi: network is disabled."); + } + else + super.writeToSocket(sock, msg, timeout); + } + + /** + * Simulate network disabling. + */ + void disableNetwork() { + networkDisabled = true; + } + + /** + * Wait until the network is disabled. + */ + boolean awaitNetworkDisabled(long timeout) throws InterruptedException { + return networkDisabledLatch.await(timeout, TimeUnit.MILLISECONDS); + } + } + + /** + * Sleeps for given number of milliseconds. + * + * @param timeout Time to sleep (2000 ms by default). + * @throws IgniteInterruptedCheckedException If current thread interrupted. + */ + static void sleep(long timeout) throws IgniteInterruptedCheckedException { + if (timeout > 0) + U.sleep(timeout); + else + U.sleep(2000); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java index 7a4de1b3feeec..1b962bc21e0c6 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java @@ -39,6 +39,7 @@ import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpiDropNodesTest; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpiFaultyClientTest; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpiHalfOpenedConnectionTest; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpiSkipMessageSendTest; import org.apache.ignite.spi.communication.tcp.TcpCommunicationStatisticsTest; /** @@ -78,6 +79,8 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(GridTcpCommunicationSpiConfigSelfTest.class)); + suite.addTest(new TestSuite(TcpCommunicationSpiSkipMessageSendTest.class)); + suite.addTest(new TestSuite(TcpCommunicationSpiFaultyClientTest.class)); suite.addTest(new TestSuite(TcpCommunicationSpiDropNodesTest.class)); suite.addTest(new TestSuite(TcpCommunicationSpiHalfOpenedConnectionTest.class)); From c1745de37891026e0a719f0c1d1afe768dfccbf3 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Tue, 10 Apr 2018 17:48:52 +0700 Subject: [PATCH 0012/1463] IGNITE-7927 Web Console: Fixed demo for non-collocated joins. (cherry picked from commit 647620b) --- modules/web-console/backend/routes/demo.js | 2 ++ .../demo/service/DemoCachesLoadService.java | 22 +++++++++++++++++-- 2 files changed, 22 insertions(+), 2 deletions(-) diff --git a/modules/web-console/backend/routes/demo.js b/modules/web-console/backend/routes/demo.js index a18fa7ac543c3..b081d0c667215 100644 --- a/modules/web-console/backend/routes/demo.js +++ b/modules/web-console/backend/routes/demo.js @@ -95,10 +95,12 @@ module.exports.factory = (errors, settings, mongo, spacesService) => { domain.space = cacheDoc.space; domain.caches.push(cacheDoc._id); + domain.clusters.push(cluster._id); return domain.save() .then((domainDoc) => { cacheDoc.domains.push(domainDoc._id); + cluster.models.push(domainDoc._id); return cacheDoc.save(); }); diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoCachesLoadService.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoCachesLoadService.java index 6691d1d3ec321..2aace06b1749f 100644 --- a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoCachesLoadService.java +++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoCachesLoadService.java @@ -269,6 +269,14 @@ private static CacheConfiguration cacheDepartment() { type.setFields(qryFlds); + // Indexes for DEPARTMENT. + + ArrayList indexes = new ArrayList<>(); + + indexes.add(new QueryIndex("countryId", QueryIndexType.SORTED, false, "DEP_COUNTRY")); + + type.setIndexes(indexes); + ccfg.setQueryEntities(qryEntities); return ccfg; @@ -312,6 +320,11 @@ private static CacheConfiguration cacheEmployee() { // Indexes for EMPLOYEE. + Collection indexes = new ArrayList<>(); + + indexes.add(new QueryIndex("departmentId", QueryIndexType.SORTED, false, "EMP_DEPARTMENT")); + indexes.add(new QueryIndex("managerId", QueryIndexType.SORTED, false, "EMP_MANAGER")); + QueryIndex idx = new QueryIndex(); idx.setName("EMP_NAMES"); @@ -323,8 +336,6 @@ private static CacheConfiguration cacheEmployee() { idx.setFields(indFlds); - Collection indexes = new ArrayList<>(); - indexes.add(idx); indexes.add(new QueryIndex("salary", QueryIndexType.SORTED, false, "EMP_SALARY")); @@ -392,6 +403,13 @@ private static CacheConfiguration cacheCar() { type.setFields(qryFlds); + // Indexes for CAR. + + ArrayList indexes = new ArrayList<>(); + + indexes.add(new QueryIndex("parkingId", QueryIndexType.SORTED, false, "CAR_PARKING")); + type.setIndexes(indexes); + ccfg.setQueryEntities(qryEntities); return ccfg; From b28287d1861fd841a18d0eef95eff309d21a55ef Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Tue, 10 Apr 2018 16:22:28 +0300 Subject: [PATCH 0013/1463] IGNITE-8025 Future must fail if assertion error has been thrown in the worker thread --- .../java/org/apache/ignite/testframework/GridTestUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 4e9a7c210fe07..e6c6657578c4d 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 @@ -814,7 +814,7 @@ public static IgniteInternalFuture runMultiThreadedAsync(Callable call, catch (IgniteFutureCancelledCheckedException e) { resFut.onCancelled(); } - catch (IgniteCheckedException e) { + catch (Throwable e) { resFut.onDone(e); } }); From a832f2b2e5788c45114c3cb5529d7cf53d08f9a6 Mon Sep 17 00:00:00 2001 From: Andrey Kuznetsov Date: Tue, 10 Apr 2018 17:30:12 +0300 Subject: [PATCH 0014/1463] ignite-7772 System workers critical failures handling Signed-off-by: Andrey Gura --- .../apache/ignite/internal/IgnitionEx.java | 15 +- .../GridClientConnectionManagerAdapter.java | 6 + .../impl/GridTcpRouterNioListenerAdapter.java | 6 + .../discovery/GridDiscoveryManager.java | 16 +- .../GridCachePartitionExchangeManager.java | 12 +- .../GridCacheSharedTtlCleanupManager.java | 41 ++++- .../GridCacheDatabaseSharedManager.java | 60 +++++-- .../wal/FileWriteAheadLogManager.java | 157 +++++++++++------- .../FsyncModeFileWriteAheadLogManager.java | 34 +++- .../timeout/GridTimeoutProcessor.java | 102 +++++++----- .../ignite/internal/util/StripedExecutor.java | 69 +++++--- .../internal/util/nio/GridNioServer.java | 43 ++++- .../util/nio/GridNioServerListener.java | 6 + .../nio/GridNioServerListenerAdapter.java | 6 + .../tcp/TcpCommunicationSpi.java | 41 ++++- .../ignite/spi/discovery/tcp/ServerImpl.java | 51 +++++- .../internal/util/StripedExecutorTest.java | 2 +- 17 files changed, 501 insertions(+), 166 deletions(-) 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 3abc7111c43a9..c0de08050fe60 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 @@ -137,6 +137,7 @@ import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_THREAD_KEEP_ALIVE_TIME; import static org.apache.ignite.configuration.MemoryConfiguration.DFLT_MEMORY_POLICY_MAX_SIZE; import static org.apache.ignite.configuration.MemoryConfiguration.DFLT_MEM_PLC_DEFAULT_NAME; +import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION; import static org.apache.ignite.internal.IgniteComponentType.SPRING; import static org.apache.ignite.plugin.segmentation.SegmentationPolicy.RESTART_JVM; @@ -1806,7 +1807,13 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { cfg.getStripedPoolSize(), cfg.getIgniteInstanceName(), "sys", - log); + log, + new Thread.UncaughtExceptionHandler() { + @Override public void uncaughtException(Thread thread, Throwable t) { + if (grid != null) + grid.context().failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, t)); + } + }); // Note that since we use 'LinkedBlockingQueue', number of // maximum threads has no effect. @@ -1846,6 +1853,12 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { cfg.getIgniteInstanceName(), "data-streamer", log, + new Thread.UncaughtExceptionHandler() { + @Override public void uncaughtException(Thread thread, Throwable t) { + if (grid != null) + grid.context().failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, t)); + } + }, true); // Note that we do not pre-start threads here as igfs pool may not be needed. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java index 829b188a823e3..fe0453f0076e4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java @@ -38,6 +38,7 @@ import javax.net.ssl.SSLContext; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.failure.FailureType; import org.apache.ignite.internal.client.GridClientClosedException; import org.apache.ignite.internal.client.GridClientConfiguration; import org.apache.ignite.internal.client.GridClientException; @@ -656,6 +657,11 @@ private NioListener(Logger log) { } } + /** {@inheritDoc} */ + @Override public void onFailure(FailureType failureType, Throwable failure) { + // No-op. + } + /** * Handles client handshake response. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java index 22f5152a2ea0c..75aa6f29ccc3f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java @@ -24,6 +24,7 @@ import java.util.Map; import java.util.UUID; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.failure.FailureType; import org.apache.ignite.internal.client.GridClientException; import org.apache.ignite.internal.client.GridClientFuture; import org.apache.ignite.internal.client.GridClientFutureListener; @@ -190,6 +191,11 @@ else if (msg instanceof GridClientPingPacket) throw new IllegalArgumentException("Unsupported input message: " + msg); } + /** {@inheritDoc} */ + @Override public void onFailure(FailureType failureType, Throwable failure) { + // No-op. + } + /** {@inheritDoc} */ @Override public void onSessionWriteTimeout(GridNioSession ses) { U.warn(log, "Closing NIO session because of write timeout."); 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 4c5690e919d34..b0d32564aebf0 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 @@ -147,6 +147,8 @@ 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.failure.FailureType.CRITICAL_ERROR; +import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_DATA_REGIONS_OFFHEAP_SIZE; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_DEPLOYMENT_MODE; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_LATE_AFFINITY_ASSIGNMENT; @@ -2669,13 +2671,21 @@ void addEvent( body0(); } catch (InterruptedException e) { + if (!isCancelled) + ctx.failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, e)); + throw e; } catch (Throwable t) { - U.error(log, "Unexpected exception in discovery worker thread (ignored).", t); + U.error(log, "Exception in discovery worker thread.", t); + + if (t instanceof Error) { + FailureType type = t instanceof OutOfMemoryError ? CRITICAL_ERROR : SYSTEM_WORKER_TERMINATION; - if (t instanceof Error) - throw (Error)t; + ctx.failure().process(new FailureContext(type, t)); + + throw t; + } } } } 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 77ffce3bdaf06..e40493fbc589a 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 @@ -123,6 +123,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.failure.FailureType.CRITICAL_ERROR; import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION; import static org.apache.ignite.internal.GridTopic.TOPIC_CACHE; import static org.apache.ignite.internal.events.DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT; @@ -2274,11 +2275,20 @@ void dumpExchangeDebugInfo() { try { body0(); } + catch (InterruptedException | IgniteInterruptedCheckedException e) { + if (!stop) + err = e; + } catch (Throwable e) { err = e; } finally { - if (!stop) + if (err == null && !stop) + err = new IllegalStateException("Thread " + name() + " is terminated unexpectedly"); + + if (err instanceof OutOfMemoryError) + cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, err)); + else if (err != null) cctx.kernalContext().failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, err)); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedTtlCleanupManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedTtlCleanupManager.java index 8f3d738c8eccb..613e93bc88c23 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedTtlCleanupManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedTtlCleanupManager.java @@ -20,11 +20,15 @@ import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.failure.FailureContext; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.util.worker.GridWorker; import org.apache.ignite.thread.IgniteThread; +import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR; +import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION; + /** * Periodically removes expired entities from caches with {@link CacheConfiguration#isEagerTtl()} flag set. */ @@ -122,19 +126,38 @@ private class CleanupWorker extends GridWorker { /** {@inheritDoc} */ @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException { - while (!isCancelled()) { - boolean expiredRemains = false; + Throwable err = null; + + try { + while (!isCancelled()) { + boolean expiredRemains = false; + + for (GridCacheTtlManager mgr : mgrs) { + if (mgr.expire(CLEANUP_WORKER_ENTRIES_PROCESS_LIMIT)) + expiredRemains = true; - for (GridCacheTtlManager mgr : mgrs) { - if (mgr.expire(CLEANUP_WORKER_ENTRIES_PROCESS_LIMIT)) - expiredRemains = true; + if (isCancelled()) + return; + } - if (isCancelled()) - return; + if (!expiredRemains) + U.sleep(CLEANUP_WORKER_SLEEP_INTERVAL); } + } + catch (Throwable t) { + if (!(t instanceof IgniteInterruptedCheckedException)) + err = t; - if (!expiredRemains) - U.sleep(CLEANUP_WORKER_SLEEP_INTERVAL); + throw t; + } + finally { + if (err == null && !isCancelled) + err = new IllegalStateException("Thread " + name() + " is terminated unexpectedly"); + + if (err instanceof OutOfMemoryError) + cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, err)); + else if (err != null) + cctx.kernalContext().failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, err)); } } } 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 71f3baa7917bc..5beaafc585c19 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 @@ -162,6 +162,8 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_SKIP_CRC; import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD; +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.processors.cache.persistence.metastorage.MetaStorage.METASTORAGE_CACHE_ID; /** @@ -2739,32 +2741,58 @@ protected Checkpointer(@Nullable String gridName, String name, IgniteLogger log) } /** {@inheritDoc} */ - @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException { - while (!isCancelled()) { - waitCheckpointEvent(); + @Override protected void body() { + Throwable err = null; - GridFutureAdapter enableChangeApplied = GridCacheDatabaseSharedManager.this.enableChangeApplied; + try { + while (!isCancelled()) { + waitCheckpointEvent(); - if (enableChangeApplied != null) { - enableChangeApplied.onDone(); + GridFutureAdapter enableChangeApplied = GridCacheDatabaseSharedManager.this.enableChangeApplied; - GridCacheDatabaseSharedManager.this.enableChangeApplied = null; - } + if (enableChangeApplied != null) { + enableChangeApplied.onDone(); - if (checkpointsEnabled) - doCheckpoint(); - else { - synchronized (this) { - scheduledCp.nextCpTs = U.currentTimeMillis() + checkpointFreq; + GridCacheDatabaseSharedManager.this.enableChangeApplied = null; + } + + if (checkpointsEnabled) + doCheckpoint(); + else { + synchronized (this) { + scheduledCp.nextCpTs = U.currentTimeMillis() + checkpointFreq; + } } } } + catch (Throwable t) { + err = t; + + scheduledCp.cpFinishFut.onDone(t); + + throw t; + } + finally { + if (err == null && !(stopping && isCancelled)) + err = new IllegalStateException("Thread " + name() + " is terminated unexpectedly"); + + if (err instanceof OutOfMemoryError) + cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, err)); + else if (err != null) + cctx.kernalContext().failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, err)); + } // Final run after the cancellation. - if (checkpointsEnabled && !shutdownNow) - doCheckpoint(); + if (checkpointsEnabled && !shutdownNow) { + try { + doCheckpoint(); - scheduledCp.cpFinishFut.onDone(new NodeStoppingException("Node is stopping.")); + scheduledCp.cpFinishFut.onDone(new NodeStoppingException("Node is stopping.")); + } + catch (Throwable e) { + scheduledCp.cpFinishFut.onDone(e); + } + } } /** 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 2fff481380c08..a40811bfb6de4 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 @@ -117,6 +117,8 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_MMAP; import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_SERIALIZER_VERSION; import static org.apache.ignite.configuration.WALMode.LOG_ONLY; +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.SWITCH_SEGMENT_RECORD; import static org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode.DIRECT; import static org.apache.ignite.internal.util.IgniteUtils.findField; @@ -682,7 +684,7 @@ private void checkWalRolloverRequiredDuringInactivityPeriod() { catch (IgniteCheckedException e) { U.error(log, "Unable to perform segment rollover: " + e.getMessage(), e); - cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e)); + cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, e)); } } @@ -1234,7 +1236,7 @@ private FileWriteHandle initNextWriteHandle(FileWriteHandle cur) throws StorageE catch (IOException e) { StorageException se = new StorageException("Unable to initialize WAL segment", e); - cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, se)); + cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, se)); throw se; } @@ -1499,6 +1501,8 @@ private synchronized boolean locked(long absIdx) { } } + Throwable err = null; + try { synchronized (this) { while (curAbsWalIdx == -1 && !stopped) @@ -1560,6 +1564,18 @@ private synchronized boolean locked(long absIdx) { catch (InterruptedException ignore) { Thread.currentThread().interrupt(); } + catch (Throwable t) { + err = t; + } + finally { + if (err == null && !stopped) + err = new IllegalStateException("Thread " + getName() + " is terminated unexpectedly"); + + if (err instanceof OutOfMemoryError) + cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, err)); + else if (err != null) + cctx.kernalContext().failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, err)); + } } /** @@ -1884,8 +1900,6 @@ private void deleteObsoleteRawSegments() { } catch (IgniteCheckedException | IOException e) { U.error(log, "Unexpected error during WAL compression", e); - - cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e)); } catch (InterruptedException ignore) { Thread.currentThread().interrupt(); @@ -2005,6 +2019,8 @@ private class FileDecompressor extends Thread { /** {@inheritDoc} */ @Override public void run() { + Throwable err = null; + while (!Thread.currentThread().isInterrupted() && !stopped) { try { long segmentToDecompress = segmentsQueue.take(); @@ -2034,10 +2050,17 @@ private class FileDecompressor extends Thread { catch (InterruptedException ignore) { Thread.currentThread().interrupt(); } - catch (IOException e) { - U.error(log, "Unexpected error during WAL decompression", e); + catch (Throwable t) { + err = t; + } + finally { + if (err == null && !stopped) + err = new IllegalStateException("Thread " + getName() + " is terminated unexpectedly"); - cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e)); + if (err instanceof OutOfMemoryError) + cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, err)); + else if (err != null) + cctx.kernalContext().failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, err)); } } } @@ -3146,78 +3169,94 @@ private class WALWriter extends Thread { /** {@inheritDoc} */ @Override public void run() { - while (!shutdown && !Thread.currentThread().isInterrupted()) { - while (waiters.isEmpty()) { - if (!shutdown) - LockSupport.park(); - else { - unparkWaiters(Long.MAX_VALUE); - - return; - } - } + Throwable err = null; - Long pos = null; + try { + while (!shutdown && !Thread.currentThread().isInterrupted()) { + while (waiters.isEmpty()) { + if (!shutdown) + LockSupport.park(); + else { + unparkWaiters(Long.MAX_VALUE); - for (Long val : waiters.values()) { - if (val > Long.MIN_VALUE) - pos = val; - } + return; + } + } - if (pos == null) - continue; - else if (pos < UNCONDITIONAL_FLUSH) { - try { - assert pos == FILE_CLOSE || pos == FILE_FORCE : pos; + Long pos = null; - if (pos == FILE_CLOSE) - currHnd.fileIO.close(); - else if (pos == FILE_FORCE) - currHnd.fileIO.force(); + for (Long val : waiters.values()) { + if (val > Long.MIN_VALUE) + pos = val; } - catch (IOException e) { - log.error("Exception in WAL writer thread: ", e); - err = e; + if (pos == null) + continue; + else if (pos < UNCONDITIONAL_FLUSH) { + try { + assert pos == FILE_CLOSE || pos == FILE_FORCE : pos; - unparkWaiters(Long.MAX_VALUE); + if (pos == FILE_CLOSE) + currHnd.fileIO.close(); + else if (pos == FILE_FORCE) + currHnd.fileIO.force(); + } + catch (IOException e) { + log.error("Exception in WAL writer thread: ", e); - return; - } + err = e; - unparkWaiters(pos); - } + unparkWaiters(Long.MAX_VALUE); - List segs = currentHandle().buf.poll(pos); + return; + } - if (segs == null) { - unparkWaiters(pos); + unparkWaiters(pos); + } - continue; - } + List segs = currentHandle().buf.poll(pos); - for (int i = 0; i < segs.size(); i++) { - SegmentedRingByteBuffer.ReadSegment seg = segs.get(i); + if (segs == null) { + unparkWaiters(pos); - try { - writeBuffer(seg.position(), seg.buffer()); + continue; } - catch (Throwable e) { - log.error("Exception in WAL writer thread: ", e); - err = e; - } - finally { - seg.release(); + for (int i = 0; i < segs.size(); i++) { + SegmentedRingByteBuffer.ReadSegment seg = segs.get(i); + + try { + writeBuffer(seg.position(), seg.buffer()); + } + catch (Throwable e) { + log.error("Exception in WAL writer thread: ", e); + + err = e; + } + finally { + seg.release(); - long p = pos <= UNCONDITIONAL_FLUSH || err != null ? Long.MAX_VALUE : currentHandle().written; + long p = pos <= UNCONDITIONAL_FLUSH || err != null ? Long.MAX_VALUE : currentHandle().written; - unparkWaiters(p); + unparkWaiters(p); + } } } + + unparkWaiters(Long.MAX_VALUE); } + catch (Throwable t) { + err = t; + } + finally { + if (err == null && !shutdown) + err = new IllegalStateException("Thread " + getName() + " is terminated unexpectedly"); - unparkWaiters(Long.MAX_VALUE); + if (err instanceof OutOfMemoryError) + cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, err)); + else if (err != null) + cctx.kernalContext().failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, err)); + } } /** @@ -3283,7 +3322,7 @@ void flushBuffer(long expPos) throws StorageException, IgniteCheckedException { Throwable err = walWriter.err; if (err != null) - cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, err)); + cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, err)); if (expPos == UNCONDITIONAL_FLUSH) expPos = (currentHandle().buf.tail()); @@ -3372,7 +3411,7 @@ private void writeBuffer(long pos, ByteBuffer buf) throws StorageException, Igni catch (IOException e) { StorageException se = new StorageException("Unable to write", e); - cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, se)); + cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, se)); throw se; } 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 59196bbaf2c8e..c7d2c115ef7e4 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 @@ -110,6 +110,7 @@ import static java.nio.file.StandardOpenOption.WRITE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_SERIALIZER_VERSION; import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR; +import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION; /** * File WAL manager. @@ -1338,6 +1339,8 @@ private synchronized void release(long absIdx) { } } + Throwable err = null; + try { synchronized (this) { while (curAbsWalIdx == -1 && !stopped) @@ -1399,6 +1402,18 @@ private synchronized void release(long absIdx) { catch (InterruptedException ignore) { Thread.currentThread().interrupt(); } + catch (Throwable t) { + err = t; + } + finally { + if (err == null && !stopped) + err = new IllegalStateException("Thread " + getName() + " is terminated unexpectedly"); + + if (err instanceof OutOfMemoryError) + cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, err)); + else if (err != null) + cctx.kernalContext().failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, err)); + } } /** @@ -1721,8 +1736,6 @@ private void deleteObsoleteRawSegments() { } catch (IgniteCheckedException | IOException e) { U.error(log, "Unexpected error during WAL compression", e); - - cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, e)); } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -1814,6 +1827,8 @@ private class FileDecompressor extends Thread { /** {@inheritDoc} */ @Override public void run() { + Throwable err = null; + while (!Thread.currentThread().isInterrupted() && !stopped) { try { long segmentToDecompress = segmentsQueue.take(); @@ -1840,13 +1855,20 @@ private class FileDecompressor extends Thread { decompressionFutures.remove(segmentToDecompress).onDone(); } } - catch (InterruptedException e){ + catch (InterruptedException ignore) { Thread.currentThread().interrupt(); } - catch (IOException e) { - U.error(log, "Unexpected error during WAL decompression", e); + catch (Throwable t) { + err = t; + } + finally { + if (err == null && !stopped) + err = new IllegalStateException("Thread " + getName() + " is terminated unexpectedly"); - cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, e)); + if (err instanceof OutOfMemoryError) + cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, err)); + else if (err != null) + cctx.kernalContext().failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, err)); } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java index ff6beb417991e..a09d6faad85e9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java @@ -21,6 +21,7 @@ import java.util.Comparator; import java.util.Iterator; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.failure.FailureContext; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.GridProcessorAdapter; import org.apache.ignite.internal.util.GridConcurrentSkipListSet; @@ -32,6 +33,9 @@ import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.thread.IgniteThread; +import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR; +import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION; + /** * Detects timeout events and processes them. */ @@ -146,61 +150,81 @@ private class TimeoutWorker extends GridWorker { /** {@inheritDoc} */ @Override protected void body() throws InterruptedException { - while (!isCancelled()) { - long now = U.currentTimeMillis(); + Throwable err = null; - for (Iterator iter = timeoutObjs.iterator(); iter.hasNext();) { - GridTimeoutObject timeoutObj = iter.next(); + try { + while (!isCancelled()) { + long now = U.currentTimeMillis(); - if (timeoutObj.endTime() <= now) { - try { - boolean rmvd = timeoutObjs.remove(timeoutObj); + for (Iterator iter = timeoutObjs.iterator(); iter.hasNext(); ) { + GridTimeoutObject timeoutObj = iter.next(); - if (log.isDebugEnabled()) - log.debug("Timeout has occurred [obj=" + timeoutObj + ", process=" + rmvd + ']'); + if (timeoutObj.endTime() <= now) { + try { + boolean rmvd = timeoutObjs.remove(timeoutObj); - if (rmvd) - timeoutObj.onTimeout(); - } - catch (Throwable e) { - if (isCancelled() && !(e instanceof Error)){ if (log.isDebugEnabled()) - log.debug("Error when executing timeout callback: " + timeoutObj); + log.debug("Timeout has occurred [obj=" + timeoutObj + ", process=" + rmvd + ']'); - return; + if (rmvd) + timeoutObj.onTimeout(); } + catch (Throwable e) { + if (isCancelled() && !(e instanceof Error)) { + if (log.isDebugEnabled()) + log.debug("Error when executing timeout callback: " + timeoutObj); - U.error(log, "Error when executing timeout callback: " + timeoutObj, e); + return; + } - if (e instanceof Error) - throw e; + U.error(log, "Error when executing timeout callback: " + timeoutObj, e); + + if (e instanceof Error) + throw e; + } } + else + break; } - else - break; - } - - synchronized (mux) { - while (!isCancelled()) { - // Access of the first element must be inside of - // synchronization block, so we don't miss out - // on thread notification events sent from - // 'addTimeoutObject(..)' method. - GridTimeoutObject first = timeoutObjs.firstx(); - - if (first != null) { - long waitTime = first.endTime() - U.currentTimeMillis(); - if (waitTime > 0) - mux.wait(waitTime); + synchronized (mux) { + while (!isCancelled()) { + // Access of the first element must be inside of + // synchronization block, so we don't miss out + // on thread notification events sent from + // 'addTimeoutObject(..)' method. + GridTimeoutObject first = timeoutObjs.firstx(); + + if (first != null) { + long waitTime = first.endTime() - U.currentTimeMillis(); + + if (waitTime > 0) + mux.wait(waitTime); + else + break; + } else - break; + mux.wait(5000); } - else - mux.wait(5000); } } } + catch (Throwable t) { + if (!(t instanceof InterruptedException)) + err = t; + + throw t; + } + finally { + if (err == null && !isCancelled) + err = new IllegalStateException("Thread " + name() + " is terminated unexpectedly."); + + if (err instanceof OutOfMemoryError) + ctx.failure().process(new FailureContext(CRITICAL_ERROR, err)); + else if (err != null) + ctx.failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, err)); + } + } } @@ -284,4 +308,4 @@ public class CancelableTask implements GridTimeoutObject, Closeable { return S.toString(CancelableTask.class, this); } } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java index 630d34c9414c8..c6383ee41319e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java @@ -64,9 +64,11 @@ public class StripedExecutor implements ExecutorService { * @param igniteInstanceName Node name. * @param poolName Pool name. * @param log Logger. + * @param errHnd Exception handler. */ - public StripedExecutor(int cnt, String igniteInstanceName, String poolName, final IgniteLogger log) { - this(cnt, igniteInstanceName, poolName, log, false); + public StripedExecutor(int cnt, String igniteInstanceName, String poolName, final IgniteLogger log, + Thread.UncaughtExceptionHandler errHnd) { + this(cnt, igniteInstanceName, poolName, log, errHnd, false); } /** @@ -74,9 +76,11 @@ public StripedExecutor(int cnt, String igniteInstanceName, String poolName, fina * @param igniteInstanceName Node name. * @param poolName Pool name. * @param log Logger. + * @param errHnd Exception handler. * @param stealTasks {@code True} to steal tasks. */ - public StripedExecutor(int cnt, String igniteInstanceName, String poolName, final IgniteLogger log, boolean stealTasks) { + public StripedExecutor(int cnt, String igniteInstanceName, String poolName, final IgniteLogger log, + Thread.UncaughtExceptionHandler errHnd, boolean stealTasks) { A.ensure(cnt > 0, "cnt > 0"); boolean success = false; @@ -91,15 +95,9 @@ public StripedExecutor(int cnt, String igniteInstanceName, String poolName, fina try { for (int i = 0; i < cnt; i++) { - stripes[i] = stealTasks ? new StripeConcurrentQueue( - igniteInstanceName, - poolName, - i, - log, stripes) : new StripeConcurrentQueue( - igniteInstanceName, - poolName, - i, - log); + stripes[i] = stealTasks + ? new StripeConcurrentQueue(igniteInstanceName, poolName, i, log, stripes, errHnd) + : new StripeConcurrentQueue(igniteInstanceName, poolName, i, log, errHnd); } for (int i = 0; i < cnt; i++) @@ -434,22 +432,28 @@ private static abstract class Stripe implements Runnable { /** Thread executing the loop. */ protected Thread thread; + /** Exception handler. */ + private Thread.UncaughtExceptionHandler errHnd; + /** * @param igniteInstanceName Ignite instance name. * @param poolName Pool name. * @param idx Stripe index. * @param log Logger. + * @param errHnd Exception handler. */ public Stripe( String igniteInstanceName, String poolName, int idx, - IgniteLogger log + IgniteLogger log, + Thread.UncaughtExceptionHandler errHnd ) { this.igniteInstanceName = igniteInstanceName; this.poolName = poolName; this.idx = idx; this.log = log; + this.errHnd = errHnd; } /** @@ -463,6 +467,8 @@ void start() { idx, GridIoPolicy.UNDEFINED); + thread.setUncaughtExceptionHandler(errHnd); + thread.start(); } @@ -518,9 +524,19 @@ void awaitStop() { return; } catch (Throwable e) { + if (e instanceof OutOfMemoryError) { + // Re-throwing to exploit uncaught exception handler. + throw e; + } + U.error(log, "Failed to execute runnable.", e); } } + + if (!stopping) { + throw new IllegalStateException("Thread " + Thread.currentThread().getName() + + " is terminated unexpectedly"); + } } /** @@ -576,14 +592,16 @@ private static class StripeConcurrentQueue extends Stripe { * @param poolName Pool name. * @param idx Stripe index. * @param log Logger. + * @param errHnd Exception handler. */ StripeConcurrentQueue( String igniteInstanceName, String poolName, int idx, - IgniteLogger log + IgniteLogger log, + Thread.UncaughtExceptionHandler errHnd ) { - this(igniteInstanceName, poolName, idx, log, null); + this(igniteInstanceName, poolName, idx, log, null, errHnd); } /** @@ -591,19 +609,22 @@ private static class StripeConcurrentQueue extends Stripe { * @param poolName Pool name. * @param idx Stripe index. * @param log Logger. + * @param errHnd Exception handler. */ StripeConcurrentQueue( String igniteInstanceName, String poolName, int idx, IgniteLogger log, - Stripe[] others + Stripe[] others, + Thread.UncaughtExceptionHandler errHnd ) { super( igniteInstanceName, poolName, idx, - log); + log, + errHnd); this.others = others; @@ -702,17 +723,20 @@ private static class StripeConcurrentQueueNoPark extends Stripe { * @param poolName Pool name. * @param idx Stripe index. * @param log Logger. + * @param errHnd Exception handler. */ public StripeConcurrentQueueNoPark( String igniteInstanceName, String poolName, int idx, - IgniteLogger log + IgniteLogger log, + Thread.UncaughtExceptionHandler errHnd ) { super(igniteInstanceName, poolName, idx, - log); + log, + errHnd); } /** {@inheritDoc} */ @@ -758,17 +782,20 @@ private static class StripeConcurrentBlockingQueue extends Stripe { * @param poolName Pool name. * @param idx Stripe index. * @param log Logger. + * @param errHnd Exception handler. */ public StripeConcurrentBlockingQueue( String igniteInstanceName, String poolName, int idx, - IgniteLogger log + IgniteLogger log, + Thread.UncaughtExceptionHandler errHnd ) { super(igniteInstanceName, poolName, idx, - log); + log, + errHnd); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java index 0fcde0e136644..3597a050051f3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java @@ -77,6 +77,8 @@ import org.apache.ignite.thread.IgniteThread; import org.jetbrains.annotations.Nullable; +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.util.nio.GridNioSessionMetaKey.MSG_WRITER; import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.NIO_OPERATION; @@ -1749,6 +1751,8 @@ protected AbstractNioClientWorker(int idx, @Nullable String igniteInstanceName, /** {@inheritDoc} */ @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException { + Throwable err = null; + try { boolean reset = false; @@ -1774,9 +1778,24 @@ protected AbstractNioClientWorker(int idx, @Nullable String igniteInstanceName, catch (Throwable e) { U.error(log, "Caught unhandled exception in NIO worker thread (restart the node).", e); + err = e; + if (e instanceof Error) throw e; } + finally { + if (err instanceof OutOfMemoryError) + lsnr.onFailure(CRITICAL_ERROR, err); + else if (!closed) { + if (err == null) + lsnr.onFailure(SYSTEM_WORKER_TERMINATION, + new IllegalStateException("Thread " + name() + " is terminated unexpectedly")); + else if (err instanceof InterruptedException) + lsnr.onFailure(SYSTEM_WORKER_TERMINATION, err); + } + else if (err != null) + lsnr.onFailure(SYSTEM_WORKER_TERMINATION, err); + } } /** @@ -2790,6 +2809,8 @@ protected GridNioAcceptWorker( /** {@inheritDoc} */ @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException { + Throwable err = null; + try { boolean reset = false; @@ -2812,8 +2833,28 @@ protected GridNioAcceptWorker( } } } + catch (Throwable t) { + if (!(t instanceof IgniteInterruptedCheckedException)) + err = t; + + throw t; + } finally { - closeSelector(); // Safety. + try { + closeSelector(); // Safety. + } + catch (RuntimeException ignore) { + // No-op. + } + + if (err == null && !closed) + err = new IllegalStateException("Thread " + name() + " is terminated unexpectedly"); + + if (err instanceof OutOfMemoryError) + lsnr.onFailure(CRITICAL_ERROR, err); + else if (err != null) + lsnr.onFailure(SYSTEM_WORKER_TERMINATION, err); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServerListener.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServerListener.java index db28792347235..14c5a748a8a81 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServerListener.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServerListener.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.util.nio; +import org.apache.ignite.failure.FailureType; import org.jetbrains.annotations.Nullable; /** @@ -69,4 +70,9 @@ public interface GridNioServerListener { * @param ses Session that is idle. */ public void onSessionIdleTimeout(GridNioSession ses); + + /** + * Called when critical failure occurs in server implementation. + */ + public void onFailure(FailureType failureType, Throwable failure); } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServerListenerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServerListenerAdapter.java index 5d222c1915886..b6b20b2854da2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServerListenerAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServerListenerAdapter.java @@ -17,6 +17,8 @@ package org.apache.ignite.internal.util.nio; +import org.apache.ignite.failure.FailureType; + /** * Server listener adapter providing empty methods implementation for rarely used methods. */ @@ -35,4 +37,8 @@ public abstract class GridNioServerListenerAdapter implements GridNioServerLi @Override public void onMessageSent(GridNioSession ses, T msg) { // No-op. } + + @Override public void onFailure(FailureType failureType, Throwable failure) { + // No-op. + } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 4a0710e790f48..9e7b59235db80 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 @@ -62,7 +62,10 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.events.DiscoveryEvent; import org.apache.ignite.events.Event; +import org.apache.ignite.failure.FailureContext; +import org.apache.ignite.failure.FailureType; import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; +import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.IgniteKernal; @@ -151,6 +154,8 @@ 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.failure.FailureType.CRITICAL_ERROR; +import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION; import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.SSL_META; import static org.apache.ignite.spi.communication.tcp.internal.TcpCommunicationConnectionCheckFuture.SES_FUT_META; import static org.apache.ignite.spi.communication.tcp.messages.RecoveryLastReceivedMessage.ALREADY_CONNECTED; @@ -798,6 +803,11 @@ else if (connKey.dummy()) { } } + /** {@inheritDoc} */ + @Override public void onFailure(FailureType failureType, Throwable failure) { + ((IgniteEx)ignite).context().failure().process(new FailureContext(failureType, failure)); + } + /** * @param recovery Recovery descriptor. * @param ses Session. @@ -4190,13 +4200,32 @@ private CommunicationWorker(String igniteInstanceName) { if (log.isDebugEnabled()) log.debug("Tcp communication worker has been started."); - while (!isInterrupted()) { - DisconnectedSessionInfo disconnectData = q.poll(idleConnTimeout, TimeUnit.MILLISECONDS); + Throwable err = null; - if (disconnectData != null) - processDisconnect(disconnectData); - else - processIdle(); + try { + while (!isInterrupted()) { + DisconnectedSessionInfo disconnectData = q.poll(idleConnTimeout, TimeUnit.MILLISECONDS); + + if (disconnectData != null) + processDisconnect(disconnectData); + else + processIdle(); + } + } + catch (Throwable t) { + if (!(t instanceof InterruptedException)) + err = t; + + throw t; + } + finally { + if (err == null && !stopping) + err = new IllegalStateException("Thread " + getName() + " is terminated unexpectedly."); + + if (err instanceof OutOfMemoryError) + ((IgniteEx)ignite).context().failure().process(new FailureContext(CRITICAL_ERROR, err)); + else if (err != null) + ((IgniteEx)ignite).context().failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, err)); } } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 4aa13163e1bf7..7bf37e1b0ac43 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -50,6 +50,7 @@ import java.util.TreeMap; import java.util.UUID; import java.util.concurrent.BlockingDeque; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.LinkedBlockingDeque; @@ -66,6 +67,8 @@ import org.apache.ignite.cache.CacheMetrics; import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.failure.FailureContext; +import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.IgniteNodeAttributes; @@ -73,6 +76,7 @@ import org.apache.ignite.internal.events.DiscoveryCustomEvent; import org.apache.ignite.internal.managers.discovery.CustomMessageWrapper; import org.apache.ignite.internal.managers.discovery.DiscoveryServerOnlyCustomMessage; +import org.apache.ignite.internal.processors.failure.FailureProcessor; import org.apache.ignite.internal.processors.security.SecurityContext; import org.apache.ignite.internal.processors.security.SecurityUtils; import org.apache.ignite.internal.util.GridBoundedLinkedHashSet; @@ -137,7 +141,6 @@ import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusCheckMessage; import org.apache.ignite.thread.IgniteThreadPoolExecutor; import org.jetbrains.annotations.Nullable; -import java.util.concurrent.ConcurrentHashMap; import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2; import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISCOVERY_CLIENT_RECONNECT_HISTORY_SIZE; @@ -149,6 +152,8 @@ 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.failure.FailureType.CRITICAL_ERROR; +import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_LATE_AFFINITY_ASSIGNMENT; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_COMPACT_FOOTER; @@ -2609,12 +2614,20 @@ void addMessage(TcpDiscoveryAbstractMessage msg) { /** {@inheritDoc} */ @Override protected void body() throws InterruptedException { + Throwable err = null; + try { super.body(); } + catch (InterruptedException e) { + if (!spi.isNodeStopping0()) + err = e; + + throw e; + } catch (Throwable e) { if (!spi.isNodeStopping0() && spiStateCopy() != DISCONNECTING) { - final Ignite ignite = spi.ignite(); + final Ignite ignite = spi.ignite(); if (ignite != null) { U.error(log, "TcpDiscoverSpi's message worker thread failed abnormally. " + @@ -2637,9 +2650,22 @@ void addMessage(TcpDiscoveryAbstractMessage msg) { } } + err = e; + // Must be processed by IgniteSpiThread as well. throw e; } + finally { + if (err == null && !spi.isNodeStopping0()) + err = new IllegalStateException("Thread " + getName() + " is terminated unexpectedly."); + + FailureProcessor failure = ((IgniteEx)spi.ignite()).context().failure(); + + if (err instanceof OutOfMemoryError) + failure.process(new FailureContext(CRITICAL_ERROR, err)); + else if (err != null) + failure.process(new FailureContext(SYSTEM_WORKER_TERMINATION, err)); + } } /** @@ -5597,7 +5623,9 @@ private class TcpServer extends IgniteSpiThread { } /** {@inheritDoc} */ - @Override protected void body() throws InterruptedException { + @Override protected void body() { + Throwable err = null; + try { while (!isInterrupted()) { Socket sock = srvrSock.accept(); @@ -5630,13 +5658,30 @@ private class TcpServer extends IgniteSpiThread { onException("Failed to accept TCP connection.", e); if (!isInterrupted()) { + err = e; + if (U.isMacInvalidArgumentError(e)) U.error(log, "Failed to accept TCP connection\n\t" + U.MAC_INVALID_ARG_MSG, e); else U.error(log, "Failed to accept TCP connection.", e); } } + catch (Throwable t) { + err = t; + + throw t; + } finally { + if (err == null && !spi.isNodeStopping0()) + err = new IllegalStateException("Thread " + getName() + " is terminated unexpectedly."); + + FailureProcessor failure = ((IgniteEx)spi.ignite()).context().failure(); + + if (err instanceof OutOfMemoryError) + failure.process(new FailureContext(CRITICAL_ERROR, err)); + else if (err != null) + failure.process(new FailureContext(SYSTEM_WORKER_TERMINATION, err)); + U.closeQuiet(srvrSock); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/StripedExecutorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/StripedExecutorTest.java index 543907fcde60a..3fca7afed7ff5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/StripedExecutorTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/StripedExecutorTest.java @@ -29,7 +29,7 @@ public class StripedExecutorTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override public void beforeTest() { - stripedExecSvc = new StripedExecutor(3, "foo name", "pool name", new JavaLogger()); + stripedExecSvc = new StripedExecutor(3, "foo name", "pool name", new JavaLogger(), (thread, t) -> {}); } /** {@inheritDoc} */ From 912433ba9aa113508d05930691b251eccd8f5870 Mon Sep 17 00:00:00 2001 From: Aleksey Plekhanov Date: Tue, 10 Apr 2018 18:54:03 +0300 Subject: [PATCH 0015/1463] IGNITE-8069 IgniteOutOfMemoryException should be handled accordingly to provided failure handler Signed-off-by: Andrey Gura --- .../pagemem/impl/PageMemoryNoStoreImpl.java | 17 ++- .../persistence/pagemem/PageMemoryImpl.java | 17 ++- .../failure/AbstractFailureHandlerTest.java | 74 +++++++++ .../failure/IoomFailureHandlerTest.java | 144 ++++++++++++++++++ .../pagemem/PageMemoryImplTest.java | 9 ++ .../testsuites/IgniteBasicTestSuite.java | 2 + 6 files changed, 259 insertions(+), 4 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/failure/AbstractFailureHandlerTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/failure/IoomFailureHandlerTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java index 7424af664a313..d4b22a6a2b1cc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java @@ -28,6 +28,8 @@ import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.failure.FailureContext; +import org.apache.ignite.failure.FailureType; import org.apache.ignite.internal.mem.DirectMemoryProvider; import org.apache.ignite.internal.mem.DirectMemoryRegion; import org.apache.ignite.internal.mem.IgniteOutOfMemoryException; @@ -158,6 +160,9 @@ public class PageMemoryNoStoreImpl implements PageMemory { /** */ private final boolean trackAcquiredPages; + /** Shared context. */ + private final GridCacheSharedContext ctx; + /** * @param log Logger. * @param directMemoryProvider Memory allocator to use. @@ -184,6 +189,7 @@ public PageMemoryNoStoreImpl( this.trackAcquiredPages = trackAcquiredPages; this.memMetrics = memMetrics; this.dataRegionCfg = dataRegionCfg; + this.ctx = sharedCtx; sysPageSize = pageSize + PAGE_OVERHEAD; @@ -288,8 +294,8 @@ public PageMemoryNoStoreImpl( } } - if (relPtr == INVALID_REL_PTR) - throw new IgniteOutOfMemoryException("Out of memory in data region [" + + if (relPtr == INVALID_REL_PTR) { + IgniteOutOfMemoryException oom = new IgniteOutOfMemoryException("Out of memory in data region [" + "name=" + dataRegionCfg.getName() + ", initSize=" + U.readableSize(dataRegionCfg.getInitialSize(), false) + ", maxSize=" + U.readableSize(dataRegionCfg.getMaxSize(), false) + @@ -299,6 +305,13 @@ public PageMemoryNoStoreImpl( " ^-- Enable eviction or expiration policies" ); + if (ctx != null) + ctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, oom)); + + throw oom; + } + + assert (relPtr & ~PageIdUtils.PAGE_IDX_MASK) == 0 : U.hexLong(relPtr & ~PageIdUtils.PAGE_IDX_MASK); // Assign page ID according to flags and partition ID. 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 46fb7ddbb37c7..4463224b64686 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 @@ -40,6 +40,8 @@ import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.configuration.DataRegionConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.failure.FailureContext; +import org.apache.ignite.failure.FailureType; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.mem.DirectMemoryProvider; import org.apache.ignite.internal.mem.DirectMemoryRegion; @@ -543,7 +545,7 @@ else if (throttlingPlc == ThrottlingPolicy.CHECKPOINT_BUFFER_ONLY) catch (IgniteOutOfMemoryException oom) { DataRegionConfiguration dataRegionCfg = getDataRegionConfiguration(); - throw (IgniteOutOfMemoryException) new IgniteOutOfMemoryException("Out of memory in data region [" + + IgniteOutOfMemoryException e = new IgniteOutOfMemoryException("Out of memory in data region [" + "name=" + dataRegionCfg.getName() + ", initSize=" + U.readableSize(dataRegionCfg.getInitialSize(), false) + ", maxSize=" + U.readableSize(dataRegionCfg.getMaxSize(), false) + @@ -551,7 +553,13 @@ else if (throttlingPlc == ThrottlingPolicy.CHECKPOINT_BUFFER_ONLY) " ^-- Increase maximum off-heap memory size (DataRegionConfiguration.maxSize)" + U.nl() + " ^-- Enable Ignite persistence (DataRegionConfiguration.persistenceEnabled)" + U.nl() + " ^-- Enable eviction or expiration policies" - ).initCause(oom); + ); + + e.initCause(oom); + + ctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e)); + + throw e; } finally { seg.writeLock().unlock(); @@ -746,6 +754,11 @@ else if (relPtr == OUTDATED_REL_PTR) { return absPtr; } + catch (IgniteOutOfMemoryException oom) { + ctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, oom)); + + throw oom; + } finally { seg.writeLock().unlock(); diff --git a/modules/core/src/test/java/org/apache/ignite/failure/AbstractFailureHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/failure/AbstractFailureHandlerTest.java new file mode 100644 index 0000000000000..dc5f1f5e9caeb --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/failure/AbstractFailureHandlerTest.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.failure; + +import org.apache.ignite.Ignite; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Abstract failure handler test. + */ +public class AbstractFailureHandlerTest extends GridCommonAbstractTest { + /** {@inheritDoc} */ + @Override protected FailureHandler getFailureHandler(String igniteInstanceName) { + return new DummyFailureHandler(); + } + + /** + * Gets dummy failure handler for ignite instance. + * + * @param ignite Ignite. + */ + protected static DummyFailureHandler dummyFailureHandler(Ignite ignite) { + return (DummyFailureHandler)ignite.configuration().getFailureHandler(); + } + + /** + * + */ + protected static class DummyFailureHandler implements FailureHandler { + /** Failure. */ + private volatile boolean failure; + + /** Failure context. */ + private volatile FailureContext ctx; + + /** {@inheritDoc} */ + @Override public boolean onFailure(Ignite ignite, FailureContext failureCtx) { + failure = true; + + ctx = failureCtx; + + return true; + } + + /** + * @return Failure. + */ + public boolean failure() { + return failure; + } + + /** + * @return Failure context. + */ + public FailureContext failureContext() { + return ctx; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/failure/IoomFailureHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/failure/IoomFailureHandlerTest.java new file mode 100644 index 0000000000000..a777f815a89e2 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/failure/IoomFailureHandlerTest.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.failure; + +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +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.mem.IgniteOutOfMemoryException; +import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.transactions.Transaction; + +/** + * IgniteOutOfMemoryError failure handler test. + */ +public class IoomFailureHandlerTest extends AbstractFailureHandlerTest { + /** Offheap size for memory policy. */ + private static final int SIZE = 10 * 1024 * 1024; + + /** Page size. */ + static final int PAGE_SIZE = 2048; + + /** Number of entries. */ + static final int ENTRIES = 10_000; + + /** PDS enabled. */ + private boolean pds; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + DataStorageConfiguration dsCfg = new DataStorageConfiguration(); + + DataRegionConfiguration dfltPlcCfg = new DataRegionConfiguration(); + + dfltPlcCfg.setName("dfltPlc"); + dfltPlcCfg.setInitialSize(SIZE); + dfltPlcCfg.setMaxSize(SIZE); + + if (pds) + dfltPlcCfg.setPersistenceEnabled(true); + + dsCfg.setDefaultDataRegionConfiguration(dfltPlcCfg); + dsCfg.setPageSize(PAGE_SIZE); + + cfg.setDataStorageConfiguration(dsCfg); + + CacheConfiguration ccfg = new CacheConfiguration<>() + .setName(DEFAULT_CACHE_NAME) + .setCacheMode(CacheMode.PARTITIONED) + .setBackups(0) + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + + cfg.setCacheConfiguration(ccfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + cleanPersistenceDir(); + + super.beforeTest(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + cleanPersistenceDir(); + } + + /** + * Test IgniteOutOfMemoryException handling with no store. + */ + public void testIoomErrorNoStoreHandling() throws Exception { + testIoomErrorHandling(false); + } + + /** + * Test IgniteOutOfMemoryException handling with PDS. + */ + public void testIoomErrorPdsHandling() throws Exception { + testIoomErrorHandling(true); + } + + /** + * Test IOOME handling. + */ + public void testIoomErrorHandling(boolean pds) throws Exception { + this.pds = pds; + + IgniteEx ignite0 = startGrid(0); + IgniteEx ignite1 = startGrid(1); + + try { + if (pds) + ignite0.cluster().active(true); + + IgniteCache cache0 = ignite0.getOrCreateCache(DEFAULT_CACHE_NAME); + IgniteCache cache1 = ignite1.getOrCreateCache(DEFAULT_CACHE_NAME); + + awaitPartitionMapExchange(); + + try (Transaction tx = ignite0.transactions().txStart()) { + for (Integer i : primaryKeys(cache1, ENTRIES)) + cache0.put(i, new byte[PAGE_SIZE / 3 * 2]); + + tx.commit(); + } + catch (Throwable ignore) { + // Expected. + } + + assertFalse(dummyFailureHandler(ignite0).failure()); + assertTrue(dummyFailureHandler(ignite1).failure()); + assertTrue(X.hasCause(dummyFailureHandler(ignite1).failureContext().error(), IgniteOutOfMemoryException.class)); + } + finally { + stopGrid(1); + stopGrid(0); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java index 31af1181cdfe7..3697c4cc7b499 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java @@ -28,6 +28,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.failure.NoOpFailureHandler; import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; import org.apache.ignite.internal.mem.DirectMemoryProvider; import org.apache.ignite.internal.mem.IgniteOutOfMemoryException; @@ -41,6 +42,7 @@ import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl; 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.failure.FailureProcessor; import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor; import org.apache.ignite.internal.util.lang.GridInClosure3X; import org.apache.ignite.plugin.PluginProvider; @@ -268,10 +270,17 @@ private PageMemoryImpl createPageMemory(PageMemoryImpl.ThrottlingPolicy throttli IgniteConfiguration igniteCfg = new IgniteConfiguration(); igniteCfg.setDataStorageConfiguration(new DataStorageConfiguration()); + igniteCfg.setFailureHandler(new NoOpFailureHandler()); GridTestKernalContext kernalCtx = new GridTestKernalContext(new GridTestLog4jLogger(), igniteCfg); kernalCtx.add(new IgnitePluginProcessor(kernalCtx, igniteCfg, Collections.emptyList())); + FailureProcessor failureProc = new FailureProcessor(kernalCtx); + + failureProc.start(); + + kernalCtx.add(failureProc); + GridCacheSharedContext sharedCtx = new GridCacheSharedContext<>( kernalCtx, null, diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java index dd9cdfdbcb835..c4b7d9227f4ee 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java @@ -21,6 +21,7 @@ import junit.framework.TestSuite; import org.apache.ignite.GridSuppressedExceptionSelfTest; import org.apache.ignite.failure.FailureHandlerTriggeredTest; +import org.apache.ignite.failure.IoomFailureHandlerTest; import org.apache.ignite.failure.StopNodeFailureHandlerTest; import org.apache.ignite.failure.StopNodeOrHaltFailureHandlerTest; import org.apache.ignite.internal.ClassSetTest; @@ -197,6 +198,7 @@ public static TestSuite suite(@Nullable final Set ignoredTests) throws Ex suite.addTestSuite(FailureHandlerTriggeredTest.class); suite.addTestSuite(StopNodeFailureHandlerTest.class); suite.addTestSuite(StopNodeOrHaltFailureHandlerTest.class); + suite.addTestSuite(IoomFailureHandlerTest.class); return suite; } From 99feab6ace66d011b677fd4d57b44fc54da8fd4f Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Tue, 10 Apr 2018 20:33:47 +0300 Subject: [PATCH 0016/1463] IGNITE-6430 Complete failing test early --- .../CacheGroupsMetricsRebalanceTest.java | 31 +++++++++++-------- 1 file changed, 18 insertions(+), 13 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupsMetricsRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupsMetricsRebalanceTest.java index 89c82365c1905..ceb98522d5a25 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupsMetricsRebalanceTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupsMetricsRebalanceTest.java @@ -227,29 +227,34 @@ public void testRebalanceEstimateFinishTime() throws Exception { System.out.println("Wait until keys left will be less " + keysLine); - while (finishRebalanceLatch.getCount() != 0) { - CacheMetrics m = ig2.cache(CACHE1).localMetrics(); + try { + while (finishRebalanceLatch.getCount() != 0) { + CacheMetrics m = ig2.cache(CACHE1).localMetrics(); - long keyLeft = m.getKeysToRebalanceLeft(); + long keyLeft = m.getKeysToRebalanceLeft(); - if (keyLeft > 0 && keyLeft < keysLine) - latch.countDown(); + if (keyLeft > 0 && keyLeft < keysLine) + latch.countDown(); - System.out.println("Keys left: " + m.getKeysToRebalanceLeft()); + System.out.println("Keys left: " + m.getKeysToRebalanceLeft()); - try { - Thread.sleep(1_000); - } - catch (InterruptedException e) { - System.out.println("Interrupt thread: " + e.getMessage()); + try { + Thread.sleep(1_000); + } + catch (InterruptedException e) { + System.out.println("Interrupt thread: " + e.getMessage()); - Thread.currentThread().interrupt(); + Thread.currentThread().interrupt(); + } } } + finally { + latch.countDown(); + } } }); - latch.await(); + assertTrue(latch.await(getTestTimeout(), TimeUnit.MILLISECONDS)); long finishTime = ig2.cache(CACHE1).localMetrics().getEstimatedRebalancingFinishTime(); From 526fb0ee612ef71fde58a1274db35e8205304a63 Mon Sep 17 00:00:00 2001 From: Dmitriy Sorokin Date: Tue, 10 Apr 2018 22:20:41 +0300 Subject: [PATCH 0017/1463] IGNITE-8101 Ability to terminate system workers by JMX for test purposes. Signed-off-by: Andrey Gura --- .../apache/ignite/IgniteSystemProperties.java | 7 ++ .../failure/StopNodeOrHaltFailureHandler.java | 2 +- .../ignite/internal/GridKernalContext.java | 8 ++ .../internal/GridKernalContextImpl.java | 10 +++ .../apache/ignite/internal/IgniteKernal.java | 16 +++- .../discovery/GridDiscoveryManager.java | 2 +- .../GridCachePartitionExchangeManager.java | 3 +- .../GridCacheSharedTtlCleanupManager.java | 3 +- .../reader/StandaloneGridKernalContext.java | 6 ++ .../timeout/GridTimeoutProcessor.java | 3 +- .../ignite/internal/util/IgniteUtils.java | 7 +- .../worker/WorkersControlMXBeanImpl.java | 62 ++++++++++++++ .../internal/worker/WorkersRegistry.java | 80 +++++++++++++++++++ .../ignite/internal/worker/package-info.java | 22 +++++ .../ignite/mxbean/WorkersControlMXBean.java | 49 ++++++++++++ 15 files changed, 271 insertions(+), 9 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersControlMXBeanImpl.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersRegistry.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/worker/package-info.java create mode 100644 modules/core/src/main/java/org/apache/ignite/mxbean/WorkersControlMXBean.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 152d845afbac5..9da123e01fb35 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -422,6 +422,13 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_MBEANS_DISABLED = "IGNITE_MBEANS_DISABLED"; + /** + * If property is set to {@code true}, then test features will be enabled. + * + * Default is {@code false}. + */ + public static final String IGNITE_TEST_FEATURES_ENABLED = "IGNITE_TEST_FEATURES_ENABLED"; + /** * Property controlling size of buffer holding last exception. Default value of {@code 1000}. */ diff --git a/modules/core/src/main/java/org/apache/ignite/failure/StopNodeOrHaltFailureHandler.java b/modules/core/src/main/java/org/apache/ignite/failure/StopNodeOrHaltFailureHandler.java index 4f7440616c097..3ce4ff6f27ef6 100644 --- a/modules/core/src/main/java/org/apache/ignite/failure/StopNodeOrHaltFailureHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/failure/StopNodeOrHaltFailureHandler.java @@ -92,7 +92,7 @@ public StopNodeOrHaltFailureHandler(boolean tryStop, long timeout) { ).start(); } else { - U.error(log, "JVM will be halted immediately on ignite failure: [failureCtx=" + failureCtx + ']'); + U.error(log, "JVM will be halted immediately due to the failure: [failureCtx=" + failureCtx + ']'); Runtime.getRuntime().halt(Ignition.KILL_EXIT_CODE); } 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 0b40054ec2606..505c3d6a4b0b1 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.worker.WorkersRegistry; import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor; import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor; import org.apache.ignite.internal.processors.cache.GridCacheProcessor; @@ -422,6 +423,13 @@ public interface GridKernalContext extends Iterable { */ public GridIndexingManager indexing(); + /** + * Gets workers registry. + * + * @return Workers registry. + */ + public WorkersRegistry workersRegistry(); + /** * Gets data structures processor. * 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 34083340bfefe..ac4970859d2b9 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 @@ -46,6 +46,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.worker.WorkersRegistry; import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor; import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor; import org.apache.ignite.internal.processors.cache.CacheConflictResolutionManager; @@ -360,6 +361,10 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable @GridToStringExclude private Map attrs = new HashMap<>(); + /** */ + @GridToStringExclude + private final WorkersRegistry workersRegistry = new WorkersRegistry(); + /** */ private IgniteEx grid; @@ -779,6 +784,11 @@ else if (helper instanceof HadoopHelper) return indexingMgr; } + /** {@inheritDoc} */ + @Override public WorkersRegistry workersRegistry() { + return workersRegistry; + } + /** {@inheritDoc} */ @Override public GridAffinityProcessor affinity() { return affProc; 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 0b102e5b665f7..1cb07b91e82b3 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,7 +125,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheProcessor; import org.apache.ignite.internal.processors.cache.GridCacheUtilityKey; -import org.apache.ignite.internal.processors.cache.GridCacheUtils; 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; @@ -185,6 +184,8 @@ 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.worker.WorkersControlMXBeanImpl; +import org.apache.ignite.internal.worker.WorkersRegistry; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgnitePredicate; @@ -197,6 +198,7 @@ import org.apache.ignite.mxbean.ClusterMetricsMXBean; import org.apache.ignite.mxbean.IgniteMXBean; import org.apache.ignite.mxbean.StripedExecutorMXBean; +import org.apache.ignite.mxbean.WorkersControlMXBean; import org.apache.ignite.mxbean.ThreadPoolMXBean; import org.apache.ignite.plugin.IgnitePlugin; import org.apache.ignite.plugin.PluginNotFoundException; @@ -1085,7 +1087,7 @@ public void start( // Register MBeans. mBeansMgr.registerAllMBeans(utilityCachePool, execSvc, svcExecSvc, sysExecSvc, stripedExecSvc, p2pExecSvc, mgmtExecSvc, igfsExecSvc, dataStreamExecSvc, restExecSvc, affExecSvc, idxExecSvc, callbackExecSvc, - qryExecSvc, schemaExecSvc, customExecSvcs); + qryExecSvc, schemaExecSvc, customExecSvcs, ctx.workersRegistry()); // Lifecycle bean notifications. notifyLifecycleBeans(AFTER_NODE_START); @@ -4175,7 +4177,8 @@ private void registerAllMBeans( IgniteStripedThreadPoolExecutor callbackExecSvc, ExecutorService qryExecSvc, ExecutorService schemaExecSvc, - @Nullable final Map customExecSvcs + @Nullable final Map customExecSvcs, + WorkersRegistry workersRegistry ) throws IgniteCheckedException { if (U.IGNITE_MBEANS_DISABLED) return; @@ -4221,6 +4224,13 @@ private void registerAllMBeans( for (Map.Entry entry : customExecSvcs.entrySet()) registerExecutorMBean(entry.getKey(), entry.getValue()); } + + if (U.IGNITE_TEST_FEATURES_ENABLED) { + WorkersControlMXBean workerCtrlMXBean = new WorkersControlMXBeanImpl(workersRegistry); + + registerMBean("Kernal", workerCtrlMXBean.getClass().getSimpleName(), + workerCtrlMXBean, WorkersControlMXBean.class); + } } /** 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 b0d32564aebf0..a1d84e56bb414 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 @@ -2591,7 +2591,7 @@ private class DiscoveryWorker extends GridWorker { * */ private DiscoveryWorker() { - super(ctx.igniteInstanceName(), "disco-event-worker", GridDiscoveryManager.this.log); + super(ctx.igniteInstanceName(), "disco-event-worker", GridDiscoveryManager.this.log, ctx.workersRegistry()); } /** 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 e40493fbc589a..1a0e65f5b8a9f 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 @@ -2077,7 +2077,8 @@ private class ExchangeWorker extends GridWorker { * Constructor. */ private ExchangeWorker() { - super(cctx.igniteInstanceName(), "partition-exchanger", GridCachePartitionExchangeManager.this.log); + super(cctx.igniteInstanceName(), "partition-exchanger", GridCachePartitionExchangeManager.this.log, + cctx.kernalContext().workersRegistry()); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedTtlCleanupManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedTtlCleanupManager.java index 613e93bc88c23..7adabc3694e9f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedTtlCleanupManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedTtlCleanupManager.java @@ -121,7 +121,8 @@ private class CleanupWorker extends GridWorker { * Creates cleanup worker. */ CleanupWorker() { - super(cctx.igniteInstanceName(), "ttl-cleanup-worker", cctx.logger(GridCacheSharedTtlCleanupManager.class)); + super(cctx.igniteInstanceName(), "ttl-cleanup-worker", cctx.logger(GridCacheSharedTtlCleanupManager.class), + cctx.kernalContext().workersRegistry()); } /** {@inheritDoc} */ 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 429a5ce82bd52..cb04575c8810a 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 @@ -86,6 +86,7 @@ import org.apache.ignite.internal.util.IgniteExceptionRegistry; import org.apache.ignite.internal.util.StripedExecutor; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.worker.WorkersRegistry; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.PluginNotFoundException; import org.apache.ignite.plugin.PluginProvider; @@ -453,6 +454,11 @@ private IgniteConfiguration prepareIgniteConfiguration() { return null; } + /** {@inheritDoc} */ + @Override public WorkersRegistry workersRegistry() { + return null; + } + /** {@inheritDoc} */ @Override public DataStructuresProcessor dataStructures() { return null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java index a09d6faad85e9..25151cf7221bf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java @@ -145,7 +145,8 @@ private class TimeoutWorker extends GridWorker { * */ TimeoutWorker() { - super(ctx.config().getIgniteInstanceName(), "grid-timeout-worker", GridTimeoutProcessor.this.log); + super(ctx.config().getIgniteInstanceName(), "grid-timeout-worker", + GridTimeoutProcessor.this.log, ctx.workersRegistry()); } /** {@inheritDoc} */ 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 93f4fb4a39f2e..42e96fb6067f0 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 @@ -522,7 +522,12 @@ public abstract class IgniteUtils { }; /** Ignite MBeans disabled flag. */ - public static boolean IGNITE_MBEANS_DISABLED = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_MBEANS_DISABLED); + public static boolean IGNITE_MBEANS_DISABLED = + IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_MBEANS_DISABLED); + + /** Ignite test features enabled flag. */ + public static boolean IGNITE_TEST_FEATURES_ENABLED = + IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_TEST_FEATURES_ENABLED); /** */ private static final boolean assertionsEnabled; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersControlMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersControlMXBeanImpl.java new file mode 100644 index 0000000000000..9e427e8f2ab1c --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersControlMXBeanImpl.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.worker; + +import java.util.ArrayList; +import java.util.List; +import org.apache.ignite.internal.util.worker.GridWorker; +import org.apache.ignite.mxbean.WorkersControlMXBean; + +/** + * MBean that provides control of system workersRegistry. + */ +public class WorkersControlMXBeanImpl implements WorkersControlMXBean { + /** System worker registry. */ + private final WorkersRegistry workerRegistry; + + /** + * Constructor. + * + * @param registry System worker registry. + */ + public WorkersControlMXBeanImpl(WorkersRegistry registry) { + workerRegistry = registry; + } + + /** {@inheritDoc} */ + @Override public List getWorkerNames() { + return new ArrayList<>(workerRegistry.names()); + } + + /** {@inheritDoc} */ + @Override public boolean terminateWorker(String name) { + GridWorker w = workerRegistry.worker(name); + + if (w == null || w.isCancelled()) + return false; + + Thread t = w.runner(); + + if (t == null) + return false; + + t.interrupt(); + + return true; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersRegistry.java b/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersRegistry.java new file mode 100644 index 0000000000000..e8d46fb2ebddb --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersRegistry.java @@ -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. + */ + +package org.apache.ignite.internal.worker; + +import java.util.Collection; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import org.apache.ignite.internal.util.worker.GridWorker; +import org.apache.ignite.internal.util.worker.GridWorkerListener; + +/** + * Workers registry. + */ +public class WorkersRegistry implements GridWorkerListener { + /** Registered workers. */ + private final ConcurrentMap registeredWorkers = new ConcurrentHashMap<>(); + + /** + * Adds worker to the registry. + * + * @param w Worker. + */ + public void register(GridWorker w) { + if (registeredWorkers.putIfAbsent(w.name(), w) != null) + throw new IllegalStateException("Worker is already registered [worker=" + w + ']'); + } + + /** + * Removes worker from the registry. + * + * @param name Worker name. + */ + public void unregister(String name) { + registeredWorkers.remove(name); + } + + /** + * Returns names of all registered workers. + * + * @return Registered worker names. + */ + public Collection names() { + return registeredWorkers.keySet(); + } + + /** + * Returns worker with given name. + * + * @param name Name. + * @return Registered {@link GridWorker} with name {@code name} or {@code null} if not found. + */ + public GridWorker worker(String name) { + return registeredWorkers.get(name); + } + + /** {@inheritDoc} */ + @Override public void onStarted(GridWorker w) { + register(w); + } + + /** {@inheritDoc} */ + @Override public void onStopped(GridWorker w) { + unregister(w.name()); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/worker/package-info.java b/modules/core/src/main/java/org/apache/ignite/internal/worker/package-info.java new file mode 100644 index 0000000000000..03ca62116b7f7 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/worker/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. + */ + +/** + * + * System worker registry and control MBean implementation. + */ +package org.apache.ignite.internal.worker; \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/WorkersControlMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/WorkersControlMXBean.java new file mode 100644 index 0000000000000..0f5419b3b4aac --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/mxbean/WorkersControlMXBean.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.mxbean; + +import java.util.List; + +/** + * MBean that provides ability to terminate worker that registered in the workers registry. + */ +@MXBeanDescription("MBean that provides ability to terminate worker that registered in the workers registry.") +public interface WorkersControlMXBean { + /** + * Returns names of all registered workers. + * + * @return Worker names. + */ + @MXBeanDescription("Names of registered workers.") + public List getWorkerNames(); + + /** + * Terminates worker. + * + * @param name Worker name. + * @return {@code True} if worker has been terminated successfully, {@code false} otherwise. + */ + @MXBeanDescription("Terminates worker.") + @MXBeanParametersNames( + "name" + ) + @MXBeanParametersDescriptions( + "Name of worker to terminate." + ) + public boolean terminateWorker(String name); +} From b4cb2f0df944534743a9d73811e047eda572258c Mon Sep 17 00:00:00 2001 From: mcherkasov Date: Tue, 10 Apr 2018 17:27:20 -0700 Subject: [PATCH 0018/1463] IGNITE-8153 Nodes fail to connect each other when SSL is enabled - Fixes #3773. Signed-off-by: Valentin Kulichenko --- .../internal/util/nio/ssl/BlockingSslHandler.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java index 638106f667a3c..0099c4675a381 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java @@ -373,9 +373,10 @@ private HandshakeStatus runTasks() { * @throws GridNioException If failed to pass event to the next filter. */ private Status unwrapHandshake() throws SSLException, IgniteCheckedException { - // Flip input buffer so we can read the collected data. - readFromNet(); + if(!inNetBuf.hasRemaining()) + readFromNet(); + // Flip input buffer so we can read the collected data. inNetBuf.flip(); SSLEngineResult res = unwrap0(); @@ -399,7 +400,10 @@ private Status unwrapHandshake() throws SSLException, IgniteCheckedException { else if (res.getStatus() == BUFFER_UNDERFLOW) { inNetBuf.compact(); - inNetBuf = expandBuffer(inNetBuf, inNetBuf.capacity() * 2); + if(inNetBuf.capacity() == inNetBuf.limit()) + inNetBuf = expandBuffer(inNetBuf, inNetBuf.capacity() * 2); + + readFromNet(); } else // prepare to be written again From b4cc9f2d45d78c360abe224165e707c23533469e Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 11 Apr 2018 11:23:46 +0300 Subject: [PATCH 0019/1463] IGNITE-7871 Implemented additional synchronization phase for correct partition counters update --- .../org/apache/ignite/internal/GridTopic.java | 5 +- .../communication/GridIoMessageFactory.java | 6 + .../discovery/GridDiscoveryManager.java | 10 + .../MetaPageUpdatePartitionDataRecord.java | 2 +- .../processors/cache/CacheMetricsImpl.java | 2 +- .../cache/GridCacheMvccManager.java | 38 + .../GridCachePartitionExchangeManager.java | 17 + .../cache/GridCacheSharedContext.java | 9 +- .../processors/cache/GridCacheUtils.java | 2 +- .../cache/IgniteCacheOffheapManager.java | 8 +- .../cache/IgniteCacheOffheapManagerImpl.java | 10 +- .../dht/GridClientPartitionTopology.java | 5 + .../dht/GridDhtLocalPartition.java | 9 +- .../dht/GridDhtPartitionTopology.java | 6 + .../dht/GridDhtPartitionTopologyImpl.java | 26 +- .../dht/GridDhtPartitionsStateValidator.java | 255 +++++++ .../cache/distributed/dht/GridDhtTxLocal.java | 5 + .../GridDhtPartitionsExchangeFuture.java | 96 ++- .../GridDhtPartitionsSingleMessage.java | 68 +- .../preloader/InitNewCoordinatorFuture.java | 2 +- .../preloader/latch/ExchangeLatchManager.java | 695 ++++++++++++++++++ .../dht/preloader/latch/Latch.java | 52 ++ .../dht/preloader/latch/LatchAckMessage.java | 165 +++++ .../distributed/near/GridNearTxLocal.java | 10 + .../persistence/GridCacheOffheapManager.java | 10 +- .../cache/transactions/IgniteTxAdapter.java | 2 +- .../cache/transactions/IgniteTxManager.java | 36 +- ...eDhtLocalPartitionAfterRemoveSelfTest.java | 2 +- .../cache/IgniteCacheGroupsTest.java | 1 + ...changeLatchManagerCoordinatorFailTest.java | 244 ++++++ ...ridCachePartitionsStateValidationTest.java | 316 ++++++++ ...CachePartitionsStateValidatorSelfTest.java | 158 ++++ .../TxOptimisticOnPartitionExchangeTest.java | 322 ++++++++ .../testsuites/IgniteCacheTestSuite.java | 4 + .../testsuites/IgniteCacheTestSuite6.java | 6 + 35 files changed, 2568 insertions(+), 36 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsStateValidator.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/Latch.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/LatchAckMessage.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteExchangeLatchManagerCoordinatorFailTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidationTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidatorSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticOnPartitionExchangeTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java index 1227e8cf4abd4..0b2d41a39ec89 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 @@ -124,7 +124,10 @@ public enum GridTopic { TOPIC_METRICS, /** */ - TOPIC_AUTH; + TOPIC_AUTH, + + /** */ + TOPIC_EXCHANGE; /** Enum values. */ private static final GridTopic[] VALS = values(); 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 5616fd035d445..581c32e4b1fb0 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 @@ -53,6 +53,7 @@ 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; @@ -921,6 +922,11 @@ public GridIoMessageFactory(MessageFactory[] ext) { break; + case 135: + msg = new LatchAckMessage(); + + 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/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java index a1d84e56bb414..400bb5fd28742 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 @@ -793,6 +793,9 @@ else if (type == EVT_CLIENT_NODE_DISCONNECTED) { ((IgniteKernal)ctx.grid()).onDisconnected(); + if (!locJoin.isDone()) + locJoin.onDone(new IgniteCheckedException("Node disconnected")); + locJoin = new GridFutureAdapter<>(); registeredCaches.clear(); @@ -2141,6 +2144,13 @@ public DiscoveryLocalJoinData localJoin() { } } + /** + * @return Local join future. + */ + public GridFutureAdapter localJoinFuture() { + return locJoin; + } + /** * @param msg Custom message. * @throws IgniteCheckedException If failed. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java index bafbf475abe9f..e5bd343bcb5ee 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java @@ -32,7 +32,7 @@ public class MetaPageUpdatePartitionDataRecord extends PageDeltaRecord { /** */ private long globalRmvId; - /** */ + /** TODO: Partition size may be long */ private int partSize; /** */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java index 6fae8feff3c6c..b402ff2bb9170 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java @@ -792,7 +792,7 @@ public EntriesStatMetrics getEntriesStat() { if (cctx.cache() == null) continue; - int cacheSize = part.dataStore().cacheSize(cctx.cacheId()); + long cacheSize = part.dataStore().cacheSize(cctx.cacheId()); offHeapEntriesCnt += cacheSize; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java index a9fa3c777a26b..fade83377a8e7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java @@ -44,6 +44,8 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.GridCacheMappedVersion; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedCacheEntry; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishFuture; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishFuture; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; @@ -313,6 +315,42 @@ public Collection> activeFutures() { return col; } + /** + * Creates a future that will wait for finishing all remote transactions (primary -> backup) + * with topology version less or equal to {@code topVer}. + * + * @param topVer Topology version. + * @return Compound future of all {@link GridDhtTxFinishFuture} futures. + */ + public IgniteInternalFuture finishRemoteTxs(AffinityTopologyVersion topVer) { + GridCompoundFuture res = new CacheObjectsReleaseFuture<>("RemoteTx", topVer); + + for (GridCacheFuture fut : futs.values()) { + if (fut instanceof GridDhtTxFinishFuture) { + GridDhtTxFinishFuture finishTxFuture = (GridDhtTxFinishFuture) fut; + + if (cctx.tm().needWaitTransaction(finishTxFuture.tx(), topVer)) + res.add(ignoreErrors(finishTxFuture)); + } + } + + res.markInitialized(); + + return res; + } + + /** + * Future wrapper which ignores any underlying future errors. + * + * @param f Underlying future. + * @return Future wrapper which ignore any underlying future errors. + */ + private IgniteInternalFuture ignoreErrors(IgniteInternalFuture f) { + GridFutureAdapter wrapper = new GridFutureAdapter(); + f.listen(future -> wrapper.onDone()); + return wrapper; + } + /** * @param leftNodeId Left node ID. * @param topVer Topology version. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index 1a0e65f5b8a9f..20a3ccbf4b2f3 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 @@ -64,6 +64,7 @@ import org.apache.ignite.internal.managers.eventstorage.DiscoveryEventListener; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.ExchangeLatchManager; import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture; @@ -216,6 +217,9 @@ public class GridCachePartitionExchangeManager extends GridCacheSharedMana /** For tests only. */ private volatile AffinityTopologyVersion exchMergeTestWaitVer; + /** Distributed latch manager. */ + private ExchangeLatchManager latchMgr; + /** Discovery listener. */ private final DiscoveryEventListener discoLsnr = new DiscoveryEventListener() { @Override public void onEvent(DiscoveryEvent evt, DiscoCache cache) { @@ -309,6 +313,8 @@ private void processEventInactive(DiscoveryEvent evt, DiscoCache cache) { exchWorker = new ExchangeWorker(); + latchMgr = new ExchangeLatchManager(cctx.kernalContext()); + cctx.gridEvents().addDiscoveryEventListener(discoLsnr, EVT_NODE_JOINED, EVT_NODE_LEFT, EVT_NODE_FAILED, EVT_DISCOVERY_CUSTOM_EVT); @@ -1255,6 +1261,8 @@ public GridDhtPartitionsSingleMessage createPartitionsSingleMessage( m.addPartitionUpdateCounters(grp.groupId(), newCntrMap ? cntrsMap : CachePartitionPartialCountersMap.toCountersMap(cntrsMap)); + + m.addPartitionSizes(grp.groupId(), grp.topology().partitionSizes()); } } } @@ -1277,6 +1285,8 @@ public GridDhtPartitionsSingleMessage createPartitionsSingleMessage( m.addPartitionUpdateCounters(top.groupId(), newCntrMap ? cntrsMap : CachePartitionPartialCountersMap.toCountersMap(cntrsMap)); + + m.addPartitionSizes(top.groupId(), top.partitionSizes()); } } @@ -1569,6 +1579,13 @@ private void processSinglePartitionRequest(ClusterNode node, GridDhtPartitionsSi } } + /** + * @return Latch manager instance. + */ + public ExchangeLatchManager latch() { + return latchMgr; + } + /** * @param exchFut Optional current exchange future. * @throws Exception If failed. 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 c2f9229b8fe5c..b3b4f0dc7b432 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 @@ -26,7 +26,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicIntegerArray; -import java.util.function.BiFunction; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteSystemProperties; @@ -711,7 +710,7 @@ public GridCacheIoManager io() { /** * @return Ttl cleanup manager. - * */ + */ public GridCacheSharedTtlCleanupManager ttl() { return ttlMgr; } @@ -854,10 +853,14 @@ public IgniteInternalFuture partitionReleaseFuture(AffinityTopologyVersion to GridCompoundFuture f = new CacheObjectsReleaseFuture("Partition", topVer); f.add(mvcc().finishExplicitLocks(topVer)); - f.add(tm().finishTxs(topVer)); f.add(mvcc().finishAtomicUpdates(topVer)); f.add(mvcc().finishDataStreamerUpdates(topVer)); + IgniteInternalFuture finishLocalTxsFuture = tm().finishLocalTxs(topVer); + // To properly track progress of finishing local tx updates we explicitly add this future to compound set. + f.add(finishLocalTxsFuture); + f.add(tm().finishAllTxs(finishLocalTxsFuture, topVer)); + f.markInitialized(); return f; 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 a5169d26dcc13..d672420fafdc8 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 @@ -1732,7 +1732,7 @@ private void process(KeyCacheObject key, CacheObject val, GridCacheVersion ver, ver, expiryPlc == null ? 0 : expiryPlc.forCreate(), expiryPlc == null ? 0 : toExpireTime(expiryPlc.forCreate()), - false, + true, topVer, GridDrType.DR_BACKUP, true); 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 3d83f87105db2..a12c0334912a5 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 @@ -22,11 +22,11 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtDemandedPartitionsMap; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; 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.distributed.dht.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; import org.apache.ignite.internal.util.GridAtomicLong; @@ -344,7 +344,7 @@ public long cacheEntriesCount(int cacheId, boolean primary, boolean backup, Affi * @param part Partition. * @return Number of entries. */ - public int totalPartitionEntriesCount(int part); + public long totalPartitionEntriesCount(int part); /** * @@ -381,7 +381,7 @@ interface CacheDataStore { * @param cacheId Cache ID. * @return Size. */ - int cacheSize(int cacheId); + long cacheSize(int cacheId); /** * @return Cache sizes if store belongs to group containing multiple caches. @@ -391,7 +391,7 @@ interface CacheDataStore { /** * @return Total size. */ - int fullSize(); + long fullSize(); /** * @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 b2019352c2f1a..f8cc86f8cecdb 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 @@ -252,7 +252,7 @@ public CacheDataStore dataStore(GridDhtLocalPartition part) { } /** {@inheritDoc} */ - @Override public int totalPartitionEntriesCount(int p) { + @Override public long totalPartitionEntriesCount(int p) { if (grp.isLocal()) return locCacheDataStore.fullSize(); else { @@ -1152,14 +1152,14 @@ void decrementSize(int cacheId) { } /** {@inheritDoc} */ - @Override public int cacheSize(int cacheId) { + @Override public long cacheSize(int cacheId) { if (grp.sharedGroup()) { AtomicLong size = cacheSizes.get(cacheId); return size != null ? (int)size.get() : 0; } - return (int)storageSize.get(); + return storageSize.get(); } /** {@inheritDoc} */ @@ -1176,8 +1176,8 @@ void decrementSize(int cacheId) { } /** {@inheritDoc} */ - @Override public int fullSize() { - return (int)storageSize.get(); + @Override public long fullSize() { + return storageSize.get(); } /** {@inheritDoc} */ 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 5bbbb3102c663..3e3bb0db9b9bb 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 @@ -1195,6 +1195,11 @@ private void removeNode(UUID nodeId) { return CachePartitionPartialCountersMap.EMPTY; } + /** {@inheritDoc} */ + @Override public Map partitionSizes() { + return Collections.emptyMap(); + } + /** {@inheritDoc} */ @Override public boolean rebalanceFinished(AffinityTopologyVersion topVer) { assert false : "Should not be called on non-affinity node"; 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 7a47f31c99055..ea20dbf4b1184 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 @@ -929,7 +929,7 @@ public long updateCounter() { /** * @return Initial update counter. */ - public Long initialUpdateCounter() { + public long initialUpdateCounter() { return store.initialUpdateCounter(); } @@ -947,6 +947,13 @@ public void initialUpdateCounter(long val) { store.updateInitialCounter(val); } + /** + * @return Total size of all caches. + */ + public long fullSize() { + return store.fullSize(); + } + /** * Removes all entries and rows from this partition. * 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 7f900cb67f533..6f68dbbaeb511 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 @@ -19,6 +19,7 @@ import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.UUID; import org.apache.ignite.IgniteCheckedException; @@ -344,6 +345,11 @@ public boolean update(@Nullable GridDhtPartitionExchangeId exchId, */ public CachePartitionPartialCountersMap localUpdateCounters(boolean skipZeros); + /** + * @return Partition cache sizes. + */ + public Map partitionSizes(); + /** * @param part Partition to own. * @return {@code True} if owned. 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 538c57ec9ba97..740903e415baf 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 @@ -31,6 +31,8 @@ import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReferenceArray; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; @@ -2525,6 +2527,28 @@ private void removeNode(UUID nodeId) { } } + /** {@inheritDoc} */ + @Override public Map partitionSizes() { + lock.readLock().lock(); + + try { + Map partitionSizes = new HashMap<>(); + + for (int p = 0; p < locParts.length(); p++) { + GridDhtLocalPartition part = locParts.get(p); + if (part == null || part.fullSize() == 0) + continue; + + partitionSizes.put(part.id(), part.fullSize()); + } + + return partitionSizes; + } + finally { + lock.readLock().unlock(); + } + } + /** {@inheritDoc} */ @Override public boolean rebalanceFinished(AffinityTopologyVersion topVer) { AffinityTopologyVersion curTopVer = this.readyTopVer; @@ -2587,7 +2611,7 @@ public void onCacheStopped(int cacheId) { if (part == null) continue; - int size = part.dataStore().fullSize(); + long size = part.dataStore().fullSize(); if (size >= threshold) X.println(">>> Local partition [part=" + part.id() + ", size=" + size + ']'); 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 new file mode 100644 index 0000000000000..92a05848e3d00 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsStateValidator.java @@ -0,0 +1,255 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Set; +import java.util.TreeMap; +import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.events.DiscoveryEvent; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +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.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.SB; +import org.apache.ignite.lang.IgniteProductVersion; + +import static org.apache.ignite.events.EventType.EVT_NODE_JOINED; + +/** + * Class to validate partitions update counters and cache sizes during exchange process. + */ +public class GridDhtPartitionsStateValidator { + /** Version since node is able to send cache sizes in {@link GridDhtPartitionsSingleMessage}. */ + private static final IgniteProductVersion SIZES_VALIDATION_AVAILABLE_SINCE = IgniteProductVersion.fromString("2.5.0"); + + /** Cache shared context. */ + private final GridCacheSharedContext cctx; + + /** + * Constructor. + * + * @param cctx Cache shared context. + */ + public GridDhtPartitionsStateValidator(GridCacheSharedContext cctx) { + this.cctx = cctx; + } + + /** + * Validates partition states - update counters and cache sizes for all nodes. + * If update counter value or cache size for the same partitions are different on some nodes + * method throws exception with full information about inconsistent partitions. + * + * @param fut Current exchange future. + * @param top Topology to validate. + * @param messages Single messages received from all nodes. + * @throws IgniteCheckedException If validation failed. Exception message contains + * full information about all partitions which update counters or cache sizes are not consistent. + */ + public void validatePartitionCountersAndSizes(GridDhtPartitionsExchangeFuture fut, + GridDhtPartitionTopology top, + Map messages) throws IgniteCheckedException { + // Ignore just joined nodes. + final Set ignoringNodes = new HashSet<>(); + + for (DiscoveryEvent evt : fut.events().events()) + if (evt.type() == EVT_NODE_JOINED) + ignoringNodes.add(evt.eventNode().id()); + + AffinityTopologyVersion topVer = fut.context().events().topologyVersion(); + + // Validate update counters. + Map> result = validatePartitionsUpdateCounters(top, messages, ignoringNodes); + if (!result.isEmpty()) + throw new IgniteCheckedException("Partitions update counters are inconsistent for " + fold(topVer, result)); + + // For sizes validation ignore also nodes which are not able to send cache sizes. + for (UUID id : messages.keySet()) { + ClusterNode node = cctx.discovery().node(id); + if (node != null && node.version().compareTo(SIZES_VALIDATION_AVAILABLE_SINCE) < 0) + ignoringNodes.add(id); + } + + // Validate cache sizes. + result = validatePartitionsSizes(top, messages, ignoringNodes); + if (!result.isEmpty()) + throw new IgniteCheckedException("Partitions cache sizes are inconsistent for " + fold(topVer, result)); + } + + /** + * Validate partitions update counters for given {@code top}. + * + * @param top Topology to validate. + * @param messages Single messages received from all nodes. + * @param ignoringNodes Nodes for what we ignore validation. + * @return Invalid partitions map with following structure: (partId, (nodeId, updateCounter)). + * If map is empty validation is successful. + */ + Map> validatePartitionsUpdateCounters( + GridDhtPartitionTopology top, + Map messages, + Set ignoringNodes) { + Map> invalidPartitions = new HashMap<>(); + + Map> updateCountersAndNodesByPartitions = new HashMap<>(); + + // Populate counters statistics from local node partitions. + for (GridDhtLocalPartition part : top.currentLocalPartitions()) { + if (top.partitionState(cctx.localNodeId(), part.id()) != GridDhtPartitionState.OWNING) + continue; + + updateCountersAndNodesByPartitions.put(part.id(), new T2<>(cctx.localNodeId(), part.updateCounter())); + } + + int partitions = top.partitions(); + + // Then process and validate counters from other nodes. + for (Map.Entry e : messages.entrySet()) { + UUID nodeId = e.getKey(); + if (ignoringNodes.contains(nodeId)) + continue; + + CachePartitionPartialCountersMap countersMap = e.getValue().partitionUpdateCounters(top.groupId(), partitions); + + for (int part = 0; part < partitions; part++) { + if (top.partitionState(nodeId, part) != GridDhtPartitionState.OWNING) + continue; + + int partIdx = countersMap.partitionIndex(part); + long currentCounter = partIdx >= 0 ? countersMap.updateCounterAt(partIdx) : 0; + + process(invalidPartitions, updateCountersAndNodesByPartitions, part, nodeId, currentCounter); + } + } + + return invalidPartitions; + } + + /** + * Validate partitions cache sizes for given {@code top}. + * + * @param top Topology to validate. + * @param messages Single messages received from all nodes. + * @param ignoringNodes Nodes for what we ignore validation. + * @return Invalid partitions map with following structure: (partId, (nodeId, cacheSize)). + * If map is empty validation is successful. + */ + Map> validatePartitionsSizes( + GridDhtPartitionTopology top, + Map messages, + Set ignoringNodes) { + Map> invalidPartitions = new HashMap<>(); + + Map> sizesAndNodesByPartitions = new HashMap<>(); + + // Populate sizes statistics from local node partitions. + for (GridDhtLocalPartition part : top.currentLocalPartitions()) { + if (top.partitionState(cctx.localNodeId(), part.id()) != GridDhtPartitionState.OWNING) + continue; + + sizesAndNodesByPartitions.put(part.id(), new T2<>(cctx.localNodeId(), part.fullSize())); + } + + int partitions = top.partitions(); + + // Then process and validate sizes from other nodes. + for (Map.Entry e : messages.entrySet()) { + UUID nodeId = e.getKey(); + if (ignoringNodes.contains(nodeId)) + continue; + + Map sizesMap = e.getValue().partitionSizes(top.groupId()); + + for (int part = 0; part < partitions; part++) { + if (top.partitionState(nodeId, part) != GridDhtPartitionState.OWNING) + continue; + + long currentSize = sizesMap.containsKey(part) ? sizesMap.get(part) : 0L; + + process(invalidPartitions, sizesAndNodesByPartitions, part, nodeId, currentSize); + } + } + + return invalidPartitions; + } + + /** + * Processes given {@code counter} for partition {@code part} reported by {@code node}. + * Populates {@code invalidPartitions} map if existing counter and current {@code counter} are different. + * + * @param invalidPartitions Invalid partitions map. + * @param countersAndNodes Current map of counters and nodes by partitions. + * @param part Processing partition. + * @param node Node id. + * @param counter Counter value reported by {@code node}. + */ + private void process(Map> invalidPartitions, + Map> countersAndNodes, + int part, + UUID node, + long counter) { + T2 existingData = countersAndNodes.get(part); + + if (existingData == null) + countersAndNodes.put(part, new T2<>(node, counter)); + + if (existingData != null && counter != existingData.get2()) { + if (!invalidPartitions.containsKey(part)) { + Map map = new HashMap<>(); + map.put(existingData.get1(), existingData.get2()); + invalidPartitions.put(part, map); + } + + invalidPartitions.get(part).put(node, counter); + } + } + + /** + * Folds given map of invalid partition states to string representation in the following format: + * Part [id]: [consistentId=value*] + * + * Value can be both update counter or cache size. + * + * @param topVer Last topology version. + * @param invalidPartitions Invalid partitions map. + * @return String representation of invalid partitions. + */ + private String fold(AffinityTopologyVersion topVer, Map> invalidPartitions) { + SB sb = new SB(); + + NavigableMap> sortedPartitions = new TreeMap<>(invalidPartitions); + + for (Map.Entry> p : sortedPartitions.entrySet()) { + sb.a("Part ").a(p.getKey()).a(": ["); + for (Map.Entry e : p.getValue().entrySet()) { + Object consistentId = cctx.discovery().node(topVer, e.getKey()).consistentId(); + sb.a(consistentId).a("=").a(e.getValue()).a(" "); + } + sb.a("] "); + } + + return sb.toString(); + } +} 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 28cc018657a6c..0609f04017643 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 @@ -447,6 +447,11 @@ private void finishTx(boolean commit, @Nullable IgniteInternalFuture prepFut, Gr err = e; } + catch (Throwable t) { + fut.onDone(t); + + throw t; + } if (primarySync) sendFinishReply(err); 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 cbb49851e1f81..dd4a57157a1e7 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 @@ -41,6 +41,7 @@ import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CacheRebalanceMode; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.events.DiscoveryEvent; @@ -75,10 +76,12 @@ import org.apache.ignite.internal.processors.cache.LocalJoinCachesContext; import org.apache.ignite.internal.processors.cache.StateChangeRequest; import org.apache.ignite.internal.processors.cache.WalStateAbstractMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.Latch; import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology; 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.distributed.dht.GridDhtPartitionsStateValidator; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFutureAdapter; import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotDiscoveryMessage; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; @@ -290,6 +293,10 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte @GridToStringExclude private GridDhtPartitionsExchangeFuture mergedWith; + /** Validator for partition states. */ + @GridToStringExclude + private final GridDhtPartitionsStateValidator validator; + /** * @param cctx Cache context. * @param busyLock Busy lock. @@ -314,6 +321,7 @@ public GridDhtPartitionsExchangeFuture( this.exchId = exchId; this.exchActions = exchActions; this.affChangeMsg = affChangeMsg; + this.validator = new GridDhtPartitionsStateValidator(cctx); log = cctx.logger(getClass()); exchLog = cctx.logger(EXCHANGE_LOG); @@ -1099,7 +1107,11 @@ private void distributedExchange() throws IgniteCheckedException { // To correctly rebalance when persistence is enabled, it is necessary to reserve history within exchange. partHistReserved = cctx.database().reserveHistoryForExchange(); - waitPartitionRelease(); + // On first phase we wait for finishing all local tx updates, atomic updates and lock releases. + waitPartitionRelease(1); + + // Second phase is needed to wait for finishing all tx updates from primary to backup nodes remaining after first phase. + waitPartitionRelease(2); boolean topChanged = firstDiscoEvt.type() != EVT_DISCOVERY_CUSTOM_EVT || affChangeMsg != null; @@ -1202,9 +1214,17 @@ private void changeWalModeIfNeeded() { * For the exact list of the objects being awaited for see * {@link GridCacheSharedContext#partitionReleaseFuture(AffinityTopologyVersion)} javadoc. * + * @param phase Phase of partition release. + * * @throws IgniteCheckedException If failed. */ - private void waitPartitionRelease() throws IgniteCheckedException { + private void waitPartitionRelease(int phase) throws IgniteCheckedException { + Latch releaseLatch = null; + + // Wait for other nodes only on first phase. + if (phase == 1) + releaseLatch = cctx.exchange().latch().getOrCreate("exchange", initialVersion()); + IgniteInternalFuture partReleaseFut = cctx.partitionReleaseFuture(initialVersion()); // Assign to class variable so it will be included into toString() method. @@ -1238,6 +1258,11 @@ private void waitPartitionRelease() throws IgniteCheckedException { nextDumpTime = U.currentTimeMillis() + nextDumpTimeout(dumpCnt++, futTimeout); } } + catch (IgniteCheckedException e) { + U.warn(log,"Unable to await partitions release future", e); + + throw e; + } } long waitEnd = U.currentTimeMillis(); @@ -1290,6 +1315,35 @@ private void waitPartitionRelease() throws IgniteCheckedException { } } } + + if (releaseLatch == null) + return; + + releaseLatch.countDown(); + + if (!localJoinExchange()) { + try { + while (true) { + try { + releaseLatch.await(futTimeout, TimeUnit.MILLISECONDS); + + if (log.isInfoEnabled()) + log.info("Finished waiting for partitions release latch: " + releaseLatch); + + break; + } + catch (IgniteFutureTimeoutCheckedException ignored) { + U.warn(log, "Unable to await partitions release latch within timeout: " + releaseLatch); + + // Try to resend ack. + releaseLatch.countDown(); + } + } + } + catch (IgniteCheckedException e) { + U.warn(log, "Stop waiting for partitions release latch: " + e.getMessage()); + } + } } /** @@ -2499,6 +2553,8 @@ private void finishExchangeOnCoordinator(@Nullable Collection sndRe } } + validatePartitionsState(); + if (firstDiscoEvt.type() == EVT_DISCOVERY_CUSTOM_EVT) { assert firstDiscoEvt instanceof DiscoveryCustomEvent; @@ -2682,6 +2738,42 @@ else if (forceAffReassignment) } } + /** + * Validates that partition update counters and cache sizes for all caches are consistent. + */ + private void validatePartitionsState() { + for (Map.Entry e : cctx.affinity().cacheGroups().entrySet()) { + CacheGroupDescriptor grpDesc = e.getValue(); + if (grpDesc.config().getCacheMode() == CacheMode.LOCAL) + continue; + + int grpId = e.getKey(); + + CacheGroupContext grpCtx = cctx.cache().cacheGroup(grpId); + + GridDhtPartitionTopology top = grpCtx != null ? + grpCtx.topology() : + cctx.exchange().clientTopology(grpId, events().discoveryCache()); + + // Do not validate read or write through caches or caches with disabled rebalance. + if (grpCtx == null + || grpCtx.config().isReadThrough() + || grpCtx.config().isWriteThrough() + || grpCtx.config().getCacheStoreFactory() != null + || grpCtx.config().getRebalanceDelay() != -1 + || grpCtx.config().getRebalanceMode() == CacheRebalanceMode.NONE) + continue; + + try { + validator.validatePartitionCountersAndSizes(this, top, msgs); + } + catch (IgniteCheckedException ex) { + log.warning("Partition states validation was failed for cache " + grpDesc.cacheOrGroupName(), ex); + // TODO: Handle such errors https://issues.apache.org/jira/browse/IGNITE-7833 + } + } + } + /** * */ 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 215152d771149..6ebafac22611e 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 @@ -17,9 +17,9 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.preloader; -import java.util.Collection; import java.io.Externalizable; import java.nio.ByteBuffer; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -67,6 +67,14 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes /** Serialized partitions counters. */ private byte[] partCntrsBytes; + /** Partitions sizes. */ + @GridToStringInclude + @GridDirectTransient + private Map> partSizes; + + /** Serialized partitions counters. */ + private byte[] partSizesBytes; + /** Partitions history reservation counters. */ @GridToStringInclude @GridDirectTransient @@ -219,6 +227,35 @@ public CachePartitionPartialCountersMap partitionUpdateCounters(int grpId, int p return CachePartitionPartialCountersMap.fromCountersMap(map, partsCnt); } + /** + * Adds partition sizes map for specified {@code grpId} to the current message. + * + * @param grpId Group id. + * @param partSizesMap Partition sizes map. + */ + public void addPartitionSizes(int grpId, Map partSizesMap) { + if (partSizesMap.isEmpty()) + return; + + if (partSizes == null) + partSizes = new HashMap<>(); + + partSizes.put(grpId, partSizesMap); + } + + /** + * Returns partition sizes map for specified {@code grpId}. + * + * @param grpId Group id. + * @return Partition sizes map (partId, partSize). + */ + public Map partitionSizes(int grpId) { + if (partSizes == null) + return Collections.emptyMap(); + + return partSizes.getOrDefault(grpId, Collections.emptyMap()); + } + /** * @param grpId Cache group ID. * @param cntrMap Partition history counters. @@ -287,12 +324,14 @@ public void setError(Exception ex) { boolean marshal = (parts != null && partsBytes == null) || (partCntrs != null && partCntrsBytes == null) || (partHistCntrs != null && partHistCntrsBytes == null) || + (partSizes != null && partSizesBytes == null) || (err != null && errBytes == null); if (marshal) { byte[] partsBytes0 = null; byte[] partCntrsBytes0 = null; byte[] partHistCntrsBytes0 = null; + byte[] partSizesBytes0 = null; byte[] errBytes0 = null; if (parts != null && partsBytes == null) @@ -304,6 +343,9 @@ public void setError(Exception ex) { if (partHistCntrs != null && partHistCntrsBytes == null) partHistCntrsBytes0 = U.marshal(ctx, partHistCntrs); + if (partSizes != null && partSizesBytes == null) + partSizesBytes0 = U.marshal(ctx, partSizes); + if (err != null && errBytes == null) errBytes0 = U.marshal(ctx, err); @@ -314,11 +356,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[] exBytesZip = U.zip(errBytes0); partsBytes0 = partsBytesZip; partCntrsBytes0 = partCntrsBytesZip; partHistCntrsBytes0 = partHistCntrsBytesZip; + partSizesBytes0 = partSizesBytesZip; errBytes0 = exBytesZip; compressed(true); @@ -331,6 +375,7 @@ public void setError(Exception ex) { partsBytes = partsBytes0; partCntrsBytes = partCntrsBytes0; partHistCntrsBytes = partHistCntrsBytes0; + partSizesBytes = partSizesBytes0; errBytes = errBytes0; } } @@ -360,6 +405,13 @@ public void setError(Exception ex) { partHistCntrs = U.unmarshal(ctx, partHistCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); } + if (partSizesBytes != null && partSizes == null) { + if (compressed()) + partSizes = U.unmarshalZip(ctx.marshaller(), partSizesBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + else + partSizes = U.unmarshal(ctx, partSizesBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + } + if (errBytes != null && err == null) { if (compressed()) err = U.unmarshalZip(ctx.marshaller(), errBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); @@ -451,6 +503,11 @@ public void setError(Exception ex) { writer.incrementState(); + case 13: + if (!writer.writeByteArray("partsSizesBytes", partSizesBytes)) + return false; + + writer.incrementState(); } return true; @@ -531,6 +588,13 @@ public void setError(Exception ex) { reader.incrementState(); + case 13: + partSizesBytes = reader.readByteArray("partsSizesBytes"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); } return reader.afterMessageRead(GridDhtPartitionsSingleMessage.class); @@ -543,7 +607,7 @@ public void setError(Exception ex) { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 13; + return 14; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/InitNewCoordinatorFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/InitNewCoordinatorFuture.java index 596fa8c4f8976..42a9ba6891955 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/InitNewCoordinatorFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/InitNewCoordinatorFuture.java @@ -235,7 +235,7 @@ public void onMessage(ClusterNode node, GridDhtPartitionsSingleMessage msg) { if (awaited.remove(node.id())) { GridDhtPartitionsFullMessage fullMsg0 = msg.finishMessage(); - if (fullMsg0 != null) { + if (fullMsg0 != null && fullMsg0.resultTopologyVersion() != null) { assert fullMsg == null || fullMsg.resultTopologyVersion().equals(fullMsg0.resultTopologyVersion()); fullMsg = fullMsg0; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java new file mode 100644 index 0000000000000..c205cb14e40d9 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java @@ -0,0 +1,695 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.latch; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.Collectors; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.GridTopic; +import org.apache.ignite.internal.managers.communication.GridIoManager; +import org.apache.ignite.internal.managers.communication.GridIoPolicy; +import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.util.GridConcurrentHashSet; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteProductVersion; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; +import static org.apache.ignite.events.EventType.EVT_NODE_LEFT; + +/** + * Class is responsible to create and manage instances of distributed latches {@link Latch}. + */ +public class ExchangeLatchManager { + /** Version since latch management is available. */ + private static final IgniteProductVersion VERSION_SINCE = IgniteProductVersion.fromString("2.5.0"); + + /** Logger. */ + private final IgniteLogger log; + + /** Context. */ + private final GridKernalContext ctx; + + /** Discovery manager. */ + private final GridDiscoveryManager discovery; + + /** IO manager. */ + private final GridIoManager io; + + /** Current coordinator. */ + private volatile ClusterNode coordinator; + + /** Pending acks collection. */ + private final ConcurrentMap, Set> pendingAcks = new ConcurrentHashMap<>(); + + /** Server latches collection. */ + private final ConcurrentMap, ServerLatch> serverLatches = new ConcurrentHashMap<>(); + + /** Client latches collection. */ + private final ConcurrentMap, ClientLatch> clientLatches = new ConcurrentHashMap<>(); + + /** Lock. */ + private final ReentrantLock lock = new ReentrantLock(); + + /** + * Constructor. + * + * @param ctx Kernal context. + */ + public ExchangeLatchManager(GridKernalContext ctx) { + this.ctx = ctx; + this.log = ctx.log(getClass()); + this.discovery = ctx.discovery(); + this.io = ctx.io(); + + if (!ctx.clientNode()) { + ctx.io().addMessageListener(GridTopic.TOPIC_EXCHANGE, (nodeId, msg, plc) -> { + if (msg instanceof LatchAckMessage) { + processAck(nodeId, (LatchAckMessage) msg); + } + }); + + // First coordinator initialization. + ctx.discovery().localJoinFuture().listen(f -> { + this.coordinator = getLatchCoordinator(AffinityTopologyVersion.NONE); + }); + + ctx.event().addDiscoveryEventListener((e, cache) -> { + assert e != null; + assert e.type() == EVT_NODE_LEFT || e.type() == EVT_NODE_FAILED : this; + + // Do not process from discovery thread. + ctx.closure().runLocalSafe(() -> processNodeLeft(e.eventNode())); + }, EVT_NODE_LEFT, EVT_NODE_FAILED); + } + } + + /** + * Creates server latch with given {@code id} and {@code topVer}. + * Adds corresponding pending acks to it. + * + * @param id Latch id. + * @param topVer Latch topology version. + * @param participants Participant nodes. + * @return Server latch instance. + */ + private Latch createServerLatch(String id, AffinityTopologyVersion topVer, Collection participants) { + final T2 latchId = new T2<>(id, topVer); + + if (serverLatches.containsKey(latchId)) + return serverLatches.get(latchId); + + ServerLatch latch = new ServerLatch(id, topVer, participants); + + serverLatches.put(latchId, latch); + + if (log.isDebugEnabled()) + log.debug("Server latch is created [latch=" + latchId + ", participantsSize=" + participants.size() + "]"); + + if (pendingAcks.containsKey(latchId)) { + Set acks = pendingAcks.get(latchId); + + for (UUID node : acks) + if (latch.hasParticipant(node) && !latch.hasAck(node)) + latch.ack(node); + + pendingAcks.remove(latchId); + } + + if (latch.isCompleted()) + serverLatches.remove(latchId); + + return latch; + } + + /** + * Creates client latch. + * If there is final ack corresponds to given {@code id} and {@code topVer}, latch will be completed immediately. + * + * @param id Latch id. + * @param topVer Latch topology version. + * @param coordinator Coordinator node. + * @param participants Participant nodes. + * @return Client latch instance. + */ + private Latch createClientLatch(String id, AffinityTopologyVersion topVer, ClusterNode coordinator, Collection participants) { + final T2 latchId = new T2<>(id, topVer); + + if (clientLatches.containsKey(latchId)) + return clientLatches.get(latchId); + + ClientLatch latch = new ClientLatch(id, topVer, coordinator, participants); + + if (log.isDebugEnabled()) + log.debug("Client latch is created [latch=" + latchId + + ", crd=" + coordinator + + ", participantsSize=" + participants.size() + "]"); + + // There is final ack for created latch. + if (pendingAcks.containsKey(latchId)) { + latch.complete(); + pendingAcks.remove(latchId); + } + else + clientLatches.put(latchId, latch); + + return latch; + } + + /** + * Creates new latch with specified {@code id} and {@code topVer} or returns existing latch. + * + * Participants of latch are calculated from given {@code topVer} as alive server nodes. + * If local node is coordinator {@code ServerLatch} instance will be created, otherwise {@code ClientLatch} instance. + * + * @param id Latch id. + * @param topVer Latch topology version. + * @return Latch instance. + */ + public Latch getOrCreate(String id, AffinityTopologyVersion topVer) { + lock.lock(); + + try { + ClusterNode coordinator = getLatchCoordinator(topVer); + + if (coordinator == null) { + ClientLatch latch = new ClientLatch(id, AffinityTopologyVersion.NONE, null, Collections.emptyList()); + latch.complete(); + + return latch; + } + + Collection participants = getLatchParticipants(topVer); + + return coordinator.isLocal() + ? createServerLatch(id, topVer, participants) + : createClientLatch(id, topVer, coordinator, participants); + } + finally { + lock.unlock(); + } + } + + /** + * @param topVer Latch topology version. + * @return Collection of alive server nodes with latch functionality. + */ + private Collection getLatchParticipants(AffinityTopologyVersion topVer) { + Collection aliveNodes = topVer == AffinityTopologyVersion.NONE + ? discovery.aliveServerNodes() + : discovery.discoCache(topVer).aliveServerNodes(); + + return aliveNodes + .stream() + .filter(node -> node.version().compareTo(VERSION_SINCE) >= 0) + .collect(Collectors.toList()); + } + + /** + * @param topVer Latch topology version. + * @return Oldest alive server node with latch functionality. + */ + @Nullable private ClusterNode getLatchCoordinator(AffinityTopologyVersion topVer) { + Collection aliveNodes = topVer == AffinityTopologyVersion.NONE + ? discovery.aliveServerNodes() + : discovery.discoCache(topVer).aliveServerNodes(); + + return aliveNodes + .stream() + .filter(node -> node.version().compareTo(VERSION_SINCE) >= 0) + .findFirst() + .orElse(null); + } + + /** + * Processes ack message from given {@code from} node. + * + * Completes client latch in case of final ack message. + * + * If no latch is associated with message, ack is placed to {@link #pendingAcks} set. + * + * @param from Node sent ack. + * @param message Ack message. + */ + private void processAck(UUID from, LatchAckMessage message) { + lock.lock(); + + try { + ClusterNode coordinator = getLatchCoordinator(AffinityTopologyVersion.NONE); + + if (coordinator == null) + return; + + T2 latchId = new T2<>(message.latchId(), message.topVer()); + + if (message.isFinal()) { + if (log.isDebugEnabled()) + log.debug("Process final ack [latch=" + latchId + ", from=" + from + "]"); + + if (clientLatches.containsKey(latchId)) { + ClientLatch latch = clientLatches.remove(latchId); + latch.complete(); + } + else if (!coordinator.isLocal()) { + pendingAcks.computeIfAbsent(latchId, (id) -> new GridConcurrentHashSet<>()); + pendingAcks.get(latchId).add(from); + } + } else { + if (log.isDebugEnabled()) + log.debug("Process ack [latch=" + latchId + ", from=" + from + "]"); + + if (serverLatches.containsKey(latchId)) { + ServerLatch latch = serverLatches.get(latchId); + + if (latch.hasParticipant(from) && !latch.hasAck(from)) { + latch.ack(from); + + if (latch.isCompleted()) + serverLatches.remove(latchId); + } + } + else { + pendingAcks.computeIfAbsent(latchId, (id) -> new GridConcurrentHashSet<>()); + pendingAcks.get(latchId).add(from); + } + } + } + finally { + lock.unlock(); + } + } + + /** + * Changes coordinator to current local node. + * Restores all server latches from pending acks and own client latches. + */ + private void becomeNewCoordinator() { + if (log.isInfoEnabled()) + log.info("Become new coordinator " + coordinator.id()); + + List> latchesToRestore = new ArrayList<>(); + latchesToRestore.addAll(pendingAcks.keySet()); + latchesToRestore.addAll(clientLatches.keySet()); + + for (T2 latchId : latchesToRestore) { + String id = latchId.get1(); + AffinityTopologyVersion topVer = latchId.get2(); + Collection participants = getLatchParticipants(topVer); + + if (!participants.isEmpty()) + createServerLatch(id, topVer, participants); + } + } + + /** + * Handles node left discovery event. + * + * Summary: + * Removes pending acks corresponds to the left node. + * Adds fake acknowledgements to server latches where such node was participant. + * Changes client latches coordinator to oldest available server node where such node was coordinator. + * Detects coordinator change. + * + * @param left Left node. + */ + private void processNodeLeft(ClusterNode left) { + assert this.coordinator != null : "Coordinator is not initialized"; + + lock.lock(); + + try { + if (log.isDebugEnabled()) + log.debug("Process node left " + left.id()); + + ClusterNode coordinator = getLatchCoordinator(AffinityTopologyVersion.NONE); + + if (coordinator == null) + return; + + // Clear pending acks. + for (Map.Entry, Set> ackEntry : pendingAcks.entrySet()) + if (ackEntry.getValue().contains(left.id())) + pendingAcks.get(ackEntry.getKey()).remove(left.id()); + + // Change coordinator for client latches. + for (Map.Entry, ClientLatch> latchEntry : clientLatches.entrySet()) { + ClientLatch latch = latchEntry.getValue(); + if (latch.hasCoordinator(left.id())) { + // Change coordinator for latch and re-send ack if necessary. + if (latch.hasParticipant(coordinator.id())) + latch.newCoordinator(coordinator); + else { + /* If new coordinator is not able to take control on the latch, + it means that all other latch participants are left from topology + and there is no reason to track such latch. */ + AffinityTopologyVersion topVer = latchEntry.getKey().get2(); + + assert getLatchParticipants(topVer).isEmpty(); + + latch.complete(new IgniteCheckedException("All latch participants are left from topology.")); + clientLatches.remove(latchEntry.getKey()); + } + } + } + + // Add acknowledgements from left node. + for (Map.Entry, ServerLatch> latchEntry : serverLatches.entrySet()) { + ServerLatch latch = latchEntry.getValue(); + + if (latch.hasParticipant(left.id()) && !latch.hasAck(left.id())) { + if (log.isDebugEnabled()) + log.debug("Process node left [latch=" + latchEntry.getKey() + ", left=" + left.id() + "]"); + + latch.ack(left.id()); + + if (latch.isCompleted()) + serverLatches.remove(latchEntry.getKey()); + } + } + + // Coordinator is changed. + if (coordinator.isLocal() && this.coordinator.id() != coordinator.id()) { + this.coordinator = coordinator; + + becomeNewCoordinator(); + } + } + finally { + lock.unlock(); + } + } + + /** + * Latch creating on coordinator node. + * Latch collects acks from participants: non-coordinator nodes and current local node. + * Latch completes when all acks from all participants are received. + * + * After latch completion final ack is sent to all participants. + */ + class ServerLatch extends CompletableLatch { + /** Number of latch permits. This is needed to track number of countDown invocations. */ + private final AtomicInteger permits; + + /** Set of received acks. */ + private final Set acks = new GridConcurrentHashSet<>(); + + /** + * Constructor. + * + * @param id Latch id. + * @param topVer Latch topology version. + * @param participants Participant nodes. + */ + ServerLatch(String id, AffinityTopologyVersion topVer, Collection participants) { + super(id, topVer, participants); + this.permits = new AtomicInteger(participants.size()); + + // Send final acks when latch is completed. + this.complete.listen(f -> { + for (ClusterNode node : participants) { + try { + if (discovery.alive(node)) { + io.sendToGridTopic(node, GridTopic.TOPIC_EXCHANGE, new LatchAckMessage(id, topVer, true), GridIoPolicy.SYSTEM_POOL); + + if (log.isDebugEnabled()) + log.debug("Final ack is ackSent [latch=" + latchId() + ", to=" + node.id() + "]"); + } + } catch (IgniteCheckedException e) { + if (log.isDebugEnabled()) + log.debug("Unable to send final ack [latch=" + latchId() + ", to=" + node.id() + "]"); + } + } + }); + } + + /** + * Checks if latch has ack from given node. + * + * @param from Node. + * @return {@code true} if latch has ack from given node. + */ + private boolean hasAck(UUID from) { + return acks.contains(from); + } + + /** + * Receives ack from given node. + * Count downs latch if ack was not already processed. + * + * @param from Node. + */ + private void ack(UUID from) { + if (log.isDebugEnabled()) + log.debug("Ack is accepted [latch=" + latchId() + ", from=" + from + "]"); + + countDown0(from); + } + + /** + * Count down latch from ack of given node. + * Completes latch if all acks are received. + * + * @param node Node. + */ + private void countDown0(UUID node) { + if (isCompleted() || acks.contains(node)) + return; + + acks.add(node); + + int remaining = permits.decrementAndGet(); + + if (log.isDebugEnabled()) + log.debug("Count down + [latch=" + latchId() + ", remaining=" + remaining + "]"); + + if (remaining == 0) + complete(); + } + + /** {@inheritDoc} */ + @Override public void countDown() { + countDown0(ctx.localNodeId()); + } + + /** {@inheritDoc} */ + @Override public String toString() { + Set pendingAcks = participants.stream().filter(ack -> !acks.contains(ack)).collect(Collectors.toSet()); + + return S.toString(ServerLatch.class, this, + "pendingAcks", pendingAcks, + "super", super.toString()); + } + } + + /** + * Latch creating on non-coordinator node. + * Latch completes when final ack from coordinator is received. + */ + class ClientLatch extends CompletableLatch { + /** Latch coordinator node. Can be changed if coordinator is left from topology. */ + private volatile ClusterNode coordinator; + + /** Flag indicates that ack is sent to coordinator. */ + private boolean ackSent; + + /** + * Constructor. + * + * @param id Latch id. + * @param topVer Latch topology version. + * @param coordinator Coordinator node. + * @param participants Participant nodes. + */ + ClientLatch(String id, AffinityTopologyVersion topVer, ClusterNode coordinator, Collection participants) { + super(id, topVer, participants); + + this.coordinator = coordinator; + } + + /** + * Checks if latch coordinator is given {@code node}. + * + * @param node Node. + * @return {@code true} if latch coordinator is given node. + */ + private boolean hasCoordinator(UUID node) { + return coordinator.id().equals(node); + } + + /** + * Changes coordinator of latch and resends ack to new coordinator if needed. + * + * @param coordinator New coordinator. + */ + private void newCoordinator(ClusterNode coordinator) { + if (log.isDebugEnabled()) + log.debug("Coordinator is changed [latch=" + latchId() + ", crd=" + coordinator.id() + "]"); + + synchronized (this) { + this.coordinator = coordinator; + + // Resend ack to new coordinator. + if (ackSent) + sendAck(); + } + } + + /** + * Sends ack to coordinator node. + * There is ack deduplication on coordinator. So it's fine to send same ack twice. + */ + private void sendAck() { + try { + ackSent = true; + + io.sendToGridTopic(coordinator, GridTopic.TOPIC_EXCHANGE, new LatchAckMessage(id, topVer, false), GridIoPolicy.SYSTEM_POOL); + + if (log.isDebugEnabled()) + log.debug("Ack is ackSent + [latch=" + latchId() + ", to=" + coordinator.id() + "]"); + } catch (IgniteCheckedException e) { + // Coordinator is unreachable. On coodinator node left discovery event ack will be resent. + if (log.isDebugEnabled()) + log.debug("Unable to send ack [latch=" + latchId() + ", to=" + coordinator.id() + "]: " + e.getMessage()); + } + } + + /** {@inheritDoc} */ + @Override public void countDown() { + if (isCompleted()) + return; + + // Synchronize in case of changed coordinator. + synchronized (this) { + sendAck(); + } + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ClientLatch.class, this, + "super", super.toString()); + } + } + + /** + * Base latch functionality with implemented complete / await logic. + */ + private abstract static class CompletableLatch implements Latch { + /** Latch id. */ + @GridToStringInclude + protected final String id; + + /** Latch topology version. */ + @GridToStringInclude + protected final AffinityTopologyVersion topVer; + + /** Latch node participants. Only participant nodes are able to change state of latch. */ + @GridToStringExclude + protected final Set participants; + + /** Future indicates that latch is completed. */ + @GridToStringExclude + protected final GridFutureAdapter complete = new GridFutureAdapter<>(); + + /** + * Constructor. + * + * @param id Latch id. + * @param topVer Latch topology version. + * @param participants Participant nodes. + */ + CompletableLatch(String id, AffinityTopologyVersion topVer, Collection participants) { + this.id = id; + this.topVer = topVer; + this.participants = participants.stream().map(ClusterNode::id).collect(Collectors.toSet()); + } + + /** {@inheritDoc} */ + @Override public void await() throws IgniteCheckedException { + complete.get(); + } + + /** {@inheritDoc} */ + @Override public void await(long timeout, TimeUnit timeUnit) throws IgniteCheckedException { + complete.get(timeout, timeUnit); + } + + /** + * Checks if latch participants contain given {@code node}. + * + * @param node Node. + * @return {@code true} if latch participants contain given node. + */ + boolean hasParticipant(UUID node) { + return participants.contains(node); + } + + /** + * @return {@code true} if latch is completed. + */ + boolean isCompleted() { + return complete.isDone(); + } + + /** + * Completes current latch. + */ + void complete() { + complete.onDone(); + } + + /** + * Completes current latch with given {@code error}. + * + * @param error Error. + */ + void complete(Throwable error) { + complete.onDone(error); + } + + /** + * @return Full latch id. + */ + String latchId() { + return id + "-" + topVer; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(CompletableLatch.class, this); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/Latch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/Latch.java new file mode 100644 index 0000000000000..9704c2eacad7e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/Latch.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.distributed.dht.preloader.latch; + +import java.util.concurrent.TimeUnit; +import org.apache.ignite.IgniteCheckedException; + +/** + * Simple distributed count down latch interface. + * Latch supports count down and await logic. + * Latch functionality is not relied on caches and has own state management {@link ExchangeLatchManager}. + */ +public interface Latch { + /** + * Decrements count on current latch. + * Release all latch waiters on all nodes if count reaches zero. + * + * This is idempotent operation. Invoking this method twice or more on the same node doesn't have any effect. + */ + void countDown(); + + /** + * Awaits current latch completion. + * + * @throws IgniteCheckedException If await is failed. + */ + void await() throws IgniteCheckedException; + + /** + * Awaits current latch completion with specified timeout. + * + * @param timeout Timeout value. + * @param timeUnit Timeout time unit. + * @throws IgniteCheckedException If await is failed. + */ + void await(long timeout, TimeUnit timeUnit) throws IgniteCheckedException; +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/LatchAckMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/LatchAckMessage.java new file mode 100644 index 0000000000000..bad1b6137bac5 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/LatchAckMessage.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.preloader.latch; + +import java.nio.ByteBuffer; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; + +/** + * Message is used to send acks for {@link Latch} instances management. + */ +public class LatchAckMessage implements Message { + /** */ + private static final long serialVersionUID = 0L; + + /** Latch id. */ + private String latchId; + + /** Latch topology version. */ + private AffinityTopologyVersion topVer; + + /** Flag indicates that ack is final. */ + private boolean isFinal; + + /** + * Constructor. + * + * @param latchId Latch id. + * @param topVer Latch topology version. + * @param isFinal Final acknowledgement flag. + */ + public LatchAckMessage(String latchId, AffinityTopologyVersion topVer, boolean isFinal) { + this.latchId = latchId; + this.topVer = topVer; + this.isFinal = isFinal; + } + + /** + * Empty constructor for marshalling purposes. + */ + public LatchAckMessage() { + } + + /** + * @return Latch id. + */ + public String latchId() { + return latchId; + } + + /** + * @return Latch topology version. + */ + public AffinityTopologyVersion topVer() { + return topVer; + } + + /** + * @return {@code} if ack is final. + */ + public boolean isFinal() { + return isFinal; + } + + /** {@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("isFinal", isFinal)) + return false; + + writer.incrementState(); + + case 1: + if (!writer.writeString("latchId", latchId)) + return false; + + writer.incrementState(); + + case 2: + 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; + + switch (reader.state()) { + case 0: + isFinal = reader.readBoolean("isFinal"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 1: + latchId = reader.readString("latchId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 2: + topVer = reader.readMessage("topVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + } + + return reader.afterMessageRead(LatchAckMessage.class); + } + + /** {@inheritDoc} */ + @Override public short directType() { + return 135; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 3; + } + + /** {@inheritDoc} */ + @Override public void onAckReceived() { + // No-op. + } +} 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 77856055a0ca0..33f84f030db71 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 @@ -3525,6 +3525,11 @@ public IgniteInternalFuture commitAsyncLocal() { U.error(log, "Failed to prepare transaction: " + this, e); } + catch (Throwable t) { + fut.onDone(t); + + throw t; + } if (err != null) fut.rollbackOnError(err); @@ -3544,6 +3549,11 @@ public IgniteInternalFuture commitAsyncLocal() { U.error(log, "Failed to prepare transaction: " + this, e); } + catch (Throwable t) { + fut.onDone(t); + + throw t; + } if (err != null) fut.rollbackOnError(err); 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 5cfd92d269051..68ec83db1b4b6 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 @@ -189,7 +189,7 @@ private boolean saveStoreMetadata( freeList.saveMetadata(); long updCntr = store.updateCounter(); - int size = store.fullSize(); + long size = store.fullSize(); long rmvId = globalRemoveId().get(); PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory(); @@ -318,7 +318,7 @@ else if (state == MOVING || state == RENTING) { partMetaId, updCntr, rmvId, - size, + (int)size, // TODO: Partition size may be long cntrsPageId, state == null ? -1 : (byte)state.ordinal(), pageCnt @@ -549,7 +549,7 @@ private static boolean addPartition( final int grpId, final int partId, final int currAllocatedPageCnt, - final int partSize + final long partSize ) { if (part != null) { boolean reserved = part.reserve(); @@ -1301,7 +1301,7 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { } /** {@inheritDoc} */ - @Override public int fullSize() { + @Override public long fullSize() { try { CacheDataStore delegate0 = init0(true); @@ -1313,7 +1313,7 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { } /** {@inheritDoc} */ - @Override public int cacheSize(int cacheId) { + @Override public long cacheSize(int cacheId) { try { CacheDataStore delegate0 = init0(true); 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 9bfaaf3b59f96..945ef486fbe5a 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 @@ -490,7 +490,7 @@ protected void uncommit(boolean nodeStopping) { @Override public AffinityTopologyVersion topologyVersion() { AffinityTopologyVersion res = topVer; - if (res.equals(AffinityTopologyVersion.NONE)) { + if (res == null || res.equals(AffinityTopologyVersion.NONE)) { if (system()) { AffinityTopologyVersion topVer = cctx.tm().lockedTopologyVersion(Thread.currentThread().getId(), this); 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 fbdeca10eb46f..9fb87770df64d 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 @@ -545,10 +545,10 @@ public GridNearTxLocal newTx( * @param topVer Topology version. * @return Future that will be completed when all ongoing transactions are finished. */ - public IgniteInternalFuture finishTxs(AffinityTopologyVersion topVer) { + public IgniteInternalFuture finishLocalTxs(AffinityTopologyVersion topVer) { GridCompoundFuture res = new CacheObjectsReleaseFuture<>( - "Tx", + "LocalTx", topVer, new IgniteReducer() { @Override public boolean collect(IgniteInternalTx e) { @@ -561,8 +561,9 @@ public IgniteInternalFuture finishTxs(AffinityTopologyVersion topVer) { }); for (IgniteInternalTx tx : txs()) { - if (needWaitTransaction(tx, topVer)) + if (needWaitTransaction(tx, topVer)) { res.add(tx.finishFuture()); + } } res.markInitialized(); @@ -570,6 +571,29 @@ public IgniteInternalFuture finishTxs(AffinityTopologyVersion topVer) { return res; } + /** + * Creates a future that will wait for finishing all tx updates on backups after all local transactions are finished. + * + * NOTE: + * As we send finish request to backup nodes after transaction successfully completed on primary node + * it's important to ensure that all updates from primary to backup are finished or at least remote transaction has created on backup node. + * + * @param finishLocalTxsFuture Local transactions finish future. + * @param topVer Topology version. + * @return Future that will be completed when all ongoing transactions are finished. + */ + public IgniteInternalFuture finishAllTxs(IgniteInternalFuture finishLocalTxsFuture, AffinityTopologyVersion topVer) { + final GridCompoundFuture finishAllTxsFuture = new CacheObjectsReleaseFuture("AllTx", topVer); + + // After finishing all local updates, wait for finishing all tx updates on backups. + finishLocalTxsFuture.listen(future -> { + finishAllTxsFuture.add(cctx.mvcc().finishRemoteTxs(topVer)); + finishAllTxsFuture.markInitialized(); + }); + + return finishAllTxsFuture; + } + /** * @param tx Transaction. * @param topVer Exchange version. @@ -1834,12 +1858,12 @@ public IgniteInternalFuture txCommitted(GridCacheVersion xidVer) { * @return Finish future for related remote transactions. */ @SuppressWarnings("unchecked") - public IgniteInternalFuture remoteTxFinishFuture(GridCacheVersion nearVer) { - GridCompoundFuture fut = new GridCompoundFuture<>(); + public IgniteInternalFuture remoteTxFinishFuture(GridCacheVersion nearVer) { + GridCompoundFuture fut = new GridCompoundFuture<>(); for (final IgniteInternalTx tx : txs()) { if (!tx.local() && nearVer.equals(tx.nearXidVersion())) - fut.add((IgniteInternalFuture) tx.finishFuture()); + fut.add(tx.finishFuture()); } fut.markInitialized(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDhtLocalPartitionAfterRemoveSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDhtLocalPartitionAfterRemoveSelfTest.java index 726365624570d..702b188bd4005 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDhtLocalPartitionAfterRemoveSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDhtLocalPartitionAfterRemoveSelfTest.java @@ -76,7 +76,7 @@ public void testMemoryUsage() throws Exception { cache = grid(g).cache(DEFAULT_CACHE_NAME); for (GridDhtLocalPartition p : dht(cache).topology().localPartitions()) { - int size = p.dataStore().fullSize(); + long size = p.dataStore().fullSize(); assertTrue("Unexpected size: " + size, size <= 32); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheGroupsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheGroupsTest.java index 468bbc8bbc009..6c570d744ff0e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheGroupsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheGroupsTest.java @@ -86,6 +86,7 @@ import org.apache.ignite.internal.util.lang.GridPlainCallable; import org.apache.ignite.internal.util.lang.gridfunc.ContainsPredicate; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.PA; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.CU; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteExchangeLatchManagerCoordinatorFailTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteExchangeLatchManagerCoordinatorFailTest.java new file mode 100644 index 0000000000000..52cd0338a87d0 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteExchangeLatchManagerCoordinatorFailTest.java @@ -0,0 +1,244 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ignite.internal.processors.cache.datastructures; + +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; +import com.google.common.collect.Lists; +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.distributed.dht.preloader.latch.Latch; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.ExchangeLatchManager; +import org.apache.ignite.internal.util.future.GridCompoundFuture; +import org.apache.ignite.lang.IgniteBiClosure; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.Assert; + +/** + * Tests for {@link ExchangeLatchManager} functionality when latch coordinator is failed. + */ +public class IgniteExchangeLatchManagerCoordinatorFailTest extends GridCommonAbstractTest { + /** */ + private static final String LATCH_NAME = "test"; + + /** 5 nodes. */ + private final AffinityTopologyVersion latchTopVer = new AffinityTopologyVersion(5, 0); + + /** Wait before latch creation. */ + private final IgniteBiClosure beforeCreate = (mgr, syncLatch) -> { + try { + syncLatch.countDown(); + syncLatch.await(); + + Latch distributedLatch = mgr.getOrCreate(LATCH_NAME, latchTopVer); + + distributedLatch.countDown(); + + distributedLatch.await(); + } catch (Exception e) { + log.error("Unexpected exception", e); + + return false; + } + + return true; + }; + + /** Wait before latch count down. */ + private final IgniteBiClosure beforeCountDown = (mgr, syncLatch) -> { + try { + Latch distributedLatch = mgr.getOrCreate(LATCH_NAME, latchTopVer); + + syncLatch.countDown(); + syncLatch.await(); + + distributedLatch.countDown(); + + distributedLatch.await(); + } catch (Exception e) { + log.error("Unexpected exception ", e); + + return false; + } + + return true; + }; + + /** Wait after all operations are successful. */ + private final IgniteBiClosure all = (mgr, syncLatch) -> { + try { + Latch distributedLatch = mgr.getOrCreate(LATCH_NAME, latchTopVer); + + distributedLatch.countDown(); + + syncLatch.countDown(); + + distributedLatch.await(); + + syncLatch.await(); + } catch (Exception e) { + log.error("Unexpected exception ", e); + + return false; + } + + return true; + }; + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * Test scenarios description: + * + * We have existing coordinator and 4 other nodes. + * Each node do following operations: + * 1) Create latch + * 2) Countdown latch + * 3) Await latch + * + * While nodes do the operations we shutdown coordinator and next oldest node become new coordinator. + * We should check that new coordinator properly restored latch and all nodes finished latch completion successfully after that. + * + * Each node before coordinator shutdown can be in 3 different states: + * + * State {@link #beforeCreate} - Node didn't create latch yet. + * State {@link #beforeCountDown} - Node created latch but didn't count down it yet. + * State {@link #all} - Node created latch and count downed it. + * + * We should check important cases when future coordinator is in one of these states, and other 3 nodes have 3 different states. + */ + + /** + * Scenario 1: + * + * Node 1 state -> {@link #beforeCreate} + * Node 2 state -> {@link #beforeCountDown} + * Node 3 state -> {@link #all} + * Node 4 state -> {@link #beforeCreate} + */ + public void testCoordinatorFail1() throws Exception { + List> nodeStates = Lists.newArrayList( + beforeCreate, + beforeCountDown, + all, + beforeCreate + ); + + doTestCoordinatorFail(nodeStates); + } + + /** + * Scenario 2: + * + * Node 1 state -> {@link #beforeCountDown} + * Node 2 state -> {@link #beforeCountDown} + * Node 3 state -> {@link #all} + * Node 4 state -> {@link #beforeCreate} + */ + public void testCoordinatorFail2() throws Exception { + List> nodeStates = Lists.newArrayList( + beforeCountDown, + beforeCountDown, + all, + beforeCreate + ); + + doTestCoordinatorFail(nodeStates); + } + + /** + * Scenario 3: + * + * Node 1 state -> {@link #all} + * Node 2 state -> {@link #beforeCountDown} + * Node 3 state -> {@link #all} + * Node 4 state -> {@link #beforeCreate} + */ + public void testCoordinatorFail3() throws Exception { + List> nodeStates = Lists.newArrayList( + all, + beforeCountDown, + all, + beforeCreate + ); + + doTestCoordinatorFail(nodeStates); + } + + /** + * Test latch coordinator fail with specified scenarios. + * + * @param nodeScenarios Node scenarios. + * @throws Exception If failed. + */ + private void doTestCoordinatorFail(List> nodeScenarios) throws Exception { + IgniteEx crd = (IgniteEx) startGridsMultiThreaded(5); + crd.cluster().active(true); + + // Latch to synchronize node states. + CountDownLatch syncLatch = new CountDownLatch(5); + + GridCompoundFuture finishAllLatches = new GridCompoundFuture(); + + AtomicBoolean hasErrors = new AtomicBoolean(); + + for (int node = 1; node < 5; node++) { + IgniteEx grid = grid(node); + ExchangeLatchManager latchMgr = grid.context().cache().context().exchange().latch(); + final int stateIdx = node - 1; + + IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(() -> { + boolean success = nodeScenarios.get(stateIdx).apply(latchMgr, syncLatch); + if (!success) + hasErrors.set(true); + }, 1, "latch-runner-" + node); + + finishAllLatches.add(fut); + } + + finishAllLatches.markInitialized(); + + // Wait while all nodes reaches their states. + while (syncLatch.getCount() != 1) { + Thread.sleep(10); + + if (hasErrors.get()) + throw new Exception("All nodes should complete latches without errors"); + } + + crd.close(); + + // Resume progress for all nodes. + syncLatch.countDown(); + + // Wait for distributed latch completion. + finishAllLatches.get(5000); + + Assert.assertFalse("All nodes should complete latches without errors", hasErrors.get()); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidationTest.java new file mode 100644 index 0000000000000..63d772a0077c0 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidationTest.java @@ -0,0 +1,316 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; +import java.util.stream.Stream; +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.CacheWriteSynchronizationMode; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +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.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; + +/** + * + */ +public class GridCachePartitionsStateValidationTest extends GridCommonAbstractTest { + /** Cache name. */ + private static final String CACHE_NAME = "cache"; + + /** */ + private boolean clientMode; + + /** {@inheritDoc */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setConsistentId(igniteInstanceName); + + cfg.setCacheConfiguration(new CacheConfiguration(CACHE_NAME) + .setBackups(1) + .setAffinity(new RendezvousAffinityFunction(false, 32)) + ); + + cfg.setCommunicationSpi(new SingleMessageInterceptorCommunicationSpi(2)); + + if (clientMode) + cfg.setClientMode(true); + + return cfg; + } + + /** {@inheritDoc */ + @Override protected void beforeTest() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + clientMode = false; + } + + /** + * Test that partitions state validation works correctly. + * + * @throws Exception If failed. + */ + public void testValidationIfPartitionCountersAreInconsistent() throws Exception { + IgniteEx ignite = (IgniteEx) startGrids(2); + ignite.cluster().active(true); + + awaitPartitionMapExchange(); + + // Modify update counter for some partition. + for (GridDhtLocalPartition partition : ignite.cachex(CACHE_NAME).context().topology().localPartitions()) { + partition.updateCounter(100500L); + break; + } + + // Trigger exchange. + startGrid(2); + + awaitPartitionMapExchange(); + + // Nothing should happen (just log error message) and we're still able to put data to corrupted cache. + ignite.cache(CACHE_NAME).put(0, 0); + + stopAllGrids(); + } + + /** + * Test that all nodes send correct {@link GridDhtPartitionsSingleMessage} with consistent update counters. + * + * @throws Exception If failed. + */ + public void testPartitionCountersConsistencyOnExchange() throws Exception { + IgniteEx ignite = (IgniteEx) startGrids(4); + ignite.cluster().active(true); + + awaitPartitionMapExchange(); + + final String atomicCacheName = "atomic-cache"; + final String txCacheName = "tx-cache"; + + clientMode = true; + + Ignite client = startGrid(4); + + clientMode = false; + + IgniteCache atomicCache = client.getOrCreateCache(new CacheConfiguration<>(atomicCacheName) + .setAtomicityMode(CacheAtomicityMode.ATOMIC) + .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC) + .setBackups(2) + .setAffinity(new RendezvousAffinityFunction(false, 32)) + ); + + IgniteCache txCache = client.getOrCreateCache(new CacheConfiguration<>(txCacheName) + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL) + .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC) + .setBackups(2) + .setAffinity(new RendezvousAffinityFunction(false, 32)) + ); + + for (int it = 0; it < 10; it++) { + SingleMessageInterceptorCommunicationSpi spi = (SingleMessageInterceptorCommunicationSpi) ignite.configuration().getCommunicationSpi(); + spi.clear(); + + // Stop load future. + final AtomicBoolean stop = new AtomicBoolean(); + + // Run atomic load. + IgniteInternalFuture atomicLoadFuture = GridTestUtils.runMultiThreadedAsync(() -> { + int k = 0; + + while (!stop.get()) { + k++; + try { + atomicCache.put(k, k); + } catch (Exception ignored) {} + } + }, 1, "atomic-load"); + + // Run tx load. + IgniteInternalFuture txLoadFuture = GridTestUtils.runMultiThreadedAsync(() -> { + final int txOps = 5; + + while (!stop.get()) { + List randomKeys = Stream.generate(() -> ThreadLocalRandom.current().nextInt(5)) + .limit(txOps) + .sorted() + .collect(Collectors.toList()); + + try (Transaction tx = ignite.transactions().txStart(TransactionConcurrency.OPTIMISTIC, TransactionIsolation.READ_COMMITTED)) { + for (Integer key : randomKeys) + txCache.put(key, key); + + tx.commit(); + } + catch (Exception ignored) { } + } + }, 4, "tx-load"); + + // Wait for some data. + Thread.sleep(1000); + + // Prevent sending full message. + spi.blockFullMessage(); + + // Trigger exchange. + IgniteInternalFuture nodeStopFuture = GridTestUtils.runAsync(() -> stopGrid(3)); + + try { + spi.waitUntilAllSingleMessagesAreSent(); + + List interceptedMessages = spi.getMessages(); + + // Associate each message with existing node UUID. + Map messagesMap = new HashMap<>(); + for (int i = 0; i < interceptedMessages.size(); i++) + messagesMap.put(grid(i + 1).context().localNodeId(), interceptedMessages.get(i)); + + GridDhtPartitionsStateValidator validator = new GridDhtPartitionsStateValidator(ignite.context().cache().context()); + + // Validate partition update counters. If counters are not consistent, exception will be thrown. + validator.validatePartitionsUpdateCounters(ignite.cachex(atomicCacheName).context().topology(), messagesMap, Collections.emptySet()); + validator.validatePartitionsUpdateCounters(ignite.cachex(txCacheName).context().topology(), messagesMap, Collections.emptySet()); + + } finally { + // Stop load and resume exchange. + spi.unblockFullMessage(); + + stop.set(true); + + atomicLoadFuture.get(); + txLoadFuture.get(); + nodeStopFuture.get(); + } + + // Return grid to initial state. + startGrid(3); + + awaitPartitionMapExchange(); + } + } + + /** + * SPI which intercepts single messages during exchange. + */ + private static class SingleMessageInterceptorCommunicationSpi extends TcpCommunicationSpi { + /** */ + private static final List messages = new CopyOnWriteArrayList<>(); + + /** Future completes when {@link #singleMessagesThreshold} messages are sent to coordinator. */ + private static final GridFutureAdapter allSingleMessagesSent = new GridFutureAdapter(); + + /** A number of single messages we're waiting for send. */ + private final int singleMessagesThreshold; + + /** Latch which blocks full message sending. */ + private volatile CountDownLatch blockFullMsgLatch; + + /** + * Constructor. + */ + private SingleMessageInterceptorCommunicationSpi(int singleMessagesThreshold) { + this.singleMessagesThreshold = singleMessagesThreshold; + } + + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) throws IgniteSpiException { + if (((GridIoMessage) msg).message() instanceof GridDhtPartitionsSingleMessage) { + GridDhtPartitionsSingleMessage singleMsg = (GridDhtPartitionsSingleMessage) ((GridIoMessage) msg).message(); + + // We're interesting for only exchange messages and when node is stopped. + if (singleMsg.exchangeId() != null && singleMsg.exchangeId().isLeft() && !singleMsg.client()) { + messages.add(singleMsg); + + if (messages.size() == singleMessagesThreshold) + allSingleMessagesSent.onDone(); + } + } + + try { + if (((GridIoMessage) msg).message() instanceof GridDhtPartitionsFullMessage) { + if (blockFullMsgLatch != null) + blockFullMsgLatch.await(); + } + } + catch (Exception ignored) { } + + super.sendMessage(node, msg, ackC); + } + + /** */ + public void clear() { + messages.clear(); + allSingleMessagesSent.reset(); + } + + /** */ + public List getMessages() { + return Collections.unmodifiableList(messages); + } + + /** */ + public void blockFullMessage() { + blockFullMsgLatch = new CountDownLatch(1); + } + + /** */ + public void unblockFullMessage() { + blockFullMsgLatch.countDown(); + } + + /** */ + public void waitUntilAllSingleMessagesAreSent() throws IgniteCheckedException { + allSingleMessagesSent.get(); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidatorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidatorSelfTest.java new file mode 100644 index 0000000000000..9ed8d54080a37 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidatorSelfTest.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.distributed.dht; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.Assert; +import org.mockito.Matchers; +import org.mockito.Mockito; + +/** + * Test correct behaviour of {@link GridDhtPartitionsStateValidator} class. + */ +public class GridCachePartitionsStateValidatorSelfTest extends GridCommonAbstractTest { + /** Mocks and stubs. */ + private final UUID localNodeId = UUID.randomUUID(); + /** */ + private GridCacheSharedContext cctxMock; + /** */ + private GridDhtPartitionTopology topologyMock; + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + // Prepare mocks. + cctxMock = Mockito.mock(GridCacheSharedContext.class); + Mockito.when(cctxMock.localNodeId()).thenReturn(localNodeId); + + topologyMock = Mockito.mock(GridDhtPartitionTopology.class); + Mockito.when(topologyMock.partitionState(Matchers.any(), Matchers.anyInt())).thenReturn(GridDhtPartitionState.OWNING); + Mockito.when(topologyMock.groupId()).thenReturn(0); + Mockito.when(topologyMock.partitions()).thenReturn(3); + + List localPartitions = Lists.newArrayList( + partitionMock(0, 1, 1), + partitionMock(1, 2, 2), + partitionMock(2, 3, 3) + ); + Mockito.when(topologyMock.localPartitions()).thenReturn(localPartitions); + Mockito.when(topologyMock.currentLocalPartitions()).thenReturn(localPartitions); + } + + /** + * @return Partition mock with specified {@code id}, {@code updateCounter} and {@code size}. + */ + private GridDhtLocalPartition partitionMock(int id, long updateCounter, long size) { + GridDhtLocalPartition partitionMock = Mockito.mock(GridDhtLocalPartition.class); + Mockito.when(partitionMock.id()).thenReturn(id); + Mockito.when(partitionMock.updateCounter()).thenReturn(updateCounter); + Mockito.when(partitionMock.fullSize()).thenReturn(size); + return partitionMock; + } + + /** + * @return Message containing specified {@code countersMap}. + */ + private GridDhtPartitionsSingleMessage fromUpdateCounters(Map> countersMap) { + GridDhtPartitionsSingleMessage msg = new GridDhtPartitionsSingleMessage(); + msg.addPartitionUpdateCounters(0, countersMap); + return msg; + } + + /** + * @return Message containing specified {@code sizesMap}. + */ + private GridDhtPartitionsSingleMessage fromCacheSizes(Map sizesMap) { + GridDhtPartitionsSingleMessage msg = new GridDhtPartitionsSingleMessage(); + msg.addPartitionSizes(0, sizesMap); + return msg; + } + + /** + * Test partition update counters validation. + */ + public void testPartitionCountersValidation() { + UUID remoteNode = UUID.randomUUID(); + UUID ignoreNode = UUID.randomUUID(); + + // For partitions 0 and 2 (zero counter) we have inconsistent update counters. + Map> updateCountersMap = new HashMap<>(); + updateCountersMap.put(0, new T2<>(2L, 2L)); + updateCountersMap.put(1, new T2<>(2L, 2L)); + + // Form single messages map. + Map messages = new HashMap<>(); + messages.put(remoteNode, fromUpdateCounters(updateCountersMap)); + messages.put(ignoreNode, fromUpdateCounters(updateCountersMap)); + + GridDhtPartitionsStateValidator validator = new GridDhtPartitionsStateValidator(cctxMock); + + // (partId, (nodeId, updateCounter)) + Map> result = validator.validatePartitionsUpdateCounters(topologyMock, messages, Sets.newHashSet(ignoreNode)); + + // Check that validation result contains all necessary information. + Assert.assertEquals(2, result.size()); + Assert.assertTrue(result.containsKey(0)); + Assert.assertTrue(result.containsKey(2)); + Assert.assertTrue(result.get(0).get(localNodeId) == 1L); + Assert.assertTrue(result.get(0).get(remoteNode) == 2L); + Assert.assertTrue(result.get(2).get(localNodeId) == 3L); + Assert.assertTrue(result.get(2).get(remoteNode) == 0L); + } + + /** + * Test partition cache sizes validation. + */ + public void testPartitionCacheSizesValidation() { + UUID remoteNode = UUID.randomUUID(); + UUID ignoreNode = UUID.randomUUID(); + + // For partitions 0 and 2 we have inconsistent cache sizes. + Map cacheSizesMap = new HashMap<>(); + cacheSizesMap.put(0, 2L); + cacheSizesMap.put(1, 2L); + cacheSizesMap.put(2, 2L); + + // Form single messages map. + Map messages = new HashMap<>(); + messages.put(remoteNode, fromCacheSizes(cacheSizesMap)); + messages.put(ignoreNode, fromCacheSizes(cacheSizesMap)); + + GridDhtPartitionsStateValidator validator = new GridDhtPartitionsStateValidator(cctxMock); + + // (partId, (nodeId, cacheSize)) + Map> result = validator.validatePartitionsSizes(topologyMock, messages, Sets.newHashSet(ignoreNode)); + + // Check that validation result contains all necessary information. + Assert.assertEquals(2, result.size()); + Assert.assertTrue(result.containsKey(0)); + Assert.assertTrue(result.containsKey(2)); + Assert.assertTrue(result.get(0).get(localNodeId) == 1L); + Assert.assertTrue(result.get(0).get(remoteNode) == 2L); + Assert.assertTrue(result.get(2).get(localNodeId) == 3L); + Assert.assertTrue(result.get(2).get(remoteNode) == 2L); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticOnPartitionExchangeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticOnPartitionExchangeTest.java new file mode 100644 index 0000000000000..03ea0f7f97d49 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticOnPartitionExchangeTest.java @@ -0,0 +1,322 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.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.atomic.AtomicInteger; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.cache.GridCacheAffinityManager; +import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.internal.util.typedef.T1; +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.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionIsolation; + +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE; +import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; +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; + +/** + * + */ +public class TxOptimisticOnPartitionExchangeTest extends GridCommonAbstractTest { + /** Nodes count. */ + private static final int NODES_CNT = 3; + + /** Tx size. */ + private static final int TX_SIZE = 20 * NODES_CNT; + + /** Cache name. */ + private static final String CACHE_NAME = "cache"; + + /** Logger started. */ + private static volatile boolean msgInterception; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrids(NODES_CNT); + + awaitPartitionMapExchange(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setCommunicationSpi(new TestCommunicationSpi(log())); + + cfg.setCacheConfiguration(defaultCacheConfiguration() + .setName(CACHE_NAME) + .setAtomicityMode(TRANSACTIONAL) + .setWriteSynchronizationMode(FULL_SYNC) + .setCacheMode(PARTITIONED) + .setBackups(1)); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testConsistencyOnPartitionExchange() throws Exception { + doTest(SERIALIZABLE, true); + doTest(READ_COMMITTED, true); + doTest(SERIALIZABLE, false); + doTest(READ_COMMITTED, false); + } + + /** + * @param isolation {@link TransactionIsolation}. + * @param txInitiatorPrimary False If the transaction does not use the keys of the node that initiated it. + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + public void doTest(final TransactionIsolation isolation, boolean txInitiatorPrimary) throws Exception { + final CountDownLatch txStarted = new CountDownLatch(1); + + final IgniteCache cache = ignite(0).cache(CACHE_NAME); + + final Map txValues = new TreeMap<>(); + + ClusterNode node = ignite(0).cluster().node(); + + GridCacheAffinityManager affinity = ((IgniteCacheProxy)cache).context().affinity(); + + for (int i = 0; txValues.size() < TX_SIZE; i++) { + if (!txInitiatorPrimary && node.equals(affinity.primaryByKey(i, NONE))) + continue; + + txValues.put(i, i); + } + + TestCommunicationSpi.init(); + + msgInterception = true; + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() { + try (Transaction tx = ignite(0).transactions().txStart(OPTIMISTIC, isolation)) { + info(">>> TX started."); + + txStarted.countDown(); + + cache.putAll(txValues); + + tx.commit(); + + info(">>> TX committed."); + } + + return null; + } + }); + + txStarted.await(); + + try { + info(">>> Grid starting."); + + IgniteEx ignite = startGrid(NODES_CNT); + + info(">>> Grid started."); + + fut.get(); + + awaitPartitionMapExchange(); + + msgInterception = false; + + IgniteCache cacheStartedNode = ignite.cache(CACHE_NAME); + + try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + Set keys = cacheStartedNode.getAll(txValues.keySet()).keySet(); + + assertEquals(txValues.keySet(), new TreeSet<>(keys)); + + tx.commit(); + } + } + finally { + msgInterception = false; + + stopGrid(NODES_CNT); + } + } + + /** + * + */ + @SuppressWarnings("ConstantConditions") + private static class TestCommunicationSpi extends TcpCommunicationSpi { + /** Partition single message sent from added node. */ + private static volatile CountDownLatch partSingleMsgSentFromAddedNode; + + /** Partition supply message sent count. */ + private static final AtomicInteger partSupplyMsgSentCnt = new AtomicInteger(); + + /** Logger. */ + private IgniteLogger log; + + /** + * @param log Logger. + */ + public TestCommunicationSpi(IgniteLogger log) { + this.log = log; + } + + /** + * + */ + public static void init() { + partSingleMsgSentFromAddedNode = new CountDownLatch(1); + + partSupplyMsgSentCnt.set(0); + } + + /** {@inheritDoc} */ + @Override public void sendMessage( + final ClusterNode node, + final Message msg, + final IgniteInClosure ackC + ) throws IgniteSpiException { + if (msgInterception) { + if (msg instanceof GridIoMessage) { + final Message msg0 = ((GridIoMessage)msg).message(); + + String locNodeId = ((IgniteEx)ignite).context().localNodeId().toString(); + + int nodeIdx = Integer.parseInt(locNodeId.substring(locNodeId.length() - 3)); + + if (nodeIdx == 0) { + if (msg0 instanceof GridNearTxPrepareRequest || msg0 instanceof GridDhtTxPrepareRequest) { + GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + partSingleMsgSentFromAddedNode.await(); + + sendMessage(node, msg, ackC, true); + + return null; + } + }); + + return; + + } + else if (msg0 instanceof GridNearTxFinishRequest || msg0 instanceof GridDhtTxFinishRequest) { + GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + final T1 i = new T1<>(0); + + while (waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return partSupplyMsgSentCnt.get() > i.get(); + } + }, i.get() == 0 ? 5_000 : 500)) + i.set(partSupplyMsgSentCnt.get()); + + sendMessage(node, msg, ackC, true); + + return null; + } + }); + + return; + } + } + else if (nodeIdx == NODES_CNT && msg0 instanceof GridDhtPartitionsSingleMessage) + partSingleMsgSentFromAddedNode.countDown(); + + if (msg0 instanceof GridDhtPartitionSupplyMessage) + partSupplyMsgSentCnt.incrementAndGet(); + } + } + + sendMessage(node, msg, ackC, msgInterception); + } + + /** + * @param node Node. + * @param msg Message. + * @param ackC Ack closure. + * @param logMsg Log Messages. + */ + private void sendMessage( + final ClusterNode node, + final Message msg, + final IgniteInClosure ackC, + boolean logMsg + ) throws IgniteSpiException { + if (logMsg) { + String id = node.id().toString(); + String locNodeId = ((IgniteEx)ignite).context().localNodeId().toString(); + + Message msg0 = ((GridIoMessage)msg).message(); + + log.info( + String.format(">>> Output msg[type=%s, fromNode= %s, toNode=%s]", + msg0.getClass().getSimpleName(), + locNodeId.charAt(locNodeId.length() - 1), + id.charAt(id.length() - 1) + ) + ); + } + + super.sendMessage(node, msg, ackC); + } + } +} \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java index bb397f7214355..06126151271e9 100755 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java @@ -133,6 +133,8 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheAtomicNearCacheSelfTest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheColocatedTxExceptionSelfTest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheGlobalLoadTest; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionsStateValidationTest; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionsStateValidatorSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheGetStoreErrorSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearTxExceptionSelfTest; import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedTxExceptionSelfTest; @@ -292,6 +294,8 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(IgniteCacheSystemTransactionsSelfTest.class); suite.addTestSuite(CacheDeferredDeleteSanitySelfTest.class); suite.addTestSuite(CacheDeferredDeleteQueueTest.class); + suite.addTestSuite(GridCachePartitionsStateValidatorSelfTest.class); + suite.addTestSuite(GridCachePartitionsStateValidationTest.class); suite.addTest(IgniteCacheTcpClientDiscoveryTestSuite.suite()); diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java index f8add30ef5535..415479dae4138 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java @@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.WalModeChangeAdvancedSelfTest; import org.apache.ignite.internal.processors.cache.WalModeChangeCoordinatorNotAffinityNodeSelfTest; import org.apache.ignite.internal.processors.cache.WalModeChangeSelfTest; +import org.apache.ignite.internal.processors.cache.datastructures.IgniteExchangeLatchManagerCoordinatorFailTest; import org.apache.ignite.internal.processors.cache.distributed.CacheExchangeMergeTest; import org.apache.ignite.internal.processors.cache.distributed.CachePartitionStateTest; import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionEvictionDuringReadThroughSelfTest; @@ -40,6 +41,7 @@ import org.apache.ignite.internal.processors.cache.distributed.IgniteOptimisticTxSuspendResumeTest; import org.apache.ignite.internal.processors.cache.distributed.IgnitePessimisticTxSuspendResumeTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsCacheAssignmentNodeRestartsTest; +import org.apache.ignite.internal.processors.cache.transactions.TxOptimisticOnPartitionExchangeTest; import org.apache.ignite.internal.processors.cache.transactions.TxRollbackOnTimeoutNearCacheTest; import org.apache.ignite.internal.processors.cache.transactions.TxRollbackOnTimeoutNoDeadlockDetectionTest; import org.apache.ignite.internal.processors.cache.transactions.TxRollbackOnTimeoutTest; @@ -93,6 +95,10 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(PartitionedTransactionalOptimisticCacheGetsDistributionTest.class); suite.addTestSuite(PartitionedTransactionalPessimisticCacheGetsDistributionTest.class); + suite.addTestSuite(TxOptimisticOnPartitionExchangeTest.class); + + suite.addTestSuite(IgniteExchangeLatchManagerCoordinatorFailTest.class); + return suite; } } From 9abfee69aa153888456f9e8574ece1f2d0cbe4d9 Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Tue, 10 Apr 2018 12:46:43 +0300 Subject: [PATCH 0020/1463] IGNITE-8059: Integrate decision tree with partition based dataset. this closes #3760 (cherry picked from commit 139c2af) --- ...isionTreeClassificationTrainerExample.java | 147 +++++ .../DecisionTreeRegressionTrainerExample.java | 124 ++++ .../ml/{trees => tree}/package-info.java | 2 +- .../ml/trees/DecisionTreesExample.java | 354 ----------- .../java/org/apache/ignite/ml/Trainer.java | 3 - .../apache/ignite/ml/tree/DecisionTree.java | 252 ++++++++ .../DecisionTreeClassificationTrainer.java | 93 +++ .../ml/tree/DecisionTreeConditionalNode.java | 78 +++ .../DecisionTreeLeafNode.java} | 35 +- .../ignite/ml/tree/DecisionTreeNode.java | 26 + .../tree/DecisionTreeRegressionTrainer.java | 60 ++ .../TreeFilter.java} | 21 +- .../ignite/ml/tree/data/DecisionTreeData.java | 128 ++++ .../ml/tree/data/DecisionTreeDataBuilder.java | 73 +++ .../ignite/ml/tree/data/package-info.java | 22 + .../ml/tree/impurity/ImpurityMeasure.java | 55 ++ .../impurity/ImpurityMeasureCalculator.java | 38 ++ .../impurity/gini/GiniImpurityMeasure.java | 115 ++++ .../gini/GiniImpurityMeasureCalculator.java | 110 ++++ .../impurity/gini}/package-info.java | 4 +- .../tree/impurity/mse/MSEImpurityMeasure.java | 133 ++++ .../mse/MSEImpurityMeasureCalculator.java | 80 +++ .../impurity/mse}/package-info.java | 4 +- .../ignite/ml/tree/impurity/package-info.java | 22 + .../util/SimpleStepFunctionCompressor.java | 149 +++++ .../ml/tree/impurity/util/StepFunction.java | 162 +++++ .../impurity/util/StepFunctionCompressor.java | 55 ++ .../ml/tree/impurity/util/package-info.java | 22 + .../leaf/DecisionTreeLeafBuilder.java} | 32 +- .../leaf/MeanDecisionTreeLeafBuilder.java | 73 +++ .../MostCommonDecisionTreeLeafBuilder.java | 86 +++ .../regcalcs => tree/leaf}/package-info.java | 4 +- .../ml/{trees => tree}/package-info.java | 4 +- .../ml/trees/CategoricalRegionInfo.java | 72 --- .../ignite/ml/trees/CategoricalSplitInfo.java | 68 --- .../ignite/ml/trees/ContinuousRegionInfo.java | 74 --- .../ml/trees/ContinuousSplitCalculator.java | 51 -- .../apache/ignite/ml/trees/RegionInfo.java | 62 -- .../ml/trees/nodes/CategoricalSplitNode.java | 50 -- .../ml/trees/nodes/ContinuousSplitNode.java | 56 -- .../ignite/ml/trees/nodes/SplitNode.java | 100 --- .../trees/trainers/columnbased/BiIndex.java | 113 ---- ...edCacheColumnDecisionTreeTrainerInput.java | 57 -- .../CacheColumnDecisionTreeTrainerInput.java | 141 ----- .../ColumnDecisionTreeTrainer.java | 568 ------------------ .../ColumnDecisionTreeTrainerInput.java | 55 -- .../MatrixColumnDecisionTreeTrainerInput.java | 83 --- .../columnbased/RegionProjection.java | 109 ---- .../trainers/columnbased/TrainingContext.java | 166 ----- .../columnbased/caches/ContextCache.java | 68 --- .../columnbased/caches/FeaturesCache.java | 151 ----- .../columnbased/caches/ProjectionsCache.java | 286 --------- .../columnbased/caches/SplitCache.java | 206 ------- .../columnbased/caches/package-info.java | 22 - .../ContinuousSplitCalculators.java | 34 -- .../contsplitcalcs/GiniSplitCalculator.java | 234 -------- .../VarianceSplitCalculator.java | 179 ------ .../contsplitcalcs/package-info.java | 22 - .../trainers/columnbased/package-info.java | 22 - .../regcalcs/RegionCalculators.java | 85 --- .../vectors/CategoricalFeatureProcessor.java | 212 ------- .../vectors/ContinuousFeatureProcessor.java | 111 ---- .../vectors/ContinuousSplitInfo.java | 71 --- .../columnbased/vectors/FeatureProcessor.java | 82 --- .../vectors/FeatureVectorProcessorUtils.java | 57 -- .../columnbased/vectors/SampleInfo.java | 80 --- .../columnbased/vectors/SplitInfo.java | 106 ---- .../columnbased/vectors/package-info.java | 22 - .../apache/ignite/ml/IgniteMLTestSuite.java | 4 +- .../ml/nn/performance/MnistMLPTestUtil.java | 9 +- ...eClassificationTrainerIntegrationTest.java | 100 +++ ...DecisionTreeClassificationTrainerTest.java | 91 +++ ...nTreeRegressionTrainerIntegrationTest.java | 100 +++ .../DecisionTreeRegressionTrainerTest.java | 91 +++ .../ignite/ml/tree/DecisionTreeTestSuite.java | 48 ++ .../ml/tree/data/DecisionTreeDataTest.java | 59 ++ .../GiniImpurityMeasureCalculatorTest.java | 103 ++++ .../gini/GiniImpurityMeasureTest.java | 131 ++++ .../mse/MSEImpurityMeasureCalculatorTest.java | 59 ++ .../impurity/mse/MSEImpurityMeasureTest.java | 109 ++++ .../SimpleStepFunctionCompressorTest.java | 75 +++ .../tree/impurity/util/StepFunctionTest.java | 71 +++ .../impurity/util/TestImpurityMeasure.java | 88 +++ .../DecisionTreeMNISTIntegrationTest.java | 105 ++++ .../performance/DecisionTreeMNISTTest.java | 74 +++ .../ignite/ml/trees/BaseDecisionTreeTest.java | 70 --- .../trees/ColumnDecisionTreeTrainerTest.java | 191 ------ .../ml/trees/DecisionTreesTestSuite.java | 33 - .../ml/trees/GiniSplitCalculatorTest.java | 141 ----- .../ignite/ml/trees/SplitDataGenerator.java | 390 ------------ .../ml/trees/VarianceSplitCalculatorTest.java | 84 --- .../ColumnDecisionTreeTrainerBenchmark.java | 456 -------------- ...IgniteColumnDecisionTreeGiniBenchmark.java | 70 --- ...teColumnDecisionTreeVarianceBenchmark.java | 71 --- .../ml/trees/SplitDataGenerator.java | 426 ------------- .../yardstick/ml/trees/package-info.java | 22 - 96 files changed, 3465 insertions(+), 6247 deletions(-) create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeClassificationTrainerExample.java create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeRegressionTrainerExample.java rename examples/src/main/java/org/apache/ignite/examples/ml/{trees => tree}/package-info.java (95%) delete mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/trees/DecisionTreesExample.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTree.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeConditionalNode.java rename modules/ml/src/main/java/org/apache/ignite/ml/{trees/nodes/Leaf.java => tree/DecisionTreeLeafNode.java} (61%) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeNode.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainer.java rename modules/ml/src/main/java/org/apache/ignite/ml/{trees/nodes/DecisionTreeNode.java => tree/TreeFilter.java} (59%) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/data/DecisionTreeData.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/data/DecisionTreeDataBuilder.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/data/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/ImpurityMeasure.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/ImpurityMeasureCalculator.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/gini/GiniImpurityMeasure.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/gini/GiniImpurityMeasureCalculator.java rename modules/ml/src/main/java/org/apache/ignite/ml/{trees/nodes => tree/impurity/gini}/package-info.java (89%) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/mse/MSEImpurityMeasure.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/mse/MSEImpurityMeasureCalculator.java rename modules/ml/src/main/java/org/apache/ignite/ml/{trees/models => tree/impurity/mse}/package-info.java (88%) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/package-info.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/util/SimpleStepFunctionCompressor.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/util/StepFunction.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/util/StepFunctionCompressor.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/util/package-info.java rename modules/ml/src/main/java/org/apache/ignite/ml/{trees/models/DecisionTreeModel.java => tree/leaf/DecisionTreeLeafBuilder.java} (55%) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/leaf/MeanDecisionTreeLeafBuilder.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/tree/leaf/MostCommonDecisionTreeLeafBuilder.java rename modules/ml/src/main/java/org/apache/ignite/ml/{trees/trainers/columnbased/regcalcs => tree/leaf}/package-info.java (90%) rename modules/ml/src/main/java/org/apache/ignite/ml/{trees => tree}/package-info.java (92%) delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/CategoricalRegionInfo.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/CategoricalSplitInfo.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/ContinuousRegionInfo.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/ContinuousSplitCalculator.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/RegionInfo.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/CategoricalSplitNode.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/ContinuousSplitNode.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/SplitNode.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/BiIndex.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/BiIndexedCacheColumnDecisionTreeTrainerInput.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/CacheColumnDecisionTreeTrainerInput.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/ColumnDecisionTreeTrainer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/ColumnDecisionTreeTrainerInput.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/MatrixColumnDecisionTreeTrainerInput.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/RegionProjection.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/TrainingContext.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/ContextCache.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/FeaturesCache.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/ProjectionsCache.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/SplitCache.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/contsplitcalcs/ContinuousSplitCalculators.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/contsplitcalcs/GiniSplitCalculator.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/contsplitcalcs/VarianceSplitCalculator.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/contsplitcalcs/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/regcalcs/RegionCalculators.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/CategoricalFeatureProcessor.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/ContinuousFeatureProcessor.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/ContinuousSplitInfo.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/FeatureProcessor.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/FeatureVectorProcessorUtils.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/SampleInfo.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/SplitInfo.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/package-info.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerIntegrationTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerIntegrationTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeTestSuite.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/data/DecisionTreeDataTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/gini/GiniImpurityMeasureCalculatorTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/gini/GiniImpurityMeasureTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/mse/MSEImpurityMeasureCalculatorTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/mse/MSEImpurityMeasureTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/util/SimpleStepFunctionCompressorTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/util/StepFunctionTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/util/TestImpurityMeasure.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTIntegrationTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/trees/BaseDecisionTreeTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/trees/ColumnDecisionTreeTrainerTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/trees/DecisionTreesTestSuite.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/trees/GiniSplitCalculatorTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/trees/SplitDataGenerator.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/trees/VarianceSplitCalculatorTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/trees/performance/ColumnDecisionTreeTrainerBenchmark.java delete mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/trees/IgniteColumnDecisionTreeGiniBenchmark.java delete mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/trees/IgniteColumnDecisionTreeVarianceBenchmark.java delete mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/trees/SplitDataGenerator.java delete mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/trees/package-info.java diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeClassificationTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeClassificationTrainerExample.java new file mode 100644 index 0000000000000..cef63683cb4ec --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeClassificationTrainerExample.java @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.examples.ml.tree; + +import java.util.Random; +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.ml.dataset.impl.cache.CacheBasedDatasetBuilder; +import org.apache.ignite.ml.tree.DecisionTreeClassificationTrainer; +import org.apache.ignite.ml.tree.DecisionTreeNode; +import org.apache.ignite.thread.IgniteThread; + +/** + * Example of using distributed {@link DecisionTreeClassificationTrainer}. + */ +public class DecisionTreeClassificationTrainerExample { + /** + * Executes example. + * + * @param args Command line arguments, none required. + */ + public static void main(String... args) throws InterruptedException { + System.out.println(">>> Decision tree classification trainer example started."); + + // Start ignite grid. + try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { + System.out.println(">>> Ignite grid started."); + + IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), + DecisionTreeClassificationTrainerExample.class.getSimpleName(), () -> { + + // Create cache with training data. + CacheConfiguration trainingSetCfg = new CacheConfiguration<>(); + trainingSetCfg.setName("TRAINING_SET"); + trainingSetCfg.setAffinity(new RendezvousAffinityFunction(false, 10)); + + IgniteCache trainingSet = ignite.createCache(trainingSetCfg); + + Random rnd = new Random(0); + + // Fill training data. + for (int i = 0; i < 1000; i++) + trainingSet.put(i, generatePoint(rnd)); + + // Create classification trainer. + DecisionTreeClassificationTrainer trainer = new DecisionTreeClassificationTrainer(4, 0); + + // Train decision tree model. + DecisionTreeNode mdl = trainer.fit( + new CacheBasedDatasetBuilder<>(ignite, trainingSet), + (k, v) -> new double[]{v.x, v.y}, + (k, v) -> v.lb + ); + + // Calculate score. + int correctPredictions = 0; + for (int i = 0; i < 1000; i++) { + LabeledPoint pnt = generatePoint(rnd); + + double prediction = mdl.apply(new double[]{pnt.x, pnt.y}); + + if (prediction == pnt.lb) + correctPredictions++; + } + + System.out.println(">>> Accuracy: " + correctPredictions / 10.0 + "%"); + + System.out.println(">>> Decision tree classification trainer example completed."); + }); + + igniteThread.start(); + + igniteThread.join(); + } + } + + /** + * Generate point with {@code x} in (-0.5, 0.5) and {@code y} in the same interval. If {@code x * y > 0} then label + * is 1, otherwise 0. + * + * @param rnd Random. + * @return Point with label. + */ + private static LabeledPoint generatePoint(Random rnd) { + + double x = rnd.nextDouble() - 0.5; + double y = rnd.nextDouble() - 0.5; + + return new LabeledPoint(x, y, x * y > 0 ? 1 : 0); + } + + /** Point data class. */ + private static class Point { + /** X coordinate. */ + final double x; + + /** Y coordinate. */ + final double y; + + /** + * Constructs a new instance of point. + * + * @param x X coordinate. + * @param y Y coordinate. + */ + Point(double x, double y) { + this.x = x; + this.y = y; + } + } + + /** Labeled point data class. */ + private static class LabeledPoint extends Point { + /** Point label. */ + final double lb; + + /** + * Constructs a new instance of labeled point data. + * + * @param x X coordinate. + * @param y Y coordinate. + * @param lb Point label. + */ + LabeledPoint(double x, double y, double lb) { + super(x, y); + this.lb = lb; + } + } +} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeRegressionTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeRegressionTrainerExample.java new file mode 100644 index 0000000000000..61ba5f9dca471 --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeRegressionTrainerExample.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.examples.ml.tree; + +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.ml.dataset.impl.cache.CacheBasedDatasetBuilder; +import org.apache.ignite.ml.tree.DecisionTreeNode; +import org.apache.ignite.ml.tree.DecisionTreeRegressionTrainer; +import org.apache.ignite.thread.IgniteThread; + +/** + * Example of using distributed {@link DecisionTreeRegressionTrainer}. + */ +public class DecisionTreeRegressionTrainerExample { + /** + * Executes example. + * + * @param args Command line arguments, none required. + */ + public static void main(String... args) throws InterruptedException { + System.out.println(">>> Decision tree regression trainer example started."); + + // Start ignite grid. + try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { + System.out.println(">>> Ignite grid started."); + + IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), + DecisionTreeRegressionTrainerExample.class.getSimpleName(), () -> { + + // Create cache with training data. + CacheConfiguration trainingSetCfg = new CacheConfiguration<>(); + trainingSetCfg.setName("TRAINING_SET"); + trainingSetCfg.setAffinity(new RendezvousAffinityFunction(false, 10)); + + IgniteCache trainingSet = ignite.createCache(trainingSetCfg); + + // Fill training data. + generatePoints(trainingSet); + + // Create regression trainer. + DecisionTreeRegressionTrainer trainer = new DecisionTreeRegressionTrainer(10, 0); + + // Train decision tree model. + DecisionTreeNode mdl = trainer.fit( + new CacheBasedDatasetBuilder<>(ignite, trainingSet), + (k, v) -> new double[] {v.x}, + (k, v) -> v.y + ); + + System.out.println(">>> Linear regression model: " + mdl); + + System.out.println(">>> ---------------------------------"); + System.out.println(">>> | Prediction\t| Ground Truth\t|"); + System.out.println(">>> ---------------------------------"); + + // Calculate score. + for (int x = 0; x < 10; x++) { + double predicted = mdl.apply(new double[] {x}); + + System.out.printf(">>> | %.4f\t\t| %.4f\t\t|\n", predicted, Math.sin(x)); + } + + System.out.println(">>> ---------------------------------"); + + System.out.println(">>> Decision tree regression trainer example completed."); + }); + + igniteThread.start(); + + igniteThread.join(); + } + } + + /** + * Generates {@code sin(x)} on interval [0, 10) and loads into the specified cache. + */ + private static void generatePoints(IgniteCache trainingSet) { + for (int i = 0; i < 1000; i++) { + double x = i / 100.0; + double y = Math.sin(x); + + trainingSet.put(i, new Point(x, y)); + } + } + + /** Point data class. */ + private static class Point { + /** X coordinate. */ + final double x; + + /** Y coordinate. */ + final double y; + + /** + * Constructs a new instance of point. + * + * @param x X coordinate. + * @param y Y coordinate. + */ + Point(double x, double y) { + this.x = x; + this.y = y; + } + } +} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/trees/package-info.java b/examples/src/main/java/org/apache/ignite/examples/ml/tree/package-info.java similarity index 95% rename from examples/src/main/java/org/apache/ignite/examples/ml/trees/package-info.java rename to examples/src/main/java/org/apache/ignite/examples/ml/tree/package-info.java index d944f60570b0b..d8d9de60a2169 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/trees/package-info.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/tree/package-info.java @@ -19,4 +19,4 @@ * * Decision trees examples. */ -package org.apache.ignite.examples.ml.trees; +package org.apache.ignite.examples.ml.tree; diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/trees/DecisionTreesExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/trees/DecisionTreesExample.java deleted file mode 100644 index b1b2c421a9f5f..0000000000000 --- a/examples/src/main/java/org/apache/ignite/examples/ml/trees/DecisionTreesExample.java +++ /dev/null @@ -1,354 +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.trees; - -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.IOException; -import java.net.URL; -import java.nio.channels.Channels; -import java.nio.channels.ReadableByteChannel; -import java.util.Collection; -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.Scanner; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import java.util.zip.GZIPInputStream; -import org.apache.commons.cli.BasicParser; -import org.apache.commons.cli.CommandLine; -import org.apache.commons.cli.CommandLineParser; -import org.apache.commons.cli.Option; -import org.apache.commons.cli.OptionBuilder; -import org.apache.commons.cli.Options; -import org.apache.commons.cli.ParseException; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteDataStreamer; -import org.apache.ignite.Ignition; -import org.apache.ignite.cache.CacheWriteSynchronizationMode; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.examples.ExampleNodeStartup; -import org.apache.ignite.examples.ml.MLExamplesCommonArgs; -import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.ml.Model; -import org.apache.ignite.ml.estimators.Estimators; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.functions.IgniteTriFunction; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; -import org.apache.ignite.ml.trees.models.DecisionTreeModel; -import org.apache.ignite.ml.trees.trainers.columnbased.BiIndex; -import org.apache.ignite.ml.trees.trainers.columnbased.BiIndexedCacheColumnDecisionTreeTrainerInput; -import org.apache.ignite.ml.trees.trainers.columnbased.ColumnDecisionTreeTrainer; -import org.apache.ignite.ml.trees.trainers.columnbased.contsplitcalcs.ContinuousSplitCalculators; -import org.apache.ignite.ml.trees.trainers.columnbased.contsplitcalcs.GiniSplitCalculator; -import org.apache.ignite.ml.trees.trainers.columnbased.regcalcs.RegionCalculators; -import org.apache.ignite.ml.util.MnistUtils; -import org.jetbrains.annotations.NotNull; - -/** - *

    - * Example of usage of decision trees algorithm for MNIST dataset - * (it can be found here: http://yann.lecun.com/exdb/mnist/).

    - *

    - * Remote nodes should always be started with special configuration file which - * enables P2P class loading: {@code 'ignite.{sh|bat} examples/config/example-ignite.xml'}.

    - *

    - * Alternatively you can run {@link ExampleNodeStartup} in another JVM which will start node - * with {@code examples/config/example-ignite.xml} configuration.

    - *

    - * It is recommended to start at least one node prior to launching this example if you intend - * to run it with default memory settings.

    - *

    - * This example should be run with program arguments, for example - * -cfg examples/config/example-ignite.xml.

    - *

    - * -cfg specifies path to a config path.

    - */ -public class DecisionTreesExample { - /** Name of parameter specifying path of Ignite config. */ - private static final String CONFIG = "cfg"; - - /** Default config path. */ - private static final String DEFAULT_CONFIG = "examples/config/example-ignite.xml"; - - /** - * Folder in which MNIST dataset is expected. - */ - private static String MNIST_DIR = "examples/src/main/resources/"; - - /** - * Key for MNIST training images. - */ - private static String MNIST_TRAIN_IMAGES = "train_images"; - - /** - * Key for MNIST training labels. - */ - private static String MNIST_TRAIN_LABELS = "train_labels"; - - /** - * Key for MNIST test images. - */ - private static String MNIST_TEST_IMAGES = "test_images"; - - /** - * Key for MNIST test labels. - */ - private static String MNIST_TEST_LABELS = "test_labels"; - - /** - * Launches example. - * - * @param args Program arguments. - */ - public static void main(String[] args) throws IOException { - System.out.println(">>> Decision trees example started."); - - String igniteCfgPath; - - CommandLineParser parser = new BasicParser(); - - String trainingImagesPath; - String trainingLabelsPath; - - String testImagesPath; - String testLabelsPath; - - Map mnistPaths = new HashMap<>(); - - mnistPaths.put(MNIST_TRAIN_IMAGES, "train-images-idx3-ubyte"); - mnistPaths.put(MNIST_TRAIN_LABELS, "train-labels-idx1-ubyte"); - mnistPaths.put(MNIST_TEST_IMAGES, "t10k-images-idx3-ubyte"); - mnistPaths.put(MNIST_TEST_LABELS, "t10k-labels-idx1-ubyte"); - - try { - // Parse the command line arguments. - CommandLine line = parser.parse(buildOptions(), args); - - if (line.hasOption(MLExamplesCommonArgs.UNATTENDED)) { - System.out.println(">>> Skipped example execution because 'unattended' mode is used."); - System.out.println(">>> Decision trees example finished."); - return; - } - - igniteCfgPath = line.getOptionValue(CONFIG, DEFAULT_CONFIG); - } - catch (ParseException e) { - e.printStackTrace(); - return; - } - - if (!getMNIST(mnistPaths.values())) { - System.out.println(">>> You should have MNIST dataset in " + MNIST_DIR + " to run this example."); - return; - } - - trainingImagesPath = Objects.requireNonNull(IgniteUtils.resolveIgnitePath(MNIST_DIR + "/" + - mnistPaths.get(MNIST_TRAIN_IMAGES))).getPath(); - trainingLabelsPath = Objects.requireNonNull(IgniteUtils.resolveIgnitePath(MNIST_DIR + "/" + - mnistPaths.get(MNIST_TRAIN_LABELS))).getPath(); - testImagesPath = Objects.requireNonNull(IgniteUtils.resolveIgnitePath(MNIST_DIR + "/" + - mnistPaths.get(MNIST_TEST_IMAGES))).getPath(); - testLabelsPath = Objects.requireNonNull(IgniteUtils.resolveIgnitePath(MNIST_DIR + "/" + - mnistPaths.get(MNIST_TEST_LABELS))).getPath(); - - try (Ignite ignite = Ignition.start(igniteCfgPath)) { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - - int ptsCnt = 60000; - int featCnt = 28 * 28; - - Stream trainingMnistStream = MnistUtils.mnistAsStream(trainingImagesPath, trainingLabelsPath, - new Random(123L), ptsCnt); - - Stream testMnistStream = MnistUtils.mnistAsStream(testImagesPath, testLabelsPath, - new Random(123L), 10_000); - - IgniteCache cache = createBiIndexedCache(ignite); - - loadVectorsIntoBiIndexedCache(cache.getName(), trainingMnistStream.iterator(), featCnt + 1, ignite); - - ColumnDecisionTreeTrainer trainer = new ColumnDecisionTreeTrainer<>(10, - ContinuousSplitCalculators.GINI.apply(ignite), - RegionCalculators.GINI, - RegionCalculators.MOST_COMMON, - ignite); - - System.out.println(">>> Training started"); - long before = System.currentTimeMillis(); - DecisionTreeModel mdl = trainer.train(new BiIndexedCacheColumnDecisionTreeTrainerInput(cache, new HashMap<>(), ptsCnt, featCnt)); - System.out.println(">>> Training finished in " + (System.currentTimeMillis() - before)); - - IgniteTriFunction, Stream>, Function, Double> mse = - Estimators.errorsPercentage(); - - Double accuracy = mse.apply(mdl, testMnistStream.map(v -> - new IgniteBiTuple<>(v.viewPart(0, featCnt), v.getX(featCnt))), Function.identity()); - - System.out.println(">>> Errs percentage: " + accuracy); - } - catch (IOException e) { - e.printStackTrace(); - } - - System.out.println(">>> Decision trees example finished."); - } - - /** - * Get MNIST dataset. Value of predicate 'MNIST dataset is present in expected folder' is returned. - * - * @param mnistFileNames File names of MNIST dataset. - * @return Value of predicate 'MNIST dataset is present in expected folder'. - * @throws IOException In case of file system errors. - */ - private static boolean getMNIST(Collection mnistFileNames) throws IOException { - List missing = mnistFileNames.stream(). - filter(f -> IgniteUtils.resolveIgnitePath(MNIST_DIR + "/" + f) == null). - collect(Collectors.toList()); - - if (!missing.isEmpty()) { - System.out.println(">>> You have not fully downloaded MNIST dataset in directory " + MNIST_DIR + - ", do you want it to be downloaded? [y]/n"); - Scanner s = new Scanner(System.in); - String str = s.nextLine(); - - if (!str.isEmpty() && !str.toLowerCase().equals("y")) - return false; - } - - for (String s : missing) { - String f = s + ".gz"; - System.out.println(">>> Downloading " + f + "..."); - URL website = new URL("http://yann.lecun.com/exdb/mnistAsStream/" + f); - ReadableByteChannel rbc = Channels.newChannel(website.openStream()); - FileOutputStream fos = new FileOutputStream(MNIST_DIR + "/" + f); - fos.getChannel().transferFrom(rbc, 0, Long.MAX_VALUE); - System.out.println(">>> Done."); - - System.out.println(">>> Unzipping " + f + "..."); - unzip(MNIST_DIR + "/" + f, MNIST_DIR + "/" + s); - - System.out.println(">>> Deleting gzip " + f + ", status: " + - Objects.requireNonNull(IgniteUtils.resolveIgnitePath(MNIST_DIR + "/" + f)).delete()); - - System.out.println(">>> Done."); - } - - return true; - } - - /** - * Unzip file located in {@code input} to {@code output}. - * - * @param input Input file path. - * @param output Output file path. - * @throws IOException In case of file system errors. - */ - private static void unzip(String input, String output) throws IOException { - byte[] buf = new byte[1024]; - - try (GZIPInputStream gis = new GZIPInputStream(new FileInputStream(input)); - FileOutputStream out = new FileOutputStream(output)) { - int sz; - while ((sz = gis.read(buf)) > 0) - out.write(buf, 0, sz); - } - } - - /** - * Build cli options. - */ - @NotNull private static Options buildOptions() { - Options options = new Options(); - - Option cfgOpt = OptionBuilder - .withArgName(CONFIG) - .withLongOpt(CONFIG) - .hasArg() - .withDescription("Path to the config.") - .isRequired(false).create(); - - Option unattended = OptionBuilder - .withArgName(MLExamplesCommonArgs.UNATTENDED) - .withLongOpt(MLExamplesCommonArgs.UNATTENDED) - .withDescription("Is example run unattended.") - .isRequired(false).create(); - - options.addOption(cfgOpt); - options.addOption(unattended); - - return options; - } - - /** - * Creates cache where data for training is stored. - * - * @param ignite Ignite instance. - * @return cache where data for training is stored. - */ - private static IgniteCache createBiIndexedCache(Ignite ignite) { - CacheConfiguration cfg = new CacheConfiguration<>(); - - // Write to primary. - cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC); - - // No copying of values. - cfg.setCopyOnRead(false); - - cfg.setName("TMP_BI_INDEXED_CACHE"); - - return ignite.getOrCreateCache(cfg); - } - - /** - * Loads vectors into cache. - * - * @param cacheName Name of cache. - * @param vectorsIter Iterator over vectors to load. - * @param vectorSize Size of vector. - * @param ignite Ignite instance. - */ - private static void loadVectorsIntoBiIndexedCache(String cacheName, Iterator vectorsIter, - int vectorSize, Ignite ignite) { - try (IgniteDataStreamer streamer = - ignite.dataStreamer(cacheName)) { - int sampleIdx = 0; - - streamer.perNodeBufferSize(10000); - - while (vectorsIter.hasNext()) { - org.apache.ignite.ml.math.Vector next = vectorsIter.next(); - - for (int i = 0; i < vectorSize; i++) - streamer.addData(new BiIndex(sampleIdx, i), next.getX(i)); - - sampleIdx++; - - if (sampleIdx % 1000 == 0) - System.out.println(">>> Loaded " + sampleIdx + " vectors."); - } - } - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java index 4e0a5704473ef..f53b80110cb4c 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java @@ -17,11 +17,8 @@ package org.apache.ignite.ml; -import org.apache.ignite.ml.trees.trainers.columnbased.ColumnDecisionTreeTrainer; - /** * Interface for Trainers. Trainer is just a function which produces model from the data. - * See for example {@link ColumnDecisionTreeTrainer}. * * @param Type of produced model. * @param Type of data needed for model producing. 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 new file mode 100644 index 0000000000000..c0b88fc05a4a9 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTree.java @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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; + +import java.io.Serializable; +import java.util.Arrays; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.DatasetBuilder; +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.trainers.DatasetTrainer; +import org.apache.ignite.ml.tree.data.DecisionTreeData; +import org.apache.ignite.ml.tree.data.DecisionTreeDataBuilder; +import org.apache.ignite.ml.tree.impurity.ImpurityMeasure; +import org.apache.ignite.ml.tree.impurity.ImpurityMeasureCalculator; +import org.apache.ignite.ml.tree.impurity.util.StepFunction; +import org.apache.ignite.ml.tree.impurity.util.StepFunctionCompressor; +import org.apache.ignite.ml.tree.leaf.DecisionTreeLeafBuilder; + +/** + * Distributed decision tree trainer that allows to fit trees using row-partitioned dataset. + * + * @param Type of impurity measure. + */ +abstract class DecisionTree> implements DatasetTrainer { + /** Max tree deep. */ + private final int maxDeep; + + /** Min impurity decrease. */ + private final double minImpurityDecrease; + + /** Step function compressor. */ + private final StepFunctionCompressor compressor; + + /** Decision tree leaf builder. */ + private final DecisionTreeLeafBuilder decisionTreeLeafBuilder; + + /** + * Constructs a new distributed decision tree trainer. + * + * @param maxDeep Max tree deep. + * @param minImpurityDecrease Min impurity decrease. + * @param compressor Impurity function compressor. + * @param decisionTreeLeafBuilder Decision tree leaf builder. + */ + DecisionTree(int maxDeep, double minImpurityDecrease, StepFunctionCompressor compressor, DecisionTreeLeafBuilder decisionTreeLeafBuilder) { + this.maxDeep = maxDeep; + this.minImpurityDecrease = minImpurityDecrease; + this.compressor = compressor; + this.decisionTreeLeafBuilder = decisionTreeLeafBuilder; + } + + /** {@inheritDoc} */ + @Override public DecisionTreeNode fit(DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + try (Dataset dataset = datasetBuilder.build( + new EmptyContextBuilder<>(), + new DecisionTreeDataBuilder<>(featureExtractor, lbExtractor) + )) { + return split(dataset, e -> true, 0, getImpurityMeasureCalculator(dataset)); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + /** + * Returns impurity measure calculator. + * + * @param dataset Dataset. + * @return Impurity measure calculator. + */ + abstract ImpurityMeasureCalculator getImpurityMeasureCalculator(Dataset dataset); + + /** + * Splits the node specified by the given dataset and predicate and returns decision tree node. + * + * @param dataset Dataset. + * @param filter Decision tree node predicate. + * @param deep Current tree deep. + * @param impurityCalc Impurity measure calculator. + * @return Decision tree node. + */ + private DecisionTreeNode split(Dataset dataset, TreeFilter filter, int deep, + ImpurityMeasureCalculator impurityCalc) { + if (deep >= maxDeep) + return decisionTreeLeafBuilder.createLeafNode(dataset, filter); + + StepFunction[] criterionFunctions = calculateImpurityForAllColumns(dataset, filter, impurityCalc); + + if (criterionFunctions == null) + return decisionTreeLeafBuilder.createLeafNode(dataset, filter); + + SplitPoint splitPnt = calculateBestSplitPoint(criterionFunctions); + + if (splitPnt == null) + return decisionTreeLeafBuilder.createLeafNode(dataset, filter); + + return new DecisionTreeConditionalNode( + splitPnt.col, + splitPnt.threshold, + split(dataset, updatePredicateForThenNode(filter, splitPnt), deep + 1, impurityCalc), + split(dataset, updatePredicateForElseNode(filter, splitPnt), deep + 1, impurityCalc) + ); + } + + /** + * Calculates impurity measure functions for all columns for the node specified by the given dataset and predicate. + * + * @param dataset Dataset. + * @param filter Decision tree node predicate. + * @param impurityCalc Impurity measure calculator. + * @return Array of impurity measure functions for all columns. + */ + private StepFunction[] calculateImpurityForAllColumns(Dataset dataset, + TreeFilter filter, ImpurityMeasureCalculator impurityCalc) { + return dataset.compute( + part -> { + if (compressor != null) + return compressor.compress(impurityCalc.calculate(part.filter(filter))); + else + return impurityCalc.calculate(part.filter(filter)); + }, this::reduce + ); + } + + /** + * Calculates best split point. + * + * @param criterionFunctions Array of impurity measure functions for all columns. + * @return Best split point. + */ + private SplitPoint calculateBestSplitPoint(StepFunction[] criterionFunctions) { + SplitPoint res = null; + + for (int col = 0; col < criterionFunctions.length; col++) { + StepFunction criterionFunctionForCol = criterionFunctions[col]; + + double[] arguments = criterionFunctionForCol.getX(); + T[] values = criterionFunctionForCol.getY(); + + for (int leftSize = 1; leftSize < values.length - 1; leftSize++) { + if ((values[0].impurity() - values[leftSize].impurity()) > minImpurityDecrease + && (res == null || values[leftSize].compareTo(res.val) < 0)) + res = new SplitPoint<>(values[leftSize], col, calculateThreshold(arguments, leftSize)); + } + } + + return res; + } + + /** + * Merges two arrays gotten from two partitions. + * + * @param a First step function. + * @param b Second step function. + * @return Merged step function. + */ + private StepFunction[] reduce(StepFunction[] a, StepFunction[] b) { + if (a == null) + return b; + if (b == null) + return a; + else { + StepFunction[] res = Arrays.copyOf(a, a.length); + + for (int i = 0; i < res.length; i++) + res[i] = res[i].add(b[i]); + + return res; + } + } + + /** + * Calculates threshold based on the given step function arguments and split point (specified left size). + * + * @param arguments Step function arguments. + * @param leftSize Split point (left size). + * @return Threshold. + */ + private double calculateThreshold(double[] arguments, int leftSize) { + return (arguments[leftSize] + arguments[leftSize + 1]) / 2.0; + } + + /** + * Constructs a new predicate for "then" node based on the parent node predicate and split point. + * + * @param filter Parent node predicate. + * @param splitPnt Split point. + * @return Predicate for "then" node. + */ + private TreeFilter updatePredicateForThenNode(TreeFilter filter, SplitPoint splitPnt) { + return filter.and(f -> f[splitPnt.col] > splitPnt.threshold); + } + + /** + * Constructs a new predicate for "else" node based on the parent node predicate and split point. + * + * @param filter Parent node predicate. + * @param splitPnt Split point. + * @return Predicate for "else" node. + */ + private TreeFilter updatePredicateForElseNode(TreeFilter filter, SplitPoint splitPnt) { + return filter.and(f -> f[splitPnt.col] <= splitPnt.threshold); + } + + /** + * Util class that represents split point. + */ + private static class SplitPoint> implements Serializable { + /** */ + private static final long serialVersionUID = -1758525953544425043L; + + /** Split point impurity measure value. */ + private final T val; + + /** Column. */ + private final int col; + + /** Threshold. */ + private final double threshold; + + /** + * Constructs a new instance of split point. + * + * @param val Split point impurity measure value. + * @param col Column. + * @param threshold Threshold. + */ + SplitPoint(T val, int col, double threshold) { + this.val = val; + this.col = col; + this.threshold = threshold; + } + } +} \ No newline at end of file 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 new file mode 100644 index 0000000000000..ce75190583e90 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainer.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.tree; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.tree.data.DecisionTreeData; +import org.apache.ignite.ml.tree.impurity.ImpurityMeasureCalculator; +import org.apache.ignite.ml.tree.impurity.gini.GiniImpurityMeasure; +import org.apache.ignite.ml.tree.impurity.gini.GiniImpurityMeasureCalculator; +import org.apache.ignite.ml.tree.impurity.util.StepFunctionCompressor; +import org.apache.ignite.ml.tree.leaf.MostCommonDecisionTreeLeafBuilder; + +/** + * Decision tree classifier based on distributed decision tree trainer that allows to fit trees using row-partitioned + * dataset. + */ +public class DecisionTreeClassificationTrainer extends DecisionTree { + /** + * Constructs a new decision tree classifier with default impurity function compressor. + * + * @param maxDeep Max tree deep. + * @param minImpurityDecrease Min impurity decrease. + */ + public DecisionTreeClassificationTrainer(int maxDeep, double minImpurityDecrease) { + this(maxDeep, minImpurityDecrease, null); + } + + /** + * Constructs a new instance of decision tree classifier. + * + * @param maxDeep Max tree deep. + * @param minImpurityDecrease Min impurity decrease. + */ + public DecisionTreeClassificationTrainer(int maxDeep, double minImpurityDecrease, + StepFunctionCompressor compressor) { + super(maxDeep, minImpurityDecrease, compressor, new MostCommonDecisionTreeLeafBuilder()); + } + + /** {@inheritDoc} */ + @Override ImpurityMeasureCalculator getImpurityMeasureCalculator( + Dataset dataset) { + Set labels = dataset.compute(part -> { + + if (part.getLabels() != null) { + Set list = new HashSet<>(); + + for (double lb : part.getLabels()) + list.add(lb); + + return list; + } + + return null; + }, (a, b) -> { + if (a == null) + return b; + else if (b == null) + return a; + else { + a.addAll(b); + return a; + } + }); + + Map encoder = new HashMap<>(); + + int idx = 0; + for (Double lb : labels) + encoder.put(lb, idx++); + + return new GiniImpurityMeasureCalculator(encoder); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeConditionalNode.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeConditionalNode.java new file mode 100644 index 0000000000000..98182399a5603 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeConditionalNode.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; + +/** + * Decision tree conditional (non-leaf) node. + */ +public class DecisionTreeConditionalNode implements DecisionTreeNode { + /** */ + private static final long serialVersionUID = 981630737007982172L; + + /** Column of the value to be tested. */ + private final int col; + + /** Threshold. */ + private final double threshold; + + /** Node that will be used in case tested value is greater then threshold. */ + private final DecisionTreeNode thenNode; + + /** Node that will be used in case tested value is not greater then threshold. */ + private final DecisionTreeNode elseNode; + + /** + * Constructs a new instance of decision tree conditional node. + * + * @param col Column of the value to be tested. + * @param threshold Threshold. + * @param thenNode Node that will be used in case tested value is greater then threshold. + * @param elseNode Node that will be used in case tested value is not greater then threshold. + */ + DecisionTreeConditionalNode(int col, double threshold, DecisionTreeNode thenNode, DecisionTreeNode elseNode) { + this.col = col; + this.threshold = threshold; + this.thenNode = thenNode; + this.elseNode = elseNode; + } + + /** {@inheritDoc} */ + @Override public Double apply(double[] features) { + return features[col] > threshold ? thenNode.apply(features) : elseNode.apply(features); + } + + /** */ + public int getCol() { + return col; + } + + /** */ + public double getThreshold() { + return threshold; + } + + /** */ + public DecisionTreeNode getThenNode() { + return thenNode; + } + + /** */ + public DecisionTreeNode getElseNode() { + return elseNode; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/Leaf.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeLeafNode.java similarity index 61% rename from modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/Leaf.java rename to modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeLeafNode.java index 79b441f3ac77b..4c6369ddd569f 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/Leaf.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeLeafNode.java @@ -15,35 +15,34 @@ * limitations under the License. */ -package org.apache.ignite.ml.trees.nodes; - -import org.apache.ignite.ml.math.Vector; +package org.apache.ignite.ml.tree; /** - * Terminal node of the decision tree. + * Decision tree leaf node which contains value. */ -public class Leaf implements DecisionTreeNode { - /** - * Value in subregion represented by this node. - */ +public class DecisionTreeLeafNode implements DecisionTreeNode { + /** */ + private static final long serialVersionUID = -472145568088482206L; + + /** Value of the node. */ private final double val; /** - * Construct the leaf of decision tree. + * Constructs a new decision tree leaf node. * - * @param val Value in subregion represented by this node. + * @param val Value of the node. */ - public Leaf(double val) { + public DecisionTreeLeafNode(double val) { this.val = val; } - /** - * Return value in subregion represented by this node. - * - * @param v Vector. - * @return Value in subregion represented by this node. - */ - @Override public double process(Vector v) { + /** {@inheritDoc} */ + @Override public Double apply(double[] doubles) { + return val; + } + + /** */ + public double getVal() { return val; } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeNode.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeNode.java new file mode 100644 index 0000000000000..94878eb4610e6 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeNode.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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; + +import org.apache.ignite.ml.Model; + +/** + * Base interface for decision tree nodes. + */ +public interface DecisionTreeNode extends Model { +} 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 new file mode 100644 index 0000000000000..2bf09d32be39c --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainer.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.ml.tree; + +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.tree.data.DecisionTreeData; +import org.apache.ignite.ml.tree.impurity.ImpurityMeasureCalculator; +import org.apache.ignite.ml.tree.impurity.mse.MSEImpurityMeasure; +import org.apache.ignite.ml.tree.impurity.mse.MSEImpurityMeasureCalculator; +import org.apache.ignite.ml.tree.impurity.util.StepFunctionCompressor; +import org.apache.ignite.ml.tree.leaf.MeanDecisionTreeLeafBuilder; + +/** + * Decision tree regressor based on distributed decision tree trainer that allows to fit trees using row-partitioned + * dataset. + */ +public class DecisionTreeRegressionTrainer extends DecisionTree { + /** + * Constructs a new decision tree regressor with default impurity function compressor. + * + * @param maxDeep Max tree deep. + * @param minImpurityDecrease Min impurity decrease. + */ + public DecisionTreeRegressionTrainer(int maxDeep, double minImpurityDecrease) { + this(maxDeep, minImpurityDecrease, null); + } + + /** + * Constructs a new decision tree regressor. + * + * @param maxDeep Max tree deep. + * @param minImpurityDecrease Min impurity decrease. + */ + public DecisionTreeRegressionTrainer(int maxDeep, double minImpurityDecrease, + StepFunctionCompressor compressor) { + super(maxDeep, minImpurityDecrease, compressor, new MeanDecisionTreeLeafBuilder()); + } + + /** {@inheritDoc} */ + @Override ImpurityMeasureCalculator getImpurityMeasureCalculator( + Dataset dataset) { + return new MSEImpurityMeasureCalculator(); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/DecisionTreeNode.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/TreeFilter.java similarity index 59% rename from modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/DecisionTreeNode.java rename to modules/ml/src/main/java/org/apache/ignite/ml/tree/TreeFilter.java index d31623d42a002..3e4dc00da7270 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/DecisionTreeNode.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/TreeFilter.java @@ -15,19 +15,24 @@ * limitations under the License. */ -package org.apache.ignite.ml.trees.nodes; +package org.apache.ignite.ml.tree; -import org.apache.ignite.ml.math.Vector; +import java.io.Serializable; +import java.util.Objects; +import java.util.function.Predicate; /** - * Node of decision tree. + * Predicate used to define objects that placed in decision tree node. */ -public interface DecisionTreeNode { +public interface TreeFilter extends Predicate, Serializable { /** - * Assign the double value to the given vector. + * Returns a composed predicate. * - * @param v Vector. - * @return Value assigned to the given vector. + * @param other Predicate that will be logically-ANDed with this predicate. + * @return Returns a composed predicate */ - double process(Vector v); + default TreeFilter and(TreeFilter other) { + Objects.requireNonNull(other); + return (t) -> test(t) && other.test(t); + } } 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 new file mode 100644 index 0000000000000..34deb46cc9e91 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/data/DecisionTreeData.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.ml.tree.data; + +import org.apache.ignite.ml.tree.TreeFilter; + +/** + * A partition {@code data} of the containing matrix of features and vector of labels stored in heap. + */ +public class DecisionTreeData implements AutoCloseable { + /** Matrix with features. */ + private final double[][] features; + + /** Vector with labels. */ + private final double[] labels; + + /** + * Constructs a new instance of decision tree data. + * + * @param features Matrix with features. + * @param labels Vector with labels. + */ + public DecisionTreeData(double[][] features, double[] labels) { + assert features.length == labels.length : "Features and labels have to be the same length"; + + this.features = features; + this.labels = labels; + } + + /** + * Filters objects and returns only data that passed filter. + * + * @param filter Filter. + * @return Data passed filter. + */ + public DecisionTreeData filter(TreeFilter filter) { + int size = 0; + + for (int i = 0; i < features.length; i++) + if (filter.test(features[i])) + size++; + + double[][] newFeatures = new double[size][]; + double[] newLabels = new double[size]; + + int ptr = 0; + + for (int i = 0; i < features.length; i++) { + if (filter.test(features[i])) { + newFeatures[ptr] = features[i]; + newLabels[ptr] = labels[i]; + + ptr++; + } + } + + return new DecisionTreeData(newFeatures, newLabels); + } + + /** + * Sorts data by specified column in ascending order. + * + * @param col Column. + */ + public void sort(int col) { + sort(col, 0, features.length - 1); + } + + /** */ + private void sort(int col, int from, int to) { + if (from < to) { + double pivot = features[(from + to) / 2][col]; + + int i = from, j = to; + + while (i <= j) { + while (features[i][col] < pivot) i++; + while (features[j][col] > pivot) j--; + + if (i <= j) { + double[] tmpFeature = features[i]; + features[i] = features[j]; + features[j] = tmpFeature; + + double tmpLb = labels[i]; + labels[i] = labels[j]; + labels[j] = tmpLb; + + i++; + j--; + } + } + + sort(col, from, j); + sort(col, i, to); + } + } + + /** */ + 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/tree/data/DecisionTreeDataBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/data/DecisionTreeDataBuilder.java new file mode 100644 index 0000000000000..67109ae9bc81e --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/data/DecisionTreeDataBuilder.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.tree.data; + +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; + +/** + * 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 DecisionTreeDataBuilder + implements PartitionDataBuilder { + /** */ + private static final long serialVersionUID = 3678784980215216039L; + + /** 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 DecisionTreeDataBuilder(IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { + this.featureExtractor = featureExtractor; + this.lbExtractor = lbExtractor; + } + + /** {@inheritDoc} */ + @Override public DecisionTreeData 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()); + labels[ptr] = lbExtractor.apply(entry.getKey(), entry.getValue()); + + ptr++; + } + + return new DecisionTreeData(features, labels); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/data/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/data/package-info.java new file mode 100644 index 0000000000000..192b07f9e0cbe --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/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. + */ + +/** + * + * Contains data and data builder required for decision tree trainers built on top of partition based dataset. + */ +package org.apache.ignite.ml.tree.data; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/ImpurityMeasure.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/ImpurityMeasure.java new file mode 100644 index 0000000000000..7ad2b80deaf43 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/ImpurityMeasure.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.impurity; + +import java.io.Serializable; + +/** + * Base interface for impurity measures that can be used in distributed decision tree algorithm. + * + * @param Type of this impurity measure. + */ +public interface ImpurityMeasure> extends Comparable, Serializable { + /** + * Calculates impurity measure as a single double value. + * + * @return Impurity measure value. + */ + public double impurity(); + + /** + * Adds the given impurity to this. + * + * @param measure Another impurity. + * @return Sum of this and the given impurity. + */ + public T add(T measure); + + /** + * Subtracts the given impurity for this. + * + * @param measure Another impurity. + * @return Difference of this and the given impurity. + */ + public T subtract(T measure); + + /** {@inheritDoc} */ + default public int compareTo(T o) { + return Double.compare(impurity(), o.impurity()); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/ImpurityMeasureCalculator.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/ImpurityMeasureCalculator.java new file mode 100644 index 0000000000000..2b69356841572 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/ImpurityMeasureCalculator.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.impurity; + +import java.io.Serializable; +import org.apache.ignite.ml.tree.data.DecisionTreeData; +import org.apache.ignite.ml.tree.impurity.util.StepFunction; + +/** + * Base interface for impurity measure calculators that calculates all impurity measures required to find a best split. + * + * @param Type of impurity measure. + */ +public interface ImpurityMeasureCalculator> extends Serializable { + /** + * Calculates all impurity measures required required to find a best split and returns them as an array of + * {@link StepFunction} (for every column). + * + * @param data Features and labels. + * @return Impurity measures as an array of {@link StepFunction} (for every column). + */ + public StepFunction[] calculate(DecisionTreeData data); +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/gini/GiniImpurityMeasure.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/gini/GiniImpurityMeasure.java new file mode 100644 index 0000000000000..817baf5da1713 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/gini/GiniImpurityMeasure.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.impurity.gini; + +import org.apache.ignite.ml.tree.impurity.ImpurityMeasure; + +/** + * Gini impurity measure which is calculated the following way: + * {@code \-frac{1}{L}\sum_{i=1}^{s}l_i^2 - \frac{1}{R}\sum_{i=s+1}^{n}r_i^2}. + */ +public class GiniImpurityMeasure implements ImpurityMeasure { + /** */ + private static final long serialVersionUID = 5338129703395229970L; + + /** Number of elements of each type in the left part. */ + private final long[] left; + + /** Number of elements of each type in the right part. */ + private final long[] right; + + /** + * Constructs a new instance of Gini impurity measure. + * + * @param left Number of elements of each type in the left part. + * @param right Number of elements of each type in the right part. + */ + GiniImpurityMeasure(long[] left, long[] right) { + assert left.length == right.length : "Left and right parts have to be the same length"; + + this.left = left; + this.right = right; + } + + /** {@inheritDoc} */ + @Override public double impurity() { + long leftCnt = 0; + long rightCnt = 0; + + double leftImpurity = 0; + double rightImpurity = 0; + + for (long e : left) + leftCnt += e; + + for (long e : right) + rightCnt += e; + + if (leftCnt > 0) + for (long e : left) + leftImpurity += Math.pow(e, 2) / leftCnt; + + if (rightCnt > 0) + for (long e : right) + rightImpurity += Math.pow(e, 2) / rightCnt; + + return -(leftImpurity + rightImpurity); + } + + /** {@inheritDoc} */ + @Override public GiniImpurityMeasure add(GiniImpurityMeasure b) { + assert left.length == b.left.length : "Subtracted measure has to have length " + left.length; + assert left.length == b.right.length : "Subtracted measure has to have length " + left.length; + + long[] leftRes = new long[left.length]; + long[] rightRes = new long[left.length]; + + for (int i = 0; i < left.length; i++) { + leftRes[i] = left[i] + b.left[i]; + rightRes[i] = right[i] + b.right[i]; + } + + return new GiniImpurityMeasure(leftRes, rightRes); + } + + /** {@inheritDoc} */ + @Override public GiniImpurityMeasure subtract(GiniImpurityMeasure b) { + assert left.length == b.left.length : "Subtracted measure has to have length " + left.length; + assert left.length == b.right.length : "Subtracted measure has to have length " + left.length; + + long[] leftRes = new long[left.length]; + long[] rightRes = new long[left.length]; + + for (int i = 0; i < left.length; i++) { + leftRes[i] = left[i] - b.left[i]; + rightRes[i] = right[i] - b.right[i]; + } + + return new GiniImpurityMeasure(leftRes, rightRes); + } + + /** */ + public long[] getLeft() { + return left; + } + + /** */ + public long[] getRight() { + return right; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/gini/GiniImpurityMeasureCalculator.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/gini/GiniImpurityMeasureCalculator.java new file mode 100644 index 0000000000000..0dd0a1049e80e --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/gini/GiniImpurityMeasureCalculator.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.impurity.gini; + +import java.util.Arrays; +import java.util.Map; +import org.apache.ignite.ml.tree.data.DecisionTreeData; +import org.apache.ignite.ml.tree.impurity.ImpurityMeasureCalculator; +import org.apache.ignite.ml.tree.impurity.util.StepFunction; + +/** + * Gini impurity measure calculator. + */ +public class GiniImpurityMeasureCalculator implements ImpurityMeasureCalculator { + /** */ + private static final long serialVersionUID = -522995134128519679L; + + /** Label encoder which defines integer value for every label class. */ + private final Map lbEncoder; + + /** + * Constructs a new instance of Gini impurity measure calculator. + * + * @param lbEncoder Label encoder which defines integer value for every label class. + */ + public GiniImpurityMeasureCalculator(Map lbEncoder) { + this.lbEncoder = lbEncoder; + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override public StepFunction[] calculate(DecisionTreeData data) { + double[][] features = data.getFeatures(); + double[] labels = data.getLabels(); + + if (features.length > 0) { + StepFunction[] res = new StepFunction[features[0].length]; + + for (int col = 0; col < res.length; col++) { + data.sort(col); + + double[] x = new double[features.length + 1]; + GiniImpurityMeasure[] y = new GiniImpurityMeasure[features.length + 1]; + + int xPtr = 0, yPtr = 0; + + long[] left = new long[lbEncoder.size()]; + long[] right = new long[lbEncoder.size()]; + + for (int i = 0; i < labels.length; i++) + right[getLabelCode(labels[i])]++; + + x[xPtr++] = Double.NEGATIVE_INFINITY; + y[yPtr++] = new GiniImpurityMeasure( + Arrays.copyOf(left, left.length), + Arrays.copyOf(right, right.length) + ); + + for (int i = 0; i < features.length; i++) { + left[getLabelCode(labels[i])]++; + right[getLabelCode(labels[i])]--; + + if (i < (features.length - 1) && features[i + 1][col] == features[i][col]) + continue; + + x[xPtr++] = features[i][col]; + y[yPtr++] = new GiniImpurityMeasure( + Arrays.copyOf(left, left.length), + Arrays.copyOf(right, right.length) + ); + } + + res[col] = new StepFunction<>(Arrays.copyOf(x, xPtr), Arrays.copyOf(y, yPtr)); + } + + return res; + } + + return null; + } + + /** + * Returns label code. + * + * @param lb Label. + * @return Label code. + */ + int getLabelCode(double lb) { + Integer code = lbEncoder.get(lb); + + assert code != null : "Can't find code for label " + lb; + + return code; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/gini/package-info.java similarity index 89% rename from modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/package-info.java rename to modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/gini/package-info.java index d6deb9d6e4fbc..d14cd9241a2cd 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/package-info.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/gini/package-info.java @@ -17,6 +17,6 @@ /** * - * Contains classes representing decision tree nodes. + * Contains Gini impurity measure and calculator. */ -package org.apache.ignite.ml.trees.nodes; \ No newline at end of file +package org.apache.ignite.ml.tree.impurity.gini; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/mse/MSEImpurityMeasure.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/mse/MSEImpurityMeasure.java new file mode 100644 index 0000000000000..3fc85153c34f7 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/mse/MSEImpurityMeasure.java @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.tree.impurity.mse; + +import org.apache.ignite.ml.tree.impurity.ImpurityMeasure; + +/** + * Mean squared error (variance) impurity measure which is calculated the following way: + * {@code \frac{1}{L}\sum_{i=0}^{n}(y_i - \mu)^2}. + */ +public class MSEImpurityMeasure implements ImpurityMeasure { + /** */ + private static final long serialVersionUID = 4536394578628409689L; + + /** Sum of all elements in the left part. */ + private final double leftY; + + /** Sum of all squared elements in the left part. */ + private final double leftY2; + + /** Number of elements in the left part. */ + private final long leftCnt; + + /** Sum of all elements in the right part. */ + private final double rightY; + + /** Sum of all squared elements in the right part. */ + private final double rightY2; + + /** Number of elements in the right part. */ + private final long rightCnt; + + /** + * Constructs a new instance of mean squared error (variance) impurity measure. + * + * @param leftY Sum of all elements in the left part. + * @param leftY2 Sum of all squared elements in the left part. + * @param leftCnt Number of elements in the left part. + * @param rightY Sum of all elements in the right part. + * @param rightY2 Sum of all squared elements in the right part. + * @param rightCnt Number of elements in the right part. + */ + public MSEImpurityMeasure(double leftY, double leftY2, long leftCnt, double rightY, double rightY2, long rightCnt) { + this.leftY = leftY; + this.leftY2 = leftY2; + this.leftCnt = leftCnt; + this.rightY = rightY; + this.rightY2 = rightY2; + this.rightCnt = rightCnt; + } + + /** {@inheritDoc} */ + @Override public double impurity() { + double impurity = 0; + + if (leftCnt > 0) + impurity += leftY2 - 2.0 * leftY / leftCnt * leftY + Math.pow(leftY / leftCnt, 2) * leftCnt; + + if (rightCnt > 0) + impurity += rightY2 - 2.0 * rightY / rightCnt * rightY + Math.pow(rightY / rightCnt, 2) * rightCnt; + + return impurity; + } + + /** {@inheritDoc} */ + @Override public MSEImpurityMeasure add(MSEImpurityMeasure b) { + return new MSEImpurityMeasure( + leftY + b.leftY, + leftY2 + b.leftY2, + leftCnt + b.leftCnt, + rightY + b.rightY, + rightY2 + b.rightY2, + rightCnt + b.rightCnt + ); + } + + /** {@inheritDoc} */ + @Override public MSEImpurityMeasure subtract(MSEImpurityMeasure b) { + return new MSEImpurityMeasure( + leftY - b.leftY, + leftY2 - b.leftY2, + leftCnt - b.leftCnt, + rightY - b.rightY, + rightY2 - b.rightY2, + rightCnt - b.rightCnt + ); + } + + /** */ + public double getLeftY() { + return leftY; + } + + /** */ + public double getLeftY2() { + return leftY2; + } + + /** */ + public long getLeftCnt() { + return leftCnt; + } + + /** */ + public double getRightY() { + return rightY; + } + + /** */ + public double getRightY2() { + return rightY2; + } + + /** */ + public long getRightCnt() { + return rightCnt; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/mse/MSEImpurityMeasureCalculator.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/mse/MSEImpurityMeasureCalculator.java new file mode 100644 index 0000000000000..cb5019c4e179d --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/mse/MSEImpurityMeasureCalculator.java @@ -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. + */ + +package org.apache.ignite.ml.tree.impurity.mse; + +import org.apache.ignite.ml.tree.data.DecisionTreeData; +import org.apache.ignite.ml.tree.impurity.ImpurityMeasureCalculator; +import org.apache.ignite.ml.tree.impurity.util.StepFunction; + +/** + * Meas squared error (variance) impurity measure calculator. + */ +public class MSEImpurityMeasureCalculator implements ImpurityMeasureCalculator { + /** */ + private static final long serialVersionUID = 288747414953756824L; + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override public StepFunction[] calculate(DecisionTreeData data) { + double[][] features = data.getFeatures(); + double[] labels = data.getLabels(); + + if (features.length > 0) { + StepFunction[] res = new StepFunction[features[0].length]; + + for (int col = 0; col < res.length; col++) { + data.sort(col); + + double[] x = new double[features.length + 1]; + MSEImpurityMeasure[] y = new MSEImpurityMeasure[features.length + 1]; + + x[0] = Double.NEGATIVE_INFINITY; + + for (int leftSize = 0; leftSize <= features.length; leftSize++) { + double leftY = 0; + double leftY2 = 0; + double rightY = 0; + double rightY2 = 0; + + for (int i = 0; i < leftSize; i++) { + leftY += labels[i]; + leftY2 += Math.pow(labels[i], 2); + } + + for (int i = leftSize; i < features.length; i++) { + rightY += labels[i]; + rightY2 += Math.pow(labels[i], 2); + } + + if (leftSize < features.length) + x[leftSize + 1] = features[leftSize][col]; + + y[leftSize] = new MSEImpurityMeasure( + leftY, leftY2, leftSize, rightY, rightY2, features.length - leftSize + ); + } + + res[col] = new StepFunction<>(x, y); + } + + return res; + } + + return null; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/models/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/mse/package-info.java similarity index 88% rename from modules/ml/src/main/java/org/apache/ignite/ml/trees/models/package-info.java rename to modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/mse/package-info.java index ce8418e87cf39..23ec4e03f09b7 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/models/package-info.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/mse/package-info.java @@ -17,6 +17,6 @@ /** * - * Contains decision tree models. + * Contains mean squared error impurity measure and calculator. */ -package org.apache.ignite.ml.trees.models; \ No newline at end of file +package org.apache.ignite.ml.tree.impurity.mse; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/package-info.java new file mode 100644 index 0000000000000..4155593aae3d7 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/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. + */ + +/** + * + * Root package for decision tree impurity measures and calculators. + */ +package org.apache.ignite.ml.tree.impurity; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/util/SimpleStepFunctionCompressor.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/util/SimpleStepFunctionCompressor.java new file mode 100644 index 0000000000000..2418571089ba6 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/util/SimpleStepFunctionCompressor.java @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.impurity.util; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import org.apache.ignite.ml.tree.impurity.ImpurityMeasure; + +/** + * Simple step function compressor. + * + * @param Type of step function values. + */ +public class SimpleStepFunctionCompressor> implements StepFunctionCompressor { + /** */ + private static final long serialVersionUID = -3231787633598409157L; + + /** Min size of step function to be compressed. */ + private final int minSizeToBeCompressed; + + /** In case of compression min impurity increase that will be recorded. */ + private final double minImpurityIncreaseForRecord; + + /** In case of compression min impurity decrease that will be recorded. */ + private final double minImpurityDecreaseForRecord; + + /** + * Constructs a new instance of simple step function compressor with default parameters. + */ + public SimpleStepFunctionCompressor() { + this(10, 0.1, 0.05); + } + + /** + * Constructs a new instance of simple step function compressor. + * + * @param minSizeToBeCompressed Min size of step function to be compressed. + * @param minImpurityIncreaseForRecord In case of compression min impurity increase that will be recorded. + * @param minImpurityDecreaseForRecord In case of compression min impurity decrease that will be recorded. + */ + public SimpleStepFunctionCompressor(int minSizeToBeCompressed, double minImpurityIncreaseForRecord, + double minImpurityDecreaseForRecord) { + this.minSizeToBeCompressed = minSizeToBeCompressed; + this.minImpurityIncreaseForRecord = minImpurityIncreaseForRecord; + this.minImpurityDecreaseForRecord = minImpurityDecreaseForRecord; + } + + /** {@inheritDoc} */ + @Override public StepFunction compress(StepFunction function) { + double[] arguments = function.getX(); + T[] values = function.getY(); + + if (arguments.length >= minSizeToBeCompressed) { + List points = new ArrayList<>(); + + for (int i = 0; i < arguments.length; i++) + points.add(new StepFunctionPoint(arguments[i], values[i])); + + points = compress(points); + + double[] resX = new double[points.size()]; + T[] resY = Arrays.copyOf(values, points.size()); + + for (int i = 0; i < points.size(); i++) { + StepFunctionPoint pnt = points.get(i); + resX[i] = pnt.x; + resY[i] = pnt.y; + } + + return new StepFunction<>(resX, resY); + } + + return function; + } + + /** + * Compresses list of step function points. + * + * @param points Step function points. + * @return Compressed step function points. + */ + private List compress(List points) { + List res = new ArrayList<>(); + + double minImpurity = Double.MAX_VALUE, maxImpurity = Double.MIN_VALUE; + for (int i = 0; i < points.size(); i++) { + StepFunctionPoint pnt = points.get(i); + + double impurity = pnt.y.impurity(); + + if (impurity > maxImpurity) + maxImpurity = impurity; + + if (impurity < minImpurity) + minImpurity = impurity; + } + + Double prev = null; + for (StepFunctionPoint pnt : points) { + double impurity = (pnt.y.impurity() - minImpurity) / (maxImpurity - minImpurity); + if (prev == null || + prev - impurity >= minImpurityDecreaseForRecord || + impurity - prev >= minImpurityIncreaseForRecord) { + prev = impurity; + res.add(pnt); + } + } + + return res; + } + + /** + * Util class that represents step function point. + */ + private class StepFunctionPoint { + /** Argument of the step start. */ + private final double x; + + /** Value of the step. */ + private final T y; + + /** + * Constructs a new instance of util class that represents step function point. + * + * @param x Argument of the step start. + * @param y Value of the step. + */ + StepFunctionPoint(double x, T y) { + this.x = x; + this.y = y; + } + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/util/StepFunction.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/util/StepFunction.java new file mode 100644 index 0000000000000..431503d3d11de --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/util/StepFunction.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.tree.impurity.util; + +import java.util.Arrays; +import org.apache.ignite.ml.tree.impurity.ImpurityMeasure; + +/** + * Step function described by {@code x} and {@code y} points. + * + * @param Type of function values. + */ +public class StepFunction> { + /** Argument of every steps start. Should be ascendingly sorted all the time. */ + private final double[] x; + + /** Value of every step. */ + private final T[] y; + + /** + * Constructs a new instance of step function. + * + * @param x Argument of every steps start. + * @param y Value of every step. + */ + public StepFunction(double[] x, T[] y) { + assert x.length == y.length : "Argument and value arrays have to be the same length"; + + this.x = x; + this.y = y; + + sort(x, y, 0, x.length - 1); + } + + /** + * Adds the given step function to this. + * + * @param b Another step function. + * @return Sum of this and the given function. + */ + public StepFunction add(StepFunction b) { + int resSize = 0, leftPtr = 0, rightPtr = 0; + double previousPnt = 0; + + while (leftPtr < x.length || rightPtr < b.x.length) { + if (rightPtr >= b.x.length || (leftPtr < x.length && x[leftPtr] < b.x[rightPtr])) { + if (resSize == 0 || x[leftPtr] != previousPnt) { + previousPnt = x[leftPtr]; + resSize++; + } + + leftPtr++; + } + else { + if (resSize == 0 || b.x[rightPtr] != previousPnt) { + previousPnt = b.x[rightPtr]; + resSize++; + } + + rightPtr++; + } + } + + double[] resX = new double[resSize]; + T[] resY = Arrays.copyOf(y, resSize); + + leftPtr = 0; + rightPtr = 0; + + for (int i = 0; leftPtr < x.length || rightPtr < b.x.length; i++) { + if (rightPtr >= b.x.length || (leftPtr < x.length && x[leftPtr] < b.x[rightPtr])) { + boolean override = i > 0 && x[leftPtr] == resX[i - 1]; + int target = override ? i - 1 : i; + + resY[target] = override ? resY[target] : null; + resY[target] = i > 0 ? resY[i - 1] : null; + resY[target] = resY[target] == null ? y[leftPtr] : resY[target].add(y[leftPtr]); + + if (leftPtr > 0) + resY[target] = resY[target].subtract(y[leftPtr - 1]); + + resX[target] = x[leftPtr]; + i = target; + + leftPtr++; + } + else { + boolean override = i > 0 && b.x[rightPtr] == resX[i - 1]; + int target = override ? i - 1 : i; + + resY[target] = override ? resY[target] : null; + resY[target] = i > 0 ? resY[i - 1] : null; + + resY[target] = resY[target] == null ? b.y[rightPtr] : resY[target].add(b.y[rightPtr]); + + if (rightPtr > 0) + resY[target] = resY[target].subtract(b.y[rightPtr - 1]); + + resX[target] = b.x[rightPtr]; + i = target; + + rightPtr++; + } + } + + return new StepFunction<>(resX, resY); + } + + /** */ + private void sort(double[] x, T[] y, int from, int to) { + if (from < to) { + double pivot = x[(from + to) / 2]; + + int i = from, j = to; + while (i <= j) { + while (x[i] < pivot) i++; + while (x[j] > pivot) j--; + + if (i <= j) { + double tmpX = x[i]; + x[i] = x[j]; + x[j] = tmpX; + + T tmpY = y[i]; + y[i] = y[j]; + y[j] = tmpY; + + i++; + j--; + } + } + + sort(x, y, from, j); + sort(x, y, i, to); + } + } + + /** */ + public double[] getX() { + return x; + } + + /** */ + public T[] getY() { + return y; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/util/StepFunctionCompressor.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/util/StepFunctionCompressor.java new file mode 100644 index 0000000000000..41baa292d4a33 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/util/StepFunctionCompressor.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.impurity.util; + +import java.io.Serializable; +import java.util.Arrays; +import org.apache.ignite.ml.tree.impurity.ImpurityMeasure; + +/** + * Base interface for step function compressors which reduces step function size. + * + * @param Type of step function value. + */ +public interface StepFunctionCompressor> extends Serializable { + /** + * Compresses the given step function. + * + * @param function Step function. + * @return Compressed step function. + */ + public StepFunction compress(StepFunction function); + + /** + * Compresses every step function in the given array. + * + * @param functions Array of step functions. + * @return Arrays of compressed step function. + */ + default public StepFunction[] compress(StepFunction[] functions) { + if (functions == null) + return null; + + StepFunction[] res = Arrays.copyOf(functions, functions.length); + + for (int i = 0; i < res.length; i++) + res[i] = compress(res[i]); + + return res; + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/util/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/util/package-info.java new file mode 100644 index 0000000000000..99df6180e5ddb --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/impurity/util/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 util classes used in decision tree impurity calculators. + */ +package org.apache.ignite.ml.tree.impurity.util; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/models/DecisionTreeModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/leaf/DecisionTreeLeafBuilder.java similarity index 55% rename from modules/ml/src/main/java/org/apache/ignite/ml/trees/models/DecisionTreeModel.java rename to modules/ml/src/main/java/org/apache/ignite/ml/tree/leaf/DecisionTreeLeafBuilder.java index 572e64a332184..976e30d34be4c 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/models/DecisionTreeModel.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/leaf/DecisionTreeLeafBuilder.java @@ -15,30 +15,24 @@ * limitations under the License. */ -package org.apache.ignite.ml.trees.models; +package org.apache.ignite.ml.tree.leaf; -import org.apache.ignite.ml.Model; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.trees.nodes.DecisionTreeNode; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.tree.DecisionTreeLeafNode; +import org.apache.ignite.ml.tree.TreeFilter; +import org.apache.ignite.ml.tree.data.DecisionTreeData; /** - * Model for decision tree. + * Base interface for decision tree leaf builders. */ -public class DecisionTreeModel implements Model { - /** Root node of the decision tree. */ - private final DecisionTreeNode root; - +public interface DecisionTreeLeafBuilder { /** - * Construct decision tree model. + * Creates new leaf node for given dataset and node predicate. * - * @param root Root of decision tree. + * @param dataset Dataset. + * @param pred Node predicate. + * @return Leaf node. */ - public DecisionTreeModel(DecisionTreeNode root) { - this.root = root; - } - - /** {@inheritDoc} */ - @Override public Double apply(Vector val) { - return root.process(val); - } + public DecisionTreeLeafNode createLeafNode(Dataset dataset, TreeFilter pred); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/leaf/MeanDecisionTreeLeafBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/leaf/MeanDecisionTreeLeafBuilder.java new file mode 100644 index 0000000000000..2e05215b9dd7c --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/leaf/MeanDecisionTreeLeafBuilder.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.tree.leaf; + +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.tree.DecisionTreeLeafNode; +import org.apache.ignite.ml.tree.TreeFilter; +import org.apache.ignite.ml.tree.data.DecisionTreeData; + +/** + * Decision tree leaf node builder that chooses mean value as a leaf value. + */ +public class MeanDecisionTreeLeafBuilder implements DecisionTreeLeafBuilder { + /** {@inheritDoc} */ + @Override public DecisionTreeLeafNode createLeafNode(Dataset dataset, + TreeFilter pred) { + double[] aa = dataset.compute(part -> { + double mean = 0; + int cnt = 0; + + for (int i = 0; i < part.getFeatures().length; i++) { + if (pred.test(part.getFeatures()[i])) { + mean += part.getLabels()[i]; + cnt++; + } + } + + if (cnt != 0) { + mean = mean / cnt; + + return new double[] {mean, cnt}; + } + + return null; + }, this::reduce); + + return aa != null ? new DecisionTreeLeafNode(aa[0]) : null; + } + + /** */ + private double[] reduce(double[] a, double[] b) { + if (a == null) + return b; + else if (b == null) + return a; + else { + double aMean = a[0]; + double aCnt = a[1]; + double bMean = b[0]; + double bCnt = b[1]; + + double mean = (aMean * aCnt + bMean * bCnt) / (aCnt + bCnt); + + return new double[] {mean, aCnt + bCnt}; + } + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/tree/leaf/MostCommonDecisionTreeLeafBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/leaf/MostCommonDecisionTreeLeafBuilder.java new file mode 100644 index 0000000000000..1e8b9414d740e --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/leaf/MostCommonDecisionTreeLeafBuilder.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.leaf; + +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.tree.DecisionTreeLeafNode; +import org.apache.ignite.ml.tree.TreeFilter; +import org.apache.ignite.ml.tree.data.DecisionTreeData; + +/** + * Decision tree leaf node builder that chooses most common value as a leaf node value. + */ +public class MostCommonDecisionTreeLeafBuilder implements DecisionTreeLeafBuilder { + /** {@inheritDoc} */ + @Override public DecisionTreeLeafNode createLeafNode(Dataset dataset, + TreeFilter pred) { + Map cnt = dataset.compute(part -> { + + if (part.getFeatures() != null) { + Map map = new HashMap<>(); + + for (int i = 0; i < part.getFeatures().length; i++) { + if (pred.test(part.getFeatures()[i])) { + double lb = part.getLabels()[i]; + + if (map.containsKey(lb)) + map.put(lb, map.get(lb) + 1); + else + map.put(lb, 1); + } + } + + return map; + } + + return null; + }, this::reduce); + + double bestVal = 0; + int bestCnt = -1; + + for (Map.Entry e : cnt.entrySet()) { + if (e.getValue() > bestCnt) { + bestCnt = e.getValue(); + bestVal = e.getKey(); + } + } + + return new DecisionTreeLeafNode(bestVal); + } + + /** */ + private Map reduce(Map a, Map b) { + if (a == null) + return b; + else if (b == null) + return a; + else { + for (Map.Entry e : b.entrySet()) { + if (a.containsKey(e.getKey())) + a.put(e.getKey(), a.get(e.getKey()) + e.getValue()); + else + a.put(e.getKey(), e.getValue()); + } + return a; + } + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/regcalcs/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/leaf/package-info.java similarity index 90% rename from modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/regcalcs/package-info.java rename to modules/ml/src/main/java/org/apache/ignite/ml/tree/leaf/package-info.java index e8edd8f16f0bb..26ec67dd97af2 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/regcalcs/package-info.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/leaf/package-info.java @@ -17,6 +17,6 @@ /** * - * Region calculators. + * Root package for decision trees leaf builders. */ -package org.apache.ignite.ml.trees.trainers.columnbased.regcalcs; \ No newline at end of file +package org.apache.ignite.ml.tree.leaf; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/tree/package-info.java similarity index 92% rename from modules/ml/src/main/java/org/apache/ignite/ml/trees/package-info.java rename to modules/ml/src/main/java/org/apache/ignite/ml/tree/package-info.java index b07ba4acb360d..660f3f3c0197e 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/package-info.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/tree/package-info.java @@ -17,6 +17,6 @@ /** * - * Contains decision tree algorithms. + * Root package for decision trees. */ -package org.apache.ignite.ml.trees; \ No newline at end of file +package org.apache.ignite.ml.tree; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/CategoricalRegionInfo.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/CategoricalRegionInfo.java deleted file mode 100644 index 3ae474e12e66e..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/CategoricalRegionInfo.java +++ /dev/null @@ -1,72 +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.trees; - -import java.io.Externalizable; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.util.BitSet; - -/** - * Information about categorical region. - */ -public class CategoricalRegionInfo extends RegionInfo implements Externalizable { - /** - * Bitset representing categories of this region. - */ - private BitSet cats; - - /** - * @param impurity Impurity of region. - * @param cats Bitset representing categories of this region. - */ - public CategoricalRegionInfo(double impurity, BitSet cats) { - super(impurity); - - this.cats = cats; - } - - /** - * No-op constructor for serialization/deserialization. - */ - public CategoricalRegionInfo() { - // No-op - } - - /** - * Get bitset representing categories of this region. - * - * @return Bitset representing categories of this region. - */ - public BitSet cats() { - return cats; - } - - /** {@inheritDoc} */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - super.writeExternal(out); - out.writeObject(cats); - } - - /** {@inheritDoc} */ - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - super.readExternal(in); - cats = (BitSet)in.readObject(); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/CategoricalSplitInfo.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/CategoricalSplitInfo.java deleted file mode 100644 index 94cb1e8be9df4..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/CategoricalSplitInfo.java +++ /dev/null @@ -1,68 +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.trees; - -import java.util.BitSet; -import org.apache.ignite.ml.trees.nodes.CategoricalSplitNode; -import org.apache.ignite.ml.trees.nodes.SplitNode; -import org.apache.ignite.ml.trees.trainers.columnbased.vectors.SplitInfo; - -/** - * Information about split of categorical feature. - * - * @param Class representing information of left and right subregions. - */ -public class CategoricalSplitInfo extends SplitInfo { - /** Bitset indicating which vectors are assigned to left subregion. */ - private final BitSet bs; - - /** - * @param regionIdx Index of region which is split. - * @param leftData Data of left subregion. - * @param rightData Data of right subregion. - * @param bs Bitset indicating which vectors are assigned to left subregion. - */ - public CategoricalSplitInfo(int regionIdx, D leftData, D rightData, - BitSet bs) { - super(regionIdx, leftData, rightData); - this.bs = bs; - } - - /** {@inheritDoc} */ - @Override public SplitNode createSplitNode(int featureIdx) { - return new CategoricalSplitNode(featureIdx, bs); - } - - /** - * Get bitset indicating which vectors are assigned to left subregion. - */ - public BitSet bitSet() { - return bs; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return "CategoricalSplitInfo [" + - "infoGain=" + infoGain + - ", regionIdx=" + regionIdx + - ", leftData=" + leftData + - ", bs=" + bs + - ", rightData=" + rightData + - ']'; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/ContinuousRegionInfo.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/ContinuousRegionInfo.java deleted file mode 100644 index e98bb728c6aaa..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/ContinuousRegionInfo.java +++ /dev/null @@ -1,74 +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.trees; - -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; - -/** - * Information about region used by continuous features. - */ -public class ContinuousRegionInfo extends RegionInfo { - /** - * Count of samples in this region. - */ - private int size; - - /** - * @param impurity Impurity of the region. - * @param size Size of this region - */ - public ContinuousRegionInfo(double impurity, int size) { - super(impurity); - this.size = size; - } - - /** - * No-op constructor for serialization/deserialization. - */ - public ContinuousRegionInfo() { - // No-op - } - - /** - * Get the size of region. - */ - public int getSize() { - return size; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return "ContinuousRegionInfo [" + - "size=" + size + - ']'; - } - - /** {@inheritDoc} */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - super.writeExternal(out); - out.writeInt(size); - } - - /** {@inheritDoc} */ - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - super.readExternal(in); - size = in.readInt(); - } -} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/ContinuousSplitCalculator.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/ContinuousSplitCalculator.java deleted file mode 100644 index 3a0e9da3982ce..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/ContinuousSplitCalculator.java +++ /dev/null @@ -1,51 +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.trees; - -import java.util.stream.DoubleStream; -import org.apache.ignite.ml.trees.trainers.columnbased.vectors.SplitInfo; -import org.apache.ignite.ml.trees.trainers.columnbased.vectors.ContinuousFeatureProcessor; - -/** - * This class is used for calculation of best split by continuous feature. - * - * @param Class in which information about region will be stored. - */ -public interface ContinuousSplitCalculator { - /** - * Calculate region info 'from scratch'. - * - * @param s Stream of labels in this region. - * @param l Index of sample projection on this feature in array sorted by this projection value and intervals - * bitsets. ({@link ContinuousFeatureProcessor}). - * @return Region info. - */ - C calculateRegionInfo(DoubleStream s, int l); - - /** - * Calculate split info of best split of region given information about this region. - * - * @param sampleIndexes Indexes of samples of this region. - * @param values All values of this feature. - * @param labels All labels of this feature. - * @param regionIdx Index of region being split. - * @param data Information about region being split which can be used for computations. - * @return Information about best split of region with index given by regionIdx. - */ - SplitInfo splitRegion(Integer[] sampleIndexes, double[] values, double[] labels, int regionIdx, C data); -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/RegionInfo.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/RegionInfo.java deleted file mode 100644 index 8ec7db3713f46..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/RegionInfo.java +++ /dev/null @@ -1,62 +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.trees; - -import java.io.Externalizable; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; - -/** Class containing information about region. */ -public class RegionInfo implements Externalizable { - /** Impurity in this region. */ - private double impurity; - - /** - * @param impurity Impurity of this region. - */ - public RegionInfo(double impurity) { - this.impurity = impurity; - } - - /** - * No-op constructor for serialization/deserialization. - */ - public RegionInfo() { - // No-op - } - - /** - * Get impurity in this region. - * - * @return Impurity of this region. - */ - public double impurity() { - return impurity; - } - - /** {@inheritDoc} */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - out.writeDouble(impurity); - } - - /** {@inheritDoc} */ - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - impurity = in.readDouble(); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/CategoricalSplitNode.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/CategoricalSplitNode.java deleted file mode 100644 index cae6d4a6d2da5..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/CategoricalSplitNode.java +++ /dev/null @@ -1,50 +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.trees.nodes; - -import java.util.BitSet; -import org.apache.ignite.ml.math.Vector; - -/** - * Split node by categorical feature. - */ -public class CategoricalSplitNode extends SplitNode { - /** Bitset specifying which categories belong to left subregion. */ - private final BitSet bs; - - /** - * Construct categorical split node. - * - * @param featureIdx Index of feature by which split is done. - * @param bs Bitset specifying which categories go to the left subtree. - */ - public CategoricalSplitNode(int featureIdx, BitSet bs) { - super(featureIdx); - this.bs = bs; - } - - /** {@inheritDoc} */ - @Override public boolean goLeft(Vector v) { - return bs.get((int)v.getX(featureIdx)); - } - - /** {@inheritDoc} */ - @Override public String toString() { - return "CategoricalSplitNode [bs=" + bs + ']'; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/ContinuousSplitNode.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/ContinuousSplitNode.java deleted file mode 100644 index 285cfcd3ef735..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/ContinuousSplitNode.java +++ /dev/null @@ -1,56 +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.trees.nodes; - -import org.apache.ignite.ml.math.Vector; - -/** - * Split node representing split of continuous feature. - */ -public class ContinuousSplitNode extends SplitNode { - /** Threshold. Values which are less or equal then threshold are assigned to the left subregion. */ - private final double threshold; - - /** - * Construct ContinuousSplitNode by threshold and feature index. - * - * @param threshold Threshold. - * @param featureIdx Feature index. - */ - public ContinuousSplitNode(double threshold, int featureIdx) { - super(featureIdx); - this.threshold = threshold; - } - - /** {@inheritDoc} */ - @Override public boolean goLeft(Vector v) { - return v.getX(featureIdx) <= threshold; - } - - /** Threshold. Values which are less or equal then threshold are assigned to the left subregion. */ - public double threshold() { - return threshold; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return "ContinuousSplitNode [" + - "threshold=" + threshold + - ']'; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/SplitNode.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/SplitNode.java deleted file mode 100644 index 4c258d175b189..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/nodes/SplitNode.java +++ /dev/null @@ -1,100 +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.trees.nodes; - -import org.apache.ignite.ml.math.Vector; - -/** - * Node in decision tree representing a split. - */ -public abstract class SplitNode implements DecisionTreeNode { - /** Left subtree. */ - protected DecisionTreeNode l; - - /** Right subtree. */ - protected DecisionTreeNode r; - - /** Feature index. */ - protected final int featureIdx; - - /** - * Constructs SplitNode with a given feature index. - * - * @param featureIdx Feature index. - */ - public SplitNode(int featureIdx) { - this.featureIdx = featureIdx; - } - - /** - * Indicates if the given vector is in left subtree. - * - * @param v Vector - * @return Status of given vector being left subtree. - */ - abstract boolean goLeft(Vector v); - - /** - * Left subtree. - * - * @return Left subtree. - */ - public DecisionTreeNode left() { - return l; - } - - /** - * Right subtree. - * - * @return Right subtree. - */ - public DecisionTreeNode right() { - return r; - } - - /** - * Set the left subtree. - * - * @param n left subtree. - */ - public void setLeft(DecisionTreeNode n) { - l = n; - } - - /** - * Set the right subtree. - * - * @param n right subtree. - */ - public void setRight(DecisionTreeNode n) { - r = n; - } - - /** - * Delegates processing to subtrees. - * - * @param v Vector. - * @return Value assigned to the given vector. - */ - @Override public double process(Vector v) { - if (left() != null && goLeft(v)) - return left().process(v); - else - return right().process(v); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/BiIndex.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/BiIndex.java deleted file mode 100644 index 0d27c8a6c06f8..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/BiIndex.java +++ /dev/null @@ -1,113 +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.trees.trainers.columnbased; - -import java.io.Externalizable; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import org.apache.ignite.cache.affinity.AffinityKeyMapped; - -/** - * Class representing a simple index in 2d matrix in the form (row, col). - */ -public class BiIndex implements Externalizable { - /** Row. */ - private int row; - - /** Column. */ - @AffinityKeyMapped - private int col; - - /** - * No-op constructor for serialization/deserialization. - */ - public BiIndex() { - // No-op. - } - - /** - * Construct BiIndex from row and column. - * - * @param row Row. - * @param col Column. - */ - public BiIndex(int row, int col) { - this.row = row; - this.col = col; - } - - /** - * Returns row. - * - * @return Row. - */ - public int row() { - return row; - } - - /** - * Returns column. - * - * @return Column. - */ - public int col() { - return col; - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - - BiIndex idx = (BiIndex)o; - - if (row != idx.row) - return false; - return col == idx.col; - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - int res = row; - res = 31 * res + col; - return res; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return "BiIndex [" + - "row=" + row + - ", col=" + col + - ']'; - } - - /** {@inheritDoc} */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - out.writeInt(row); - out.writeInt(col); - } - - /** {@inheritDoc} */ - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - row = in.readInt(); - col = in.readInt(); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/BiIndexedCacheColumnDecisionTreeTrainerInput.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/BiIndexedCacheColumnDecisionTreeTrainerInput.java deleted file mode 100644 index 04281fba8de45..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/BiIndexedCacheColumnDecisionTreeTrainerInput.java +++ /dev/null @@ -1,57 +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.trees.trainers.columnbased; - -import java.util.Map; -import java.util.stream.DoubleStream; -import java.util.stream.IntStream; -import java.util.stream.Stream; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.lang.IgniteBiTuple; - -/** - * Adapter for column decision tree trainer for bi-indexed cache. - */ -public class BiIndexedCacheColumnDecisionTreeTrainerInput extends CacheColumnDecisionTreeTrainerInput { - /** - * Construct an input for {@link ColumnDecisionTreeTrainer}. - * - * @param cache Bi-indexed cache. - * @param catFeaturesInfo Information about categorical feature in the form (feature index -> number of - * categories). - * @param samplesCnt Count of samples. - * @param featuresCnt Count of features. - */ - public BiIndexedCacheColumnDecisionTreeTrainerInput(IgniteCache cache, - Map catFeaturesInfo, int samplesCnt, int featuresCnt) { - super(cache, - () -> IntStream.range(0, samplesCnt).mapToObj(s -> new BiIndex(s, featuresCnt)), - e -> Stream.of(new IgniteBiTuple<>(e.getKey().row(), e.getValue())), - DoubleStream::of, - fIdx -> IntStream.range(0, samplesCnt).mapToObj(s -> new BiIndex(s, fIdx)), - catFeaturesInfo, - featuresCnt, - samplesCnt); - } - - /** {@inheritDoc} */ - @Override public Object affinityKey(int idx, Ignite ignite) { - return idx; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/CacheColumnDecisionTreeTrainerInput.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/CacheColumnDecisionTreeTrainerInput.java deleted file mode 100644 index 40927b77d7fa6..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/CacheColumnDecisionTreeTrainerInput.java +++ /dev/null @@ -1,141 +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.trees.trainers.columnbased; - -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.DoubleStream; -import java.util.stream.Stream; -import javax.cache.Cache; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.Ignition; -import org.apache.ignite.internal.processors.cache.CacheEntryImpl; -import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.functions.IgniteSupplier; - -/** - * Adapter of a given cache to {@link CacheColumnDecisionTreeTrainerInput} - * - * @param Class of keys of the cache. - * @param Class of values of the cache. - */ -public abstract class CacheColumnDecisionTreeTrainerInput implements ColumnDecisionTreeTrainerInput { - /** Supplier of labels key. */ - private final IgniteSupplier> labelsKeys; - - /** Count of features. */ - private final int featuresCnt; - - /** Function which maps feature index to Stream of keys corresponding to this feature index. */ - private final IgniteFunction> keyMapper; - - /** Information about which features are categorical in form of feature index -> number of categories. */ - private final Map catFeaturesInfo; - - /** Cache name. */ - private final String cacheName; - - /** Count of samples. */ - private final int samplesCnt; - - /** Function used for mapping cache values to stream of tuples. */ - private final IgniteFunction, Stream>> valuesMapper; - - /** - * Function which map value of entry with label key to DoubleStream. - * Look at {@code CacheColumnDecisionTreeTrainerInput::labels} for understanding how {@code labelsKeys} and - * {@code labelsMapper} interact. - */ - private final IgniteFunction labelsMapper; - - /** - * Constructs input for {@link ColumnDecisionTreeTrainer}. - * - * @param c Cache. - * @param valuesMapper Function for mapping cache entry to stream used by {@link ColumnDecisionTreeTrainer}. - * @param labelsMapper Function used for mapping cache value to labels array. - * @param keyMapper Function used for mapping feature index to the cache key. - * @param catFeaturesInfo Information about which features are categorical in form of feature index -> number of - * categories. - * @param featuresCnt Count of features. - * @param samplesCnt Count of samples. - */ - // TODO: IGNITE-5724 think about boxing/unboxing - public CacheColumnDecisionTreeTrainerInput(IgniteCache c, - IgniteSupplier> labelsKeys, - IgniteFunction, Stream>> valuesMapper, - IgniteFunction labelsMapper, - IgniteFunction> keyMapper, - Map catFeaturesInfo, - int featuresCnt, int samplesCnt) { - - cacheName = c.getName(); - this.labelsKeys = labelsKeys; - this.valuesMapper = valuesMapper; - this.labelsMapper = labelsMapper; - this.keyMapper = keyMapper; - this.catFeaturesInfo = catFeaturesInfo; - this.samplesCnt = samplesCnt; - this.featuresCnt = featuresCnt; - } - - /** {@inheritDoc} */ - @Override public Stream> values(int idx) { - return cache(Ignition.localIgnite()).getAll(keyMapper.apply(idx).collect(Collectors.toSet())). - entrySet(). - stream(). - flatMap(ent -> valuesMapper.apply(new CacheEntryImpl<>(ent.getKey(), ent.getValue()))); - } - - /** {@inheritDoc} */ - @Override public double[] labels(Ignite ignite) { - return labelsKeys.get().map(k -> get(k, ignite)).flatMapToDouble(labelsMapper).toArray(); - } - - /** {@inheritDoc} */ - @Override public Map catFeaturesInfo() { - return catFeaturesInfo; - } - - /** {@inheritDoc} */ - @Override public int featuresCount() { - return featuresCnt; - } - - /** {@inheritDoc} */ - @Override public Object affinityKey(int idx, Ignite ignite) { - return ignite.affinity(cacheName).affinityKey(keyMapper.apply(idx)); - } - - /** */ - private V get(K k, Ignite ignite) { - V res = cache(ignite).localPeek(k); - - if (res == null) - res = cache(ignite).get(k); - - return res; - } - - /** */ - private IgniteCache cache(Ignite ignite) { - return ignite.getOrCreateCache(cacheName); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/ColumnDecisionTreeTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/ColumnDecisionTreeTrainer.java deleted file mode 100644 index fec0a83fd503c..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/ColumnDecisionTreeTrainer.java +++ /dev/null @@ -1,568 +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.trees.trainers.columnbased; - -import com.zaxxer.sparsebits.SparseBitSet; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.function.Consumer; -import java.util.stream.Collectors; -import java.util.stream.DoubleStream; -import java.util.stream.IntStream; -import java.util.stream.Stream; -import javax.cache.Cache; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteLogger; -import org.apache.ignite.Ignition; -import org.apache.ignite.cache.CachePeekMode; -import org.apache.ignite.cache.affinity.Affinity; -import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.internal.processors.cache.CacheEntryImpl; -import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.ml.Trainer; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.distributed.CacheUtils; -import org.apache.ignite.ml.math.functions.Functions; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.math.functions.IgniteCurriedBiFunction; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.functions.IgniteSupplier; -import org.apache.ignite.ml.trees.ContinuousRegionInfo; -import org.apache.ignite.ml.trees.ContinuousSplitCalculator; -import org.apache.ignite.ml.trees.models.DecisionTreeModel; -import org.apache.ignite.ml.trees.nodes.DecisionTreeNode; -import org.apache.ignite.ml.trees.nodes.Leaf; -import org.apache.ignite.ml.trees.nodes.SplitNode; -import org.apache.ignite.ml.trees.trainers.columnbased.caches.ContextCache; -import org.apache.ignite.ml.trees.trainers.columnbased.caches.FeaturesCache; -import org.apache.ignite.ml.trees.trainers.columnbased.caches.FeaturesCache.FeatureKey; -import org.apache.ignite.ml.trees.trainers.columnbased.caches.ProjectionsCache; -import org.apache.ignite.ml.trees.trainers.columnbased.caches.ProjectionsCache.RegionKey; -import org.apache.ignite.ml.trees.trainers.columnbased.caches.SplitCache; -import org.apache.ignite.ml.trees.trainers.columnbased.caches.SplitCache.SplitKey; -import org.apache.ignite.ml.trees.trainers.columnbased.vectors.FeatureProcessor; -import org.apache.ignite.ml.trees.trainers.columnbased.vectors.SplitInfo; -import org.jetbrains.annotations.NotNull; - -import static org.apache.ignite.ml.trees.trainers.columnbased.caches.FeaturesCache.getFeatureCacheKey; - -/** - * This trainer stores observations as columns and features as rows. - * Ideas from https://github.com/fabuzaid21/yggdrasil are used here. - */ -public class ColumnDecisionTreeTrainer implements - Trainer { - /** - * Function used to assign a value to a region. - */ - private final IgniteFunction regCalc; - - /** - * Function used to calculate impurity in regions used by categorical features. - */ - private final IgniteFunction> continuousCalculatorProvider; - - /** - * Categorical calculator provider. - **/ - private final IgniteFunction> categoricalCalculatorProvider; - - /** - * Cache used for storing data for training. - */ - private IgniteCache> prjsCache; - - /** - * Minimal information gain. - */ - private static final double MIN_INFO_GAIN = 1E-10; - - /** - * Maximal depth of the decision tree. - */ - private final int maxDepth; - - /** - * Size of block which is used for storing regions in cache. - */ - private static final int BLOCK_SIZE = 1 << 4; - - /** Ignite instance. */ - private final Ignite ignite; - - /** Logger */ - private final IgniteLogger log; - - /** - * Construct {@link ColumnDecisionTreeTrainer}. - * - * @param maxDepth Maximal depth of the decision tree. - * @param continuousCalculatorProvider Provider of calculator of splits for region projection on continuous - * features. - * @param categoricalCalculatorProvider Provider of calculator of splits for region projection on categorical - * features. - * @param regCalc Function used to assign a value to a region. - */ - public ColumnDecisionTreeTrainer(int maxDepth, - IgniteFunction> continuousCalculatorProvider, - IgniteFunction> categoricalCalculatorProvider, - IgniteFunction regCalc, - Ignite ignite) { - this.maxDepth = maxDepth; - this.continuousCalculatorProvider = continuousCalculatorProvider; - this.categoricalCalculatorProvider = categoricalCalculatorProvider; - this.regCalc = regCalc; - this.ignite = ignite; - this.log = ignite.log(); - } - - /** - * Utility class used to get index of feature by which split is done and split info. - */ - private static class IndexAndSplitInfo { - /** - * Index of feature by which split is done. - */ - private final int featureIdx; - - /** - * Split information. - */ - private final SplitInfo info; - - /** - * @param featureIdx Index of feature by which split is done. - * @param info Split information. - */ - IndexAndSplitInfo(int featureIdx, SplitInfo info) { - this.featureIdx = featureIdx; - this.info = info; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return "IndexAndSplitInfo [featureIdx=" + featureIdx + ", info=" + info + ']'; - } - } - - /** - * Utility class used to build decision tree. Basically it is pointer to leaf node. - */ - private static class TreeTip { - /** */ - private Consumer leafSetter; - - /** */ - private int depth; - - /** */ - TreeTip(Consumer leafSetter, int depth) { - this.leafSetter = leafSetter; - this.depth = depth; - } - } - - /** - * Utility class used as decision tree root node. - */ - private static class RootNode implements DecisionTreeNode { - /** */ - private DecisionTreeNode s; - - /** - * {@inheritDoc} - */ - @Override public double process(Vector v) { - return s.process(v); - } - - /** */ - void setSplit(DecisionTreeNode s) { - this.s = s; - } - } - - /** - * {@inheritDoc} - */ - @Override public DecisionTreeModel train(ColumnDecisionTreeTrainerInput i) { - prjsCache = ProjectionsCache.getOrCreate(ignite); - IgniteCache> ctxtCache = ContextCache.getOrCreate(ignite); - SplitCache.getOrCreate(ignite); - - UUID trainingUUID = UUID.randomUUID(); - - TrainingContext ct = new TrainingContext<>(i, continuousCalculatorProvider.apply(i), categoricalCalculatorProvider.apply(i), trainingUUID, ignite); - ctxtCache.put(trainingUUID, ct); - - CacheUtils.bcast(prjsCache.getName(), ignite, () -> { - Ignite ignite = Ignition.localIgnite(); - IgniteCache> projCache = ProjectionsCache.getOrCreate(ignite); - IgniteCache featuresCache = FeaturesCache.getOrCreate(ignite); - - Affinity targetAffinity = ignite.affinity(ProjectionsCache.CACHE_NAME); - - ClusterNode locNode = ignite.cluster().localNode(); - - Map fm = new ConcurrentHashMap<>(); - Map> pm = new ConcurrentHashMap<>(); - - targetAffinity. - mapKeysToNodes(IntStream.range(0, i.featuresCount()). - mapToObj(idx -> ProjectionsCache.key(idx, 0, i.affinityKey(idx, ignite), trainingUUID)). - collect(Collectors.toSet())).getOrDefault(locNode, Collections.emptyList()). - forEach(k -> { - FeatureProcessor vec; - - int featureIdx = k.featureIdx(); - - IgniteCache> ctxCache = ContextCache.getOrCreate(ignite); - TrainingContext ctx = ctxCache.get(trainingUUID); - double[] vals = new double[ctx.labels().length]; - - vec = ctx.featureProcessor(featureIdx); - i.values(featureIdx).forEach(t -> vals[t.get1()] = t.get2()); - - fm.put(getFeatureCacheKey(featureIdx, trainingUUID, i.affinityKey(featureIdx, ignite)), vals); - - List newReg = new ArrayList<>(BLOCK_SIZE); - newReg.add(vec.createInitialRegion(getSamples(i.values(featureIdx), ctx.labels().length), vals, ctx.labels())); - pm.put(k, newReg); - }); - - featuresCache.putAll(fm); - projCache.putAll(pm); - - return null; - }); - - return doTrain(i, trainingUUID); - } - - /** - * Get samples array. - * - * @param values Stream of tuples in the form of (index, value). - * @param size size of stream. - * @return Samples array. - */ - private Integer[] getSamples(Stream> values, int size) { - Integer[] res = new Integer[size]; - - values.forEach(v -> res[v.get1()] = v.get1()); - - return res; - } - - /** */ - @NotNull - private DecisionTreeModel doTrain(ColumnDecisionTreeTrainerInput input, UUID uuid) { - RootNode root = new RootNode(); - - // List containing setters of leaves of the tree. - List tips = new LinkedList<>(); - tips.add(new TreeTip(root::setSplit, 0)); - - int curDepth = 0; - int regsCnt = 1; - - int featuresCnt = input.featuresCount(); - IntStream.range(0, featuresCnt).mapToObj(fIdx -> SplitCache.key(fIdx, input.affinityKey(fIdx, ignite), uuid)). - forEach(k -> SplitCache.getOrCreate(ignite).put(k, new IgniteBiTuple<>(0, 0.0))); - updateSplitCache(0, regsCnt, featuresCnt, ig -> i -> input.affinityKey(i, ig), uuid); - - // TODO: IGNITE-5893 Currently if the best split makes tree deeper than max depth process will be terminated, but actually we should - // only stop when *any* improving split makes tree deeper than max depth. Can be fixed if we will store which - // regions cannot be split more and split only those that can. - while (true) { - long before = System.currentTimeMillis(); - - IgniteBiTuple> b = findBestSplitIndexForFeatures(featuresCnt, input::affinityKey, uuid); - - long findBestRegIdx = System.currentTimeMillis() - before; - - Integer bestFeatureIdx = b.get1(); - - Integer regIdx = b.get2().get1(); - Double bestInfoGain = b.get2().get2(); - - if (regIdx >= 0 && bestInfoGain > MIN_INFO_GAIN) { - before = System.currentTimeMillis(); - - SplitInfo bi = ignite.compute().affinityCall(ProjectionsCache.CACHE_NAME, - input.affinityKey(bestFeatureIdx, ignite), - () -> { - TrainingContext ctx = ContextCache.getOrCreate(ignite).get(uuid); - Ignite ignite = Ignition.localIgnite(); - RegionKey key = ProjectionsCache.key(bestFeatureIdx, - regIdx / BLOCK_SIZE, - input.affinityKey(bestFeatureIdx, Ignition.localIgnite()), - uuid); - RegionProjection reg = ProjectionsCache.getOrCreate(ignite).localPeek(key).get(regIdx % BLOCK_SIZE); - return ctx.featureProcessor(bestFeatureIdx).findBestSplit(reg, ctx.values(bestFeatureIdx, ignite), ctx.labels(), regIdx); - }); - - long findBestSplit = System.currentTimeMillis() - before; - - IndexAndSplitInfo best = new IndexAndSplitInfo(bestFeatureIdx, bi); - - regsCnt++; - - if (log.isDebugEnabled()) - log.debug("Globally best: " + best.info + " idx time: " + findBestRegIdx + ", calculate best: " + findBestSplit + " fi: " + best.featureIdx + ", regs: " + regsCnt); - // Request bitset for split region. - int ind = best.info.regionIndex(); - - SparseBitSet bs = ignite.compute().affinityCall(ProjectionsCache.CACHE_NAME, - input.affinityKey(bestFeatureIdx, ignite), - () -> { - Ignite ignite = Ignition.localIgnite(); - IgniteCache featuresCache = FeaturesCache.getOrCreate(ignite); - IgniteCache> ctxCache = ContextCache.getOrCreate(ignite); - TrainingContext ctx = ctxCache.localPeek(uuid); - - double[] values = featuresCache.localPeek(getFeatureCacheKey(bestFeatureIdx, uuid, input.affinityKey(bestFeatureIdx, Ignition.localIgnite()))); - RegionKey key = ProjectionsCache.key(bestFeatureIdx, - regIdx / BLOCK_SIZE, - input.affinityKey(bestFeatureIdx, Ignition.localIgnite()), - uuid); - RegionProjection reg = ProjectionsCache.getOrCreate(ignite).localPeek(key).get(regIdx % BLOCK_SIZE); - return ctx.featureProcessor(bestFeatureIdx).calculateOwnershipBitSet(reg, values, best.info); - - }); - - SplitNode sn = best.info.createSplitNode(best.featureIdx); - - TreeTip tipToSplit = tips.get(ind); - tipToSplit.leafSetter.accept(sn); - tipToSplit.leafSetter = sn::setLeft; - int d = tipToSplit.depth++; - tips.add(new TreeTip(sn::setRight, d)); - - if (d > curDepth) { - curDepth = d; - if (log.isDebugEnabled()) { - log.debug("Depth: " + curDepth); - log.debug("Cache size: " + prjsCache.size(CachePeekMode.PRIMARY)); - } - } - - before = System.currentTimeMillis(); - // Perform split on all feature vectors. - IgniteSupplier> bestRegsKeys = () -> IntStream.range(0, featuresCnt). - mapToObj(fIdx -> ProjectionsCache.key(fIdx, ind / BLOCK_SIZE, input.affinityKey(fIdx, Ignition.localIgnite()), uuid)). - collect(Collectors.toSet()); - - int rc = regsCnt; - - // Perform split. - CacheUtils.update(prjsCache.getName(), ignite, - (Ignite ign, Cache.Entry> e) -> { - RegionKey k = e.getKey(); - - List leftBlock = e.getValue(); - - int fIdx = k.featureIdx(); - int idxInBlock = ind % BLOCK_SIZE; - - IgniteCache> ctxCache = ContextCache.getOrCreate(ign); - TrainingContext ctx = ctxCache.get(uuid); - - RegionProjection targetRegProj = leftBlock.get(idxInBlock); - - IgniteBiTuple regs = ctx. - performSplit(input, bs, fIdx, best.featureIdx, targetRegProj, best.info.leftData(), best.info.rightData(), ign); - - RegionProjection left = regs.get1(); - RegionProjection right = regs.get2(); - - leftBlock.set(idxInBlock, left); - RegionKey rightKey = ProjectionsCache.key(fIdx, (rc - 1) / BLOCK_SIZE, input.affinityKey(fIdx, ign), uuid); - - IgniteCache> c = ProjectionsCache.getOrCreate(ign); - - List rightBlock = rightKey.equals(k) ? leftBlock : c.localPeek(rightKey); - - if (rightBlock == null) { - List newBlock = new ArrayList<>(BLOCK_SIZE); - newBlock.add(right); - return Stream.of(new CacheEntryImpl<>(k, leftBlock), new CacheEntryImpl<>(rightKey, newBlock)); - } - else { - rightBlock.add(right); - return rightBlock.equals(k) ? - Stream.of(new CacheEntryImpl<>(k, leftBlock)) : - Stream.of(new CacheEntryImpl<>(k, leftBlock), new CacheEntryImpl<>(rightKey, rightBlock)); - } - }, - bestRegsKeys); - - if (log.isDebugEnabled()) - log.debug("Update of projections cache time: " + (System.currentTimeMillis() - before)); - - before = System.currentTimeMillis(); - - updateSplitCache(ind, rc, featuresCnt, ig -> i -> input.affinityKey(i, ig), uuid); - - if (log.isDebugEnabled()) - log.debug("Update of split cache time: " + (System.currentTimeMillis() - before)); - } - else { - if (log.isDebugEnabled()) - log.debug("Best split [bestFeatureIdx=" + bestFeatureIdx + ", bestInfoGain=" + bestInfoGain + "]"); - break; - } - } - - int rc = regsCnt; - - IgniteSupplier>>> featZeroRegs = () -> { - IgniteCache> projsCache = ProjectionsCache.getOrCreate(Ignition.localIgnite()); - - return () -> IntStream.range(0, (rc - 1) / BLOCK_SIZE + 1). - mapToObj(rBIdx -> ProjectionsCache.key(0, rBIdx, input.affinityKey(0, Ignition.localIgnite()), uuid)). - map(k -> (Cache.Entry>)new CacheEntryImpl<>(k, projsCache.localPeek(k))).iterator(); - }; - - Map vals = CacheUtils.reduce(prjsCache.getName(), ignite, - (TrainingContext ctx, Cache.Entry> e, Map m) -> { - int regBlockIdx = e.getKey().regionBlockIndex(); - - if (e.getValue() != null) { - for (int i = 0; i < e.getValue().size(); i++) { - int regIdx = regBlockIdx * BLOCK_SIZE + i; - RegionProjection reg = e.getValue().get(i); - - Double res = regCalc.apply(Arrays.stream(reg.sampleIndexes()).mapToDouble(s -> ctx.labels()[s])); - m.put(regIdx, res); - } - } - - return m; - }, - () -> ContextCache.getOrCreate(Ignition.localIgnite()).get(uuid), - featZeroRegs, - (infos, infos2) -> { - Map res = new HashMap<>(); - res.putAll(infos); - res.putAll(infos2); - return res; - }, - HashMap::new - ); - - int i = 0; - for (TreeTip tip : tips) { - tip.leafSetter.accept(new Leaf(vals.get(i))); - i++; - } - - ProjectionsCache.clear(featuresCnt, rc, input::affinityKey, uuid, ignite); - ContextCache.getOrCreate(ignite).remove(uuid); - FeaturesCache.clear(featuresCnt, input::affinityKey, uuid, ignite); - SplitCache.clear(featuresCnt, input::affinityKey, uuid, ignite); - - return new DecisionTreeModel(root.s); - } - - /** - * Find the best split in the form (feature index, (index of region with the best split, impurity of region with the - * best split)). - * - * @param featuresCnt Count of features. - * @param affinity Affinity function. - * @param trainingUUID UUID of training. - * @return Best split in the form (feature index, (index of region with the best split, impurity of region with the - * best split)). - */ - private IgniteBiTuple> findBestSplitIndexForFeatures(int featuresCnt, - IgniteBiFunction affinity, - UUID trainingUUID) { - Set featureIndexes = IntStream.range(0, featuresCnt).boxed().collect(Collectors.toSet()); - - return CacheUtils.reduce(SplitCache.CACHE_NAME, ignite, - (Object ctx, Cache.Entry> e, IgniteBiTuple> r) -> - Functions.MAX_GENERIC(new IgniteBiTuple<>(e.getKey().featureIdx(), e.getValue()), r, comparator()), - () -> null, - () -> SplitCache.localEntries(featureIndexes, affinity, trainingUUID), - (i1, i2) -> Functions.MAX_GENERIC(i1, i2, Comparator.comparingDouble(bt -> bt.get2().get2())), - () -> new IgniteBiTuple<>(-1, new IgniteBiTuple<>(-1, Double.NEGATIVE_INFINITY)) - ); - } - - /** */ - private static Comparator>> comparator() { - return Comparator.comparingDouble(bt -> bt != null && bt.get2() != null ? bt.get2().get2() : Double.NEGATIVE_INFINITY); - } - - /** - * Update split cache. - * - * @param lastSplitRegionIdx Index of region which had last best split. - * @param regsCnt Count of regions. - * @param featuresCnt Count of features. - * @param affinity Affinity function. - * @param trainingUUID UUID of current training. - */ - private void updateSplitCache(int lastSplitRegionIdx, int regsCnt, int featuresCnt, - IgniteCurriedBiFunction affinity, - UUID trainingUUID) { - CacheUtils.update(SplitCache.CACHE_NAME, ignite, - (Ignite ign, Cache.Entry> e) -> { - Integer bestRegIdx = e.getValue().get1(); - int fIdx = e.getKey().featureIdx(); - TrainingContext ctx = ContextCache.getOrCreate(ign).get(trainingUUID); - - Map toCompare; - - // Fully recalculate best. - if (bestRegIdx == lastSplitRegionIdx) - toCompare = ProjectionsCache.projectionsOfFeature(fIdx, maxDepth, regsCnt, BLOCK_SIZE, affinity.apply(ign), trainingUUID, ign); - // Just compare previous best and two regions which are produced by split. - else - toCompare = ProjectionsCache.projectionsOfRegions(fIdx, maxDepth, - IntStream.of(bestRegIdx, lastSplitRegionIdx, regsCnt - 1), BLOCK_SIZE, affinity.apply(ign), trainingUUID, ign); - - double[] values = ctx.values(fIdx, ign); - double[] labels = ctx.labels(); - - Optional> max = toCompare.entrySet().stream(). - map(ent -> { - SplitInfo bestSplit = ctx.featureProcessor(fIdx).findBestSplit(ent.getValue(), values, labels, ent.getKey()); - return new IgniteBiTuple<>(ent.getKey(), bestSplit != null ? bestSplit.infoGain() : Double.NEGATIVE_INFINITY); - }). - max(Comparator.comparingDouble(IgniteBiTuple::get2)); - - return max.>>> - map(objects -> Stream.of(new CacheEntryImpl<>(e.getKey(), objects))).orElseGet(Stream::empty); - }, - () -> IntStream.range(0, featuresCnt).mapToObj(fIdx -> SplitCache.key(fIdx, affinity.apply(ignite).apply(fIdx), trainingUUID)).collect(Collectors.toSet()) - ); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/ColumnDecisionTreeTrainerInput.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/ColumnDecisionTreeTrainerInput.java deleted file mode 100644 index bf8790b705de8..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/ColumnDecisionTreeTrainerInput.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.ml.trees.trainers.columnbased; - -import java.util.Map; -import java.util.stream.Stream; -import org.apache.ignite.Ignite; -import org.apache.ignite.lang.IgniteBiTuple; - -/** - * Input for {@link ColumnDecisionTreeTrainer}. - */ -public interface ColumnDecisionTreeTrainerInput { - /** - * Projection of data on feature with the given index. - * - * @param idx Feature index. - * @return Projection of data on feature with the given index. - */ - Stream> values(int idx); - - /** - * Labels. - * - * @param ignite Ignite instance. - */ - double[] labels(Ignite ignite); - - /** Information about which features are categorical in the form of feature index -> number of categories. */ - Map catFeaturesInfo(); - - /** Number of features. */ - int featuresCount(); - - /** - * Get affinity key for the given column index. - * Affinity key should be pure-functionally dependent from idx. - */ - Object affinityKey(int idx, Ignite ignite); -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/MatrixColumnDecisionTreeTrainerInput.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/MatrixColumnDecisionTreeTrainerInput.java deleted file mode 100644 index 3da6bad218f11..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/MatrixColumnDecisionTreeTrainerInput.java +++ /dev/null @@ -1,83 +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.trees.trainers.columnbased; - -import java.util.HashMap; -import java.util.Map; -import java.util.stream.DoubleStream; -import java.util.stream.IntStream; -import java.util.stream.Stream; -import javax.cache.Cache; -import org.apache.ignite.Ignite; -import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.ml.math.distributed.keys.RowColMatrixKey; -import org.apache.ignite.ml.math.distributed.keys.impl.SparseMatrixKey; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.ml.math.impls.storage.matrix.SparseDistributedMatrixStorage; -import org.apache.ignite.ml.math.StorageConstants; -import org.jetbrains.annotations.NotNull; - -/** - * Adapter of SparseDistributedMatrix to ColumnDecisionTreeTrainerInput. - * Sparse SparseDistributedMatrix should be in {@link StorageConstants#COLUMN_STORAGE_MODE} and - * should contain samples in rows last position in row being label of this sample. - */ -public class MatrixColumnDecisionTreeTrainerInput extends CacheColumnDecisionTreeTrainerInput> { - /** - * @param m Sparse SparseDistributedMatrix should be in {@link StorageConstants#COLUMN_STORAGE_MODE} - * containing samples in rows last position in row being label of this sample. - * @param catFeaturesInfo Information about which features are categorical in form of feature index -> number of - * categories. - */ - public MatrixColumnDecisionTreeTrainerInput(SparseDistributedMatrix m, Map catFeaturesInfo) { - super(((SparseDistributedMatrixStorage)m.getStorage()).cache(), - () -> Stream.of(new SparseMatrixKey(m.columnSize() - 1, m.getUUID(), m.columnSize() - 1)), - valuesMapper(m), - labels(m), - keyMapper(m), - catFeaturesInfo, - m.columnSize() - 1, - m.rowSize()); - } - - /** Values mapper. See {@link CacheColumnDecisionTreeTrainerInput#valuesMapper} */ - @NotNull - private static IgniteFunction>, Stream>> valuesMapper( - SparseDistributedMatrix m) { - return ent -> { - Map map = ent.getValue() != null ? ent.getValue() : new HashMap<>(); - return IntStream.range(0, m.rowSize()).mapToObj(k -> new IgniteBiTuple<>(k, map.getOrDefault(k, 0.0))); - }; - } - - /** Key mapper. See {@link CacheColumnDecisionTreeTrainerInput#keyMapper} */ - @NotNull private static IgniteFunction> keyMapper(SparseDistributedMatrix m) { - return i -> Stream.of(new SparseMatrixKey(i, ((SparseDistributedMatrixStorage)m.getStorage()).getUUID(), i)); - } - - /** Labels mapper. See {@link CacheColumnDecisionTreeTrainerInput#labelsMapper} */ - @NotNull private static IgniteFunction, DoubleStream> labels(SparseDistributedMatrix m) { - return mp -> IntStream.range(0, m.rowSize()).mapToDouble(k -> mp.getOrDefault(k, 0.0)); - } - - /** {@inheritDoc} */ - @Override public Object affinityKey(int idx, Ignite ignite) { - return idx; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/RegionProjection.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/RegionProjection.java deleted file mode 100644 index e95f57b6a79a0..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/RegionProjection.java +++ /dev/null @@ -1,109 +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.trees.trainers.columnbased; - -import java.io.Externalizable; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import org.apache.ignite.ml.trees.RegionInfo; - -/** - * Projection of region on given feature. - * - * @param Data of region. - */ -public class RegionProjection implements Externalizable { - /** Samples projections. */ - protected Integer[] sampleIndexes; - - /** Region data */ - protected D data; - - /** Depth of this region. */ - protected int depth; - - /** - * @param sampleIndexes Samples indexes. - * @param data Region data. - * @param depth Depth of this region. - */ - public RegionProjection(Integer[] sampleIndexes, D data, int depth) { - this.data = data; - this.depth = depth; - this.sampleIndexes = sampleIndexes; - } - - /** - * No-op constructor used for serialization/deserialization. - */ - public RegionProjection() { - // No-op. - } - - /** - * Get samples indexes. - * - * @return Samples indexes. - */ - public Integer[] sampleIndexes() { - return sampleIndexes; - } - - /** - * Get region data. - * - * @return Region data. - */ - public D data() { - return data; - } - - /** - * Get region depth. - * - * @return Region depth. - */ - public int depth() { - return depth; - } - - /** {@inheritDoc} */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - out.writeInt(sampleIndexes.length); - - for (Integer sampleIndex : sampleIndexes) - out.writeInt(sampleIndex); - - out.writeObject(data); - out.writeInt(depth); - } - - /** {@inheritDoc} */ - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - int size = in.readInt(); - - sampleIndexes = new Integer[size]; - - for (int i = 0; i < size; i++) - sampleIndexes[i] = in.readInt(); - - data = (D)in.readObject(); - depth = in.readInt(); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/TrainingContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/TrainingContext.java deleted file mode 100644 index 6415dab4bbfce..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/TrainingContext.java +++ /dev/null @@ -1,166 +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.trees.trainers.columnbased; - -import com.zaxxer.sparsebits.SparseBitSet; -import java.util.Map; -import java.util.UUID; -import java.util.stream.DoubleStream; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.Ignition; -import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.trees.ContinuousRegionInfo; -import org.apache.ignite.ml.trees.ContinuousSplitCalculator; -import org.apache.ignite.ml.trees.RegionInfo; -import org.apache.ignite.ml.trees.trainers.columnbased.caches.FeaturesCache; -import org.apache.ignite.ml.trees.trainers.columnbased.vectors.CategoricalFeatureProcessor; -import org.apache.ignite.ml.trees.trainers.columnbased.vectors.ContinuousFeatureProcessor; -import org.apache.ignite.ml.trees.trainers.columnbased.vectors.FeatureProcessor; - -import static org.apache.ignite.ml.trees.trainers.columnbased.caches.FeaturesCache.COLUMN_DECISION_TREE_TRAINER_FEATURES_CACHE_NAME; - -/** - * Context of training with {@link ColumnDecisionTreeTrainer}. - * - * @param Class for storing of information used in calculation of impurity of continuous feature region. - */ -public class TrainingContext { - /** Input for training with {@link ColumnDecisionTreeTrainer}. */ - private final ColumnDecisionTreeTrainerInput input; - - /** Labels. */ - private final double[] labels; - - /** Calculator used for finding splits of region of continuous features. */ - private final ContinuousSplitCalculator continuousSplitCalculator; - - /** Calculator used for finding splits of region of categorical feature. */ - private final IgniteFunction categoricalSplitCalculator; - - /** UUID of current training. */ - private final UUID trainingUUID; - - /** - * Construct context for training with {@link ColumnDecisionTreeTrainer}. - * - * @param input Input for training. - * @param continuousSplitCalculator Calculator used for calculations of splits of continuous features regions. - * @param categoricalSplitCalculator Calculator used for calculations of splits of categorical features regions. - * @param trainingUUID UUID of the current training. - * @param ignite Ignite instance. - */ - public TrainingContext(ColumnDecisionTreeTrainerInput input, - ContinuousSplitCalculator continuousSplitCalculator, - IgniteFunction categoricalSplitCalculator, - UUID trainingUUID, - Ignite ignite) { - this.input = input; - this.labels = input.labels(ignite); - this.continuousSplitCalculator = continuousSplitCalculator; - this.categoricalSplitCalculator = categoricalSplitCalculator; - this.trainingUUID = trainingUUID; - } - - /** - * Get processor used for calculating splits of categorical features. - * - * @param catsCnt Count of categories. - * @return Processor used for calculating splits of categorical features. - */ - public CategoricalFeatureProcessor categoricalFeatureProcessor(int catsCnt) { - return new CategoricalFeatureProcessor(categoricalSplitCalculator, catsCnt); - } - - /** - * Get processor used for calculating splits of continuous features. - * - * @return Processor used for calculating splits of continuous features. - */ - public ContinuousFeatureProcessor continuousFeatureProcessor() { - return new ContinuousFeatureProcessor<>(continuousSplitCalculator); - } - - /** - * Get labels. - * - * @return Labels. - */ - public double[] labels() { - return labels; - } - - /** - * Get values of feature with given index. - * - * @param featIdx Feature index. - * @param ignite Ignite instance. - * @return Values of feature with given index. - */ - public double[] values(int featIdx, Ignite ignite) { - IgniteCache featuresCache = ignite.getOrCreateCache(COLUMN_DECISION_TREE_TRAINER_FEATURES_CACHE_NAME); - return featuresCache.localPeek(FeaturesCache.getFeatureCacheKey(featIdx, trainingUUID, input.affinityKey(featIdx, ignite))); - } - - /** - * Perform best split on the given region projection. - * - * @param input Input of {@link ColumnDecisionTreeTrainer} performing split. - * @param bitSet Bit set specifying split. - * @param targetFeatIdx Index of feature for performing split. - * @param bestFeatIdx Index of feature with best split. - * @param targetRegionPrj Projection of region to split on feature with index {@code featureIdx}. - * @param leftData Data of left region of split. - * @param rightData Data of right region of split. - * @param ignite Ignite instance. - * @return Perform best split on the given region projection. - */ - public IgniteBiTuple performSplit(ColumnDecisionTreeTrainerInput input, - SparseBitSet bitSet, int targetFeatIdx, int bestFeatIdx, RegionProjection targetRegionPrj, RegionInfo leftData, - RegionInfo rightData, Ignite ignite) { - - Map catFeaturesInfo = input.catFeaturesInfo(); - - if (!catFeaturesInfo.containsKey(targetFeatIdx) && !catFeaturesInfo.containsKey(bestFeatIdx)) - return continuousFeatureProcessor().performSplit(bitSet, targetRegionPrj, (D)leftData, (D)rightData); - else if (catFeaturesInfo.containsKey(targetFeatIdx)) - return categoricalFeatureProcessor(catFeaturesInfo.get(targetFeatIdx)).performSplitGeneric(bitSet, values(targetFeatIdx, ignite), targetRegionPrj, leftData, rightData); - return continuousFeatureProcessor().performSplitGeneric(bitSet, labels, targetRegionPrj, leftData, rightData); - } - - /** - * Processor used for calculating splits for feature with the given index. - * - * @param featureIdx Index of feature to process. - * @return Processor used for calculating splits for feature with the given index. - */ - public FeatureProcessor featureProcessor(int featureIdx) { - return input.catFeaturesInfo().containsKey(featureIdx) ? categoricalFeatureProcessor(input.catFeaturesInfo().get(featureIdx)) : continuousFeatureProcessor(); - } - - /** - * Shortcut for affinity key. - * - * @param idx Feature index. - * @return Affinity key. - */ - public Object affinityKey(int idx) { - return input.affinityKey(idx, Ignition.localIgnite()); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/ContextCache.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/ContextCache.java deleted file mode 100644 index 51ea359a3e00f..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/ContextCache.java +++ /dev/null @@ -1,68 +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.trees.trainers.columnbased.caches; - -import java.util.UUID; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.cache.CacheAtomicityMode; -import org.apache.ignite.cache.CacheMode; -import org.apache.ignite.cache.CacheWriteSynchronizationMode; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.ml.trees.ContinuousRegionInfo; -import org.apache.ignite.ml.trees.trainers.columnbased.ColumnDecisionTreeTrainer; -import org.apache.ignite.ml.trees.trainers.columnbased.TrainingContext; - -/** - * Class for operations related to cache containing training context for {@link ColumnDecisionTreeTrainer}. - */ -public class ContextCache { - /** - * Name of cache containing training context for {@link ColumnDecisionTreeTrainer}. - */ - public static final String COLUMN_DECISION_TREE_TRAINER_CONTEXT_CACHE_NAME = "COLUMN_DECISION_TREE_TRAINER_CONTEXT_CACHE_NAME"; - - /** - * Get or create cache for training context. - * - * @param ignite Ignite instance. - * @param Class storing information about continuous regions. - * @return Cache for training context. - */ - public static IgniteCache> getOrCreate(Ignite ignite) { - CacheConfiguration> cfg = new CacheConfiguration<>(); - - cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); - - cfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); - - cfg.setEvictionPolicy(null); - - cfg.setCopyOnRead(false); - - cfg.setCacheMode(CacheMode.REPLICATED); - - cfg.setOnheapCacheEnabled(true); - - cfg.setReadFromBackup(true); - - cfg.setName(COLUMN_DECISION_TREE_TRAINER_CONTEXT_CACHE_NAME); - - return ignite.getOrCreateCache(cfg); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/FeaturesCache.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/FeaturesCache.java deleted file mode 100644 index fcc1f16489317..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/FeaturesCache.java +++ /dev/null @@ -1,151 +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.trees.trainers.columnbased.caches; - -import java.util.Set; -import java.util.UUID; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.cache.CacheAtomicityMode; -import org.apache.ignite.cache.CacheMode; -import org.apache.ignite.cache.CacheWriteSynchronizationMode; -import org.apache.ignite.cache.affinity.AffinityKeyMapped; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.trees.trainers.columnbased.ColumnDecisionTreeTrainer; - -/** - * Cache storing features for {@link ColumnDecisionTreeTrainer}. - */ -public class FeaturesCache { - /** - * Name of cache which is used for storing features for {@link ColumnDecisionTreeTrainer}. - */ - public static final String COLUMN_DECISION_TREE_TRAINER_FEATURES_CACHE_NAME = "COLUMN_DECISION_TREE_TRAINER_FEATURES_CACHE_NAME"; - - /** - * Key of features cache. - */ - public static class FeatureKey { - /** Column key of cache used as input for {@link ColumnDecisionTreeTrainer}. */ - @AffinityKeyMapped - private Object parentColKey; - - /** Index of feature. */ - private final int featureIdx; - - /** UUID of training. */ - private final UUID trainingUUID; - - /** - * Construct FeatureKey. - * - * @param featureIdx Feature index. - * @param trainingUUID UUID of training. - * @param parentColKey Column key of cache used as input. - */ - public FeatureKey(int featureIdx, UUID trainingUUID, Object parentColKey) { - this.parentColKey = parentColKey; - this.featureIdx = featureIdx; - this.trainingUUID = trainingUUID; - this.parentColKey = parentColKey; - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - - FeatureKey key = (FeatureKey)o; - - if (featureIdx != key.featureIdx) - return false; - return trainingUUID != null ? trainingUUID.equals(key.trainingUUID) : key.trainingUUID == null; - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - int res = trainingUUID != null ? trainingUUID.hashCode() : 0; - res = 31 * res + featureIdx; - return res; - } - } - - /** - * Create new projections cache for ColumnDecisionTreeTrainer if needed. - * - * @param ignite Ignite instance. - */ - public static IgniteCache getOrCreate(Ignite ignite) { - CacheConfiguration cfg = new CacheConfiguration<>(); - - // Write to primary. - cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC); - - // Atomic transactions only. - cfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); - - // No eviction. - cfg.setEvictionPolicy(null); - - // No copying of values. - cfg.setCopyOnRead(false); - - // Cache is partitioned. - cfg.setCacheMode(CacheMode.PARTITIONED); - - cfg.setOnheapCacheEnabled(true); - - cfg.setBackups(0); - - cfg.setName(COLUMN_DECISION_TREE_TRAINER_FEATURES_CACHE_NAME); - - return ignite.getOrCreateCache(cfg); - } - - /** - * Construct FeatureKey from index, uuid and affinity key. - * - * @param idx Feature index. - * @param uuid UUID of training. - * @param aff Affinity key. - * @return FeatureKey. - */ - public static FeatureKey getFeatureCacheKey(int idx, UUID uuid, Object aff) { - return new FeatureKey(idx, uuid, aff); - } - - /** - * Clear all data from features cache related to given training. - * - * @param featuresCnt Count of features. - * @param affinity Affinity function. - * @param uuid Training uuid. - * @param ignite Ignite instance. - */ - public static void clear(int featuresCnt, IgniteBiFunction affinity, UUID uuid, - Ignite ignite) { - Set toRmv = IntStream.range(0, featuresCnt).boxed().map(fIdx -> getFeatureCacheKey(fIdx, uuid, affinity.apply(fIdx, ignite))).collect(Collectors.toSet()); - - getOrCreate(ignite).removeAll(toRmv); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/ProjectionsCache.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/ProjectionsCache.java deleted file mode 100644 index 080cb6636ff7a..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/ProjectionsCache.java +++ /dev/null @@ -1,286 +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.trees.trainers.columnbased.caches; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.PrimitiveIterator; -import java.util.Set; -import java.util.UUID; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.Ignition; -import org.apache.ignite.cache.CacheAtomicityMode; -import org.apache.ignite.cache.CacheMode; -import org.apache.ignite.cache.CacheWriteSynchronizationMode; -import org.apache.ignite.cache.affinity.Affinity; -import org.apache.ignite.cache.affinity.AffinityKeyMapped; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.trees.trainers.columnbased.ColumnDecisionTreeTrainer; -import org.apache.ignite.ml.trees.trainers.columnbased.RegionProjection; - -/** - * Cache used for storing data of region projections on features. - */ -public class ProjectionsCache { - /** - * Name of cache which is used for storing data of region projections on features of {@link - * ColumnDecisionTreeTrainer}. - */ - public static final String CACHE_NAME = "COLUMN_DECISION_TREE_TRAINER_PROJECTIONS_CACHE_NAME"; - - /** - * Key of region projections cache. - */ - public static class RegionKey { - /** Column key of cache used as input for {@link ColumnDecisionTreeTrainer}. */ - @AffinityKeyMapped - private final Object parentColKey; - - /** Feature index. */ - private final int featureIdx; - - /** Region index. */ - private final int regBlockIdx; - - /** Training UUID. */ - private final UUID trainingUUID; - - /** - * Construct a RegionKey from feature index, index of block, key of column in input cache and UUID of training. - * - * @param featureIdx Feature index. - * @param regBlockIdx Index of block. - * @param parentColKey Key of column in input cache. - * @param trainingUUID UUID of training. - */ - public RegionKey(int featureIdx, int regBlockIdx, Object parentColKey, UUID trainingUUID) { - this.featureIdx = featureIdx; - this.regBlockIdx = regBlockIdx; - this.trainingUUID = trainingUUID; - this.parentColKey = parentColKey; - } - - /** - * Feature index. - * - * @return Feature index. - */ - public int featureIdx() { - return featureIdx; - } - - /** - * Region block index. - * - * @return Region block index. - */ - public int regionBlockIndex() { - return regBlockIdx; - } - - /** - * UUID of training. - * - * @return UUID of training. - */ - public UUID trainingUUID() { - return trainingUUID; - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - - RegionKey key = (RegionKey)o; - - if (featureIdx != key.featureIdx) - return false; - if (regBlockIdx != key.regBlockIdx) - return false; - return trainingUUID != null ? trainingUUID.equals(key.trainingUUID) : key.trainingUUID == null; - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - int res = trainingUUID != null ? trainingUUID.hashCode() : 0; - res = 31 * res + featureIdx; - res = 31 * res + regBlockIdx; - return res; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return "RegionKey [" + - "parentColKey=" + parentColKey + - ", featureIdx=" + featureIdx + - ", regBlockIdx=" + regBlockIdx + - ", trainingUUID=" + trainingUUID + - ']'; - } - } - - /** - * Affinity service for region projections cache. - * - * @return Affinity service for region projections cache. - */ - public static Affinity affinity() { - return Ignition.localIgnite().affinity(CACHE_NAME); - } - - /** - * Get or create region projections cache. - * - * @param ignite Ignite instance. - * @return Region projections cache. - */ - public static IgniteCache> getOrCreate(Ignite ignite) { - CacheConfiguration> cfg = new CacheConfiguration<>(); - - // Write to primary. - cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC); - - // Atomic transactions only. - cfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); - - // No eviction. - cfg.setEvictionPolicy(null); - - // No copying of values. - cfg.setCopyOnRead(false); - - // Cache is partitioned. - cfg.setCacheMode(CacheMode.PARTITIONED); - - cfg.setBackups(0); - - cfg.setOnheapCacheEnabled(true); - - cfg.setName(CACHE_NAME); - - return ignite.getOrCreateCache(cfg); - } - - /** - * Get region projections in the form of map (regionIndex -> regionProjections). - * - * @param featureIdx Feature index. - * @param maxDepth Max depth of decision tree. - * @param regionIndexes Indexes of regions for which we want get projections. - * @param blockSize Size of regions block. - * @param affinity Affinity function. - * @param trainingUUID UUID of training. - * @param ignite Ignite instance. - * @return Region projections in the form of map (regionIndex -> regionProjections). - */ - public static Map projectionsOfRegions(int featureIdx, int maxDepth, - IntStream regionIndexes, int blockSize, IgniteFunction affinity, UUID trainingUUID, - Ignite ignite) { - HashMap regsForSearch = new HashMap<>(); - IgniteCache> cache = getOrCreate(ignite); - - PrimitiveIterator.OfInt itr = regionIndexes.iterator(); - - int curBlockIdx = -1; - List block = null; - - Object affinityKey = affinity.apply(featureIdx); - - while (itr.hasNext()) { - int i = itr.nextInt(); - - int blockIdx = i / blockSize; - - if (blockIdx != curBlockIdx) { - block = cache.localPeek(key(featureIdx, blockIdx, affinityKey, trainingUUID)); - curBlockIdx = blockIdx; - } - - if (block == null) - throw new IllegalStateException("Unexpected null block at index " + i); - - RegionProjection reg = block.get(i % blockSize); - - if (reg.depth() < maxDepth) - regsForSearch.put(i, reg); - } - - return regsForSearch; - } - - /** - * Returns projections of regions on given feature filtered by maximal depth in the form of (region index -> region - * projection). - * - * @param featureIdx Feature index. - * @param maxDepth Maximal depth of the tree. - * @param regsCnt Count of regions. - * @param blockSize Size of regions blocks. - * @param affinity Affinity function. - * @param trainingUUID UUID of training. - * @param ignite Ignite instance. - * @return Projections of regions on given feature filtered by maximal depth in the form of (region index -> region - * projection). - */ - public static Map projectionsOfFeature(int featureIdx, int maxDepth, int regsCnt, - int blockSize, IgniteFunction affinity, UUID trainingUUID, Ignite ignite) { - return projectionsOfRegions(featureIdx, maxDepth, IntStream.range(0, regsCnt), blockSize, affinity, trainingUUID, ignite); - } - - /** - * Construct key for projections cache. - * - * @param featureIdx Feature index. - * @param regBlockIdx Region block index. - * @param parentColKey Column key of cache used as input for {@link ColumnDecisionTreeTrainer}. - * @param uuid UUID of training. - * @return Key for projections cache. - */ - public static RegionKey key(int featureIdx, int regBlockIdx, Object parentColKey, UUID uuid) { - return new RegionKey(featureIdx, regBlockIdx, parentColKey, uuid); - } - - /** - * Clear data from projections cache related to given training. - * - * @param featuresCnt Features count. - * @param regs Regions count. - * @param aff Affinity function. - * @param uuid UUID of training. - * @param ignite Ignite instance. - */ - public static void clear(int featuresCnt, int regs, IgniteBiFunction aff, UUID uuid, - Ignite ignite) { - Set toRmv = IntStream.range(0, featuresCnt).boxed(). - flatMap(fIdx -> IntStream.range(0, regs).boxed().map(reg -> new IgniteBiTuple<>(fIdx, reg))). - map(t -> key(t.get1(), t.get2(), aff.apply(t.get1(), ignite), uuid)). - collect(Collectors.toSet()); - - getOrCreate(ignite).removeAll(toRmv); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/SplitCache.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/SplitCache.java deleted file mode 100644 index ecbc86198ece6..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/SplitCache.java +++ /dev/null @@ -1,206 +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.trees.trainers.columnbased.caches; - -import java.util.Collection; -import java.util.Collections; -import java.util.Set; -import java.util.UUID; -import java.util.function.Function; -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.Ignition; -import org.apache.ignite.cache.CacheAtomicityMode; -import org.apache.ignite.cache.CacheMode; -import org.apache.ignite.cache.CacheWriteSynchronizationMode; -import org.apache.ignite.cache.affinity.Affinity; -import org.apache.ignite.cache.affinity.AffinityKeyMapped; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.internal.processors.cache.CacheEntryImpl; -import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.trees.trainers.columnbased.ColumnDecisionTreeTrainer; - -/** - * Class for working with cache used for storing of best splits during training with {@link ColumnDecisionTreeTrainer}. - */ -public class SplitCache { - /** Name of splits cache. */ - public static final String CACHE_NAME = "COLUMN_DECISION_TREE_TRAINER_SPLIT_CACHE_NAME"; - - /** - * Class used for keys in the splits cache. - */ - public static class SplitKey { - /** UUID of current training. */ - private final UUID trainingUUID; - - /** Affinity key of input data. */ - @AffinityKeyMapped - private final Object parentColKey; - - /** Index of feature by which the split is made. */ - private final int featureIdx; - - /** - * Construct SplitKey. - * - * @param trainingUUID UUID of the training. - * @param parentColKey Affinity key used to ensure that cache entry for given feature will be on the same node - * as column with that feature in input. - * @param featureIdx Feature index. - */ - public SplitKey(UUID trainingUUID, Object parentColKey, int featureIdx) { - this.trainingUUID = trainingUUID; - this.featureIdx = featureIdx; - this.parentColKey = parentColKey; - } - - /** Get UUID of current training. */ - public UUID trainingUUID() { - return trainingUUID; - } - - /** - * Get feature index. - * - * @return Feature index. - */ - public int featureIdx() { - return featureIdx; - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - - SplitKey splitKey = (SplitKey)o; - - if (featureIdx != splitKey.featureIdx) - return false; - return trainingUUID != null ? trainingUUID.equals(splitKey.trainingUUID) : splitKey.trainingUUID == null; - - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - int res = trainingUUID != null ? trainingUUID.hashCode() : 0; - res = 31 * res + featureIdx; - return res; - } - } - - /** - * Construct the key for splits cache. - * - * @param featureIdx Feature index. - * @param parentColKey Affinity key used to ensure that cache entry for given feature will be on the same node as - * column with that feature in input. - * @param uuid UUID of current training. - * @return Key for splits cache. - */ - public static SplitKey key(int featureIdx, Object parentColKey, UUID uuid) { - return new SplitKey(uuid, parentColKey, featureIdx); - } - - /** - * Get or create splits cache. - * - * @param ignite Ignite instance. - * @return Splits cache. - */ - public static IgniteCache> getOrCreate(Ignite ignite) { - CacheConfiguration> cfg = new CacheConfiguration<>(); - - // Write to primary. - cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC); - - // Atomic transactions only. - cfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); - - // No eviction. - cfg.setEvictionPolicy(null); - - // No copying of values. - cfg.setCopyOnRead(false); - - // Cache is partitioned. - cfg.setCacheMode(CacheMode.PARTITIONED); - - cfg.setBackups(0); - - cfg.setOnheapCacheEnabled(true); - - cfg.setName(CACHE_NAME); - - return ignite.getOrCreateCache(cfg); - } - - /** - * Affinity function used in splits cache. - * - * @return Affinity function used in splits cache. - */ - public static Affinity affinity() { - return Ignition.localIgnite().affinity(CACHE_NAME); - } - - /** - * Returns local entries for keys corresponding to {@code featureIndexes}. - * - * @param featureIndexes Index of features. - * @param affinity Affinity function. - * @param trainingUUID UUID of training. - * @return local entries for keys corresponding to {@code featureIndexes}. - */ - public static Iterable>> localEntries( - Set featureIndexes, - IgniteBiFunction affinity, - UUID trainingUUID) { - Ignite ignite = Ignition.localIgnite(); - Set keys = featureIndexes.stream().map(fIdx -> new SplitKey(trainingUUID, affinity.apply(fIdx, ignite), fIdx)).collect(Collectors.toSet()); - - Collection locKeys = affinity().mapKeysToNodes(keys).getOrDefault(ignite.cluster().localNode(), Collections.emptyList()); - - return () -> { - Function>> f = k -> (new CacheEntryImpl<>(k, getOrCreate(ignite).localPeek(k))); - return locKeys.stream().map(f).iterator(); - }; - } - - /** - * Clears data related to current training from splits cache related to given training. - * - * @param featuresCnt Count of features. - * @param affinity Affinity function. - * @param uuid UUID of the given training. - * @param ignite Ignite instance. - */ - public static void clear(int featuresCnt, IgniteBiFunction affinity, UUID uuid, - Ignite ignite) { - Set toRmv = IntStream.range(0, featuresCnt).boxed().map(fIdx -> new SplitKey(uuid, affinity.apply(fIdx, ignite), fIdx)).collect(Collectors.toSet()); - - getOrCreate(ignite).removeAll(toRmv); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/package-info.java deleted file mode 100644 index 0a488ab1f0289..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/caches/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. - */ - -/** - * - * Contains cache configurations for columnbased decision tree trainer with some related logic. - */ -package org.apache.ignite.ml.trees.trainers.columnbased.caches; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/contsplitcalcs/ContinuousSplitCalculators.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/contsplitcalcs/ContinuousSplitCalculators.java deleted file mode 100644 index 9fd4c6682b0d8..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/contsplitcalcs/ContinuousSplitCalculators.java +++ /dev/null @@ -1,34 +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.trees.trainers.columnbased.contsplitcalcs; - -import org.apache.ignite.Ignite; -import org.apache.ignite.ml.math.functions.IgniteCurriedBiFunction; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.trees.trainers.columnbased.ColumnDecisionTreeTrainerInput; - -/** Continuous Split Calculators. */ -public class ContinuousSplitCalculators { - /** Variance split calculator. */ - public static IgniteFunction VARIANCE = input -> - new VarianceSplitCalculator(); - - /** Gini split calculator. */ - public static IgniteCurriedBiFunction GINI = ignite -> - input -> new GiniSplitCalculator(input.labels(ignite)); -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/contsplitcalcs/GiniSplitCalculator.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/contsplitcalcs/GiniSplitCalculator.java deleted file mode 100644 index 259c84c76fddf..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/contsplitcalcs/GiniSplitCalculator.java +++ /dev/null @@ -1,234 +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.trees.trainers.columnbased.contsplitcalcs; - -import it.unimi.dsi.fastutil.doubles.Double2IntArrayMap; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.util.HashMap; -import java.util.Map; -import java.util.PrimitiveIterator; -import java.util.stream.DoubleStream; -import org.apache.ignite.ml.trees.ContinuousRegionInfo; -import org.apache.ignite.ml.trees.ContinuousSplitCalculator; -import org.apache.ignite.ml.trees.trainers.columnbased.vectors.ContinuousSplitInfo; -import org.apache.ignite.ml.trees.trainers.columnbased.vectors.SplitInfo; - -/** - * Calculator for Gini impurity. - */ -public class GiniSplitCalculator implements ContinuousSplitCalculator { - /** Mapping assigning index to each member value */ - private final Map mapping = new Double2IntArrayMap(); - - /** - * Create Gini split calculator from labels. - * - * @param labels Labels. - */ - public GiniSplitCalculator(double[] labels) { - int i = 0; - - for (double label : labels) { - if (!mapping.containsKey(label)) { - mapping.put(label, i); - i++; - } - } - } - - /** {@inheritDoc} */ - @Override public GiniData calculateRegionInfo(DoubleStream s, int l) { - PrimitiveIterator.OfDouble itr = s.iterator(); - - Map m = new HashMap<>(); - - int size = 0; - - while (itr.hasNext()) { - size++; - m.compute(itr.next(), (a, i) -> i != null ? i + 1 : 1); - } - - double c2 = m.values().stream().mapToDouble(v -> v * v).sum(); - - int[] cnts = new int[mapping.size()]; - - m.forEach((key, value) -> cnts[mapping.get(key)] = value); - - return new GiniData(size != 0 ? 1 - c2 / (size * size) : 0.0, size, cnts, c2); - } - - /** {@inheritDoc} */ - @Override public SplitInfo splitRegion(Integer[] s, double[] values, double[] labels, int regionIdx, - GiniData d) { - int size = d.getSize(); - - double lg = 0.0; - double rg = d.impurity(); - - double lc2 = 0.0; - double rc2 = d.c2; - int lSize = 0; - - double minImpurity = d.impurity() * size; - double curThreshold; - double curImpurity; - double threshold = Double.NEGATIVE_INFINITY; - - int i = 0; - int nextIdx = s[0]; - i++; - double[] lrImps = new double[] {0.0, d.impurity(), lc2, rc2}; - - int[] lMapCur = new int[d.counts().length]; - int[] rMapCur = new int[d.counts().length]; - - System.arraycopy(d.counts(), 0, rMapCur, 0, d.counts().length); - - int[] lMap = new int[d.counts().length]; - int[] rMap = new int[d.counts().length]; - - System.arraycopy(d.counts(), 0, rMap, 0, d.counts().length); - - do { - // Process all values equal to prev. - while (i < s.length) { - moveLeft(labels[nextIdx], i, size - i, lMapCur, rMapCur, lrImps); - curImpurity = (i * lrImps[0] + (size - i) * lrImps[1]); - curThreshold = values[nextIdx]; - - if (values[nextIdx] != values[(nextIdx = s[i++])]) { - if (curImpurity < minImpurity) { - lSize = i - 1; - - lg = lrImps[0]; - rg = lrImps[1]; - - lc2 = lrImps[2]; - rc2 = lrImps[3]; - - System.arraycopy(lMapCur, 0, lMap, 0, lMapCur.length); - System.arraycopy(rMapCur, 0, rMap, 0, rMapCur.length); - - minImpurity = curImpurity; - threshold = curThreshold; - } - - break; - } - } - } - while (i < s.length - 1); - - if (lSize == size || lSize == 0) - return null; - - GiniData lData = new GiniData(lg, lSize, lMap, lc2); - int rSize = size - lSize; - GiniData rData = new GiniData(rg, rSize, rMap, rc2); - - return new ContinuousSplitInfo<>(regionIdx, threshold, lData, rData); - } - - /** - * Add point to the left interval and remove it from the right interval and calculate necessary statistics on - * intervals with new bounds. - */ - private void moveLeft(double x, int lSize, int rSize, int[] lMap, int[] rMap, double[] data) { - double lc2 = data[2]; - double rc2 = data[3]; - - Integer idx = mapping.get(x); - - int cxl = lMap[idx]; - int cxr = rMap[idx]; - - lc2 += 2 * cxl + 1; - rc2 -= 2 * cxr - 1; - - lMap[idx] += 1; - rMap[idx] -= 1; - - data[0] = 1 - lc2 / (lSize * lSize); - data[1] = 1 - rc2 / (rSize * rSize); - - data[2] = lc2; - data[3] = rc2; - } - - /** - * Data used for gini impurity calculations. - */ - public static class GiniData extends ContinuousRegionInfo { - /** Sum of squares of counts of each label. */ - private double c2; - - /** Counts of each label. On i-th position there is count of label which is mapped to index i. */ - private int[] m; - - /** - * Create Gini data. - * - * @param impurity Impurity (i.e. Gini impurity). - * @param size Count of samples. - * @param m Counts of each label. - * @param c2 Sum of squares of counts of each label. - */ - public GiniData(double impurity, int size, int[] m, double c2) { - super(impurity, size); - this.m = m; - this.c2 = c2; - } - - /** - * No-op constructor for serialization/deserialization.. - */ - public GiniData() { - // No-op. - } - - /** Get counts of each label. */ - public int[] counts() { - return m; - } - - /** {@inheritDoc} */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - super.writeExternal(out); - out.writeDouble(c2); - out.writeInt(m.length); - for (int i : m) - out.writeInt(i); - - } - - /** {@inheritDoc} */ - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - super.readExternal(in); - - c2 = in.readDouble(); - int size = in.readInt(); - m = new int[size]; - - for (int i = 0; i < size; i++) - m[i] = in.readInt(); - } - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/contsplitcalcs/VarianceSplitCalculator.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/contsplitcalcs/VarianceSplitCalculator.java deleted file mode 100644 index 66c54f299f805..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/contsplitcalcs/VarianceSplitCalculator.java +++ /dev/null @@ -1,179 +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.trees.trainers.columnbased.contsplitcalcs; - -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.util.PrimitiveIterator; -import java.util.stream.DoubleStream; -import org.apache.ignite.ml.trees.ContinuousRegionInfo; -import org.apache.ignite.ml.trees.ContinuousSplitCalculator; -import org.apache.ignite.ml.trees.trainers.columnbased.vectors.ContinuousSplitInfo; -import org.apache.ignite.ml.trees.trainers.columnbased.vectors.SplitInfo; - -/** - * Calculator of variance in a given region. - */ -public class VarianceSplitCalculator implements ContinuousSplitCalculator { - /** - * Data used in variance calculations. - */ - public static class VarianceData extends ContinuousRegionInfo { - /** Mean value in a given region. */ - double mean; - - /** - * @param var Variance in this region. - * @param size Size of data for which variance is calculated. - * @param mean Mean value in this region. - */ - public VarianceData(double var, int size, double mean) { - super(var, size); - this.mean = mean; - } - - /** - * No-op constructor. For serialization/deserialization. - */ - public VarianceData() { - // No-op. - } - - /** {@inheritDoc} */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - super.writeExternal(out); - out.writeDouble(mean); - } - - /** {@inheritDoc} */ - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - super.readExternal(in); - mean = in.readDouble(); - } - - /** - * Returns mean. - */ - public double mean() { - return mean; - } - } - - /** {@inheritDoc} */ - @Override public VarianceData calculateRegionInfo(DoubleStream s, int size) { - PrimitiveIterator.OfDouble itr = s.iterator(); - int i = 0; - - double mean = 0.0; - double m2 = 0.0; - - // Here we calculate variance and mean by incremental computation. - while (itr.hasNext()) { - i++; - double x = itr.next(); - double delta = x - mean; - mean += delta / i; - double delta2 = x - mean; - m2 += delta * delta2; - } - - return new VarianceData(m2 / i, size, mean); - } - - /** {@inheritDoc} */ - @Override public SplitInfo splitRegion(Integer[] s, double[] values, double[] labels, int regionIdx, - VarianceData d) { - int size = d.getSize(); - - double lm2 = 0.0; - double rm2 = d.impurity() * size; - int lSize = size; - - double lMean = 0.0; - double rMean = d.mean; - - double minImpurity = d.impurity() * size; - double curThreshold; - double curImpurity; - double threshold = Double.NEGATIVE_INFINITY; - - int i = 0; - int nextIdx = s[0]; - i++; - double[] lrImps = new double[] {lm2, rm2, lMean, rMean}; - - do { - // Process all values equal to prev. - while (i < s.length) { - moveLeft(labels[nextIdx], lrImps[2], i, lrImps[0], lrImps[3], size - i, lrImps[1], lrImps); - curImpurity = (lrImps[0] + lrImps[1]); - curThreshold = values[nextIdx]; - - if (values[nextIdx] != values[(nextIdx = s[i++])]) { - if (curImpurity < minImpurity) { - lSize = i - 1; - - lm2 = lrImps[0]; - rm2 = lrImps[1]; - - lMean = lrImps[2]; - rMean = lrImps[3]; - - minImpurity = curImpurity; - threshold = curThreshold; - } - - break; - } - } - } - while (i < s.length - 1); - - if (lSize == size) - return null; - - VarianceData lData = new VarianceData(lm2 / (lSize != 0 ? lSize : 1), lSize, lMean); - int rSize = size - lSize; - VarianceData rData = new VarianceData(rm2 / (rSize != 0 ? rSize : 1), rSize, rMean); - - return new ContinuousSplitInfo<>(regionIdx, threshold, lData, rData); - } - - /** - * Add point to the left interval and remove it from the right interval and calculate necessary statistics on - * intervals with new bounds. - */ - private void moveLeft(double x, double lMean, int lSize, double lm2, double rMean, int rSize, double rm2, - double[] data) { - // We add point to the left interval. - double lDelta = x - lMean; - double lMeanNew = lMean + lDelta / lSize; - double lm2New = lm2 + lDelta * (x - lMeanNew); - - // We remove point from the right interval. lSize + 1 is the size of right interval before removal. - double rMeanNew = (rMean * (rSize + 1) - x) / rSize; - double rm2New = rm2 - (x - rMean) * (x - rMeanNew); - - data[0] = lm2New; - data[1] = rm2New; - - data[2] = lMeanNew; - data[3] = rMeanNew; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/contsplitcalcs/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/contsplitcalcs/package-info.java deleted file mode 100644 index 08c8a75bd7d8e..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/contsplitcalcs/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. - */ - -/** - * - * Calculators of splits by continuous features. - */ -package org.apache.ignite.ml.trees.trainers.columnbased.contsplitcalcs; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/package-info.java deleted file mode 100644 index 85239141eb0b4..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/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. - */ - -/** - * - * Contains column based decision tree algorithms. - */ -package org.apache.ignite.ml.trees.trainers.columnbased; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/regcalcs/RegionCalculators.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/regcalcs/RegionCalculators.java deleted file mode 100644 index 5c4b354854eeb..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/regcalcs/RegionCalculators.java +++ /dev/null @@ -1,85 +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.trees.trainers.columnbased.regcalcs; - -import it.unimi.dsi.fastutil.doubles.Double2IntOpenHashMap; -import java.util.Comparator; -import java.util.HashMap; -import java.util.Map; -import java.util.PrimitiveIterator; -import java.util.stream.DoubleStream; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.trees.trainers.columnbased.ColumnDecisionTreeTrainerInput; - -/** Some commonly used functions for calculations of regions of space which correspond to decision tree leaf nodes. */ -public class RegionCalculators { - /** Mean value in the region. */ - public static final IgniteFunction MEAN = s -> s.average().orElse(0.0); - - /** Most common value in the region. */ - public static final IgniteFunction MOST_COMMON = - s -> { - PrimitiveIterator.OfDouble itr = s.iterator(); - Map voc = new HashMap<>(); - - while (itr.hasNext()) - voc.compute(itr.next(), (d, i) -> i != null ? i + 1 : 0); - - return voc.entrySet().stream().max(Comparator.comparing(Map.Entry::getValue)).map(Map.Entry::getKey).orElse(0.0); - }; - - /** Variance of a region. */ - public static final IgniteFunction> VARIANCE = input -> - s -> { - PrimitiveIterator.OfDouble itr = s.iterator(); - int i = 0; - - double mean = 0.0; - double m2 = 0.0; - - while (itr.hasNext()) { - i++; - double x = itr.next(); - double delta = x - mean; - mean += delta / i; - double delta2 = x - mean; - m2 += delta * delta2; - } - - return i > 0 ? m2 / i : 0.0; - }; - - /** Gini impurity of a region. */ - public static final IgniteFunction> GINI = input -> - s -> { - PrimitiveIterator.OfDouble itr = s.iterator(); - - Double2IntOpenHashMap m = new Double2IntOpenHashMap(); - - int size = 0; - - while (itr.hasNext()) { - size++; - m.compute(itr.next(), (a, i) -> i != null ? i + 1 : 1); - } - - double c2 = m.values().stream().mapToDouble(v -> v * v).sum(); - - return size != 0 ? 1 - c2 / (size * size) : 0.0; - }; -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/CategoricalFeatureProcessor.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/CategoricalFeatureProcessor.java deleted file mode 100644 index 3232ac209c7c5..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/CategoricalFeatureProcessor.java +++ /dev/null @@ -1,212 +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.trees.trainers.columnbased.vectors; - -import com.zaxxer.sparsebits.SparseBitSet; -import java.util.Arrays; -import java.util.BitSet; -import java.util.Comparator; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.DoubleStream; -import java.util.stream.Stream; -import java.util.stream.StreamSupport; -import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.trees.CategoricalRegionInfo; -import org.apache.ignite.ml.trees.CategoricalSplitInfo; -import org.apache.ignite.ml.trees.RegionInfo; -import org.apache.ignite.ml.trees.trainers.columnbased.ColumnDecisionTreeTrainer; -import org.apache.ignite.ml.trees.trainers.columnbased.RegionProjection; - -import static org.apache.ignite.ml.trees.trainers.columnbased.vectors.FeatureVectorProcessorUtils.splitByBitSet; - -/** - * Categorical feature vector processor implementation used by {@link ColumnDecisionTreeTrainer}. - */ -public class CategoricalFeatureProcessor - implements FeatureProcessor> { - /** Count of categories for this feature. */ - private final int catsCnt; - - /** Function for calculating impurity of a given region of points. */ - private final IgniteFunction calc; - - /** - * @param calc Function for calculating impurity of a given region of points. - * @param catsCnt Number of categories. - */ - public CategoricalFeatureProcessor(IgniteFunction calc, int catsCnt) { - this.calc = calc; - this.catsCnt = catsCnt; - } - - /** */ - private SplitInfo split(BitSet leftCats, int intervalIdx, Map mapping, - Integer[] sampleIndexes, double[] values, double[] labels, double impurity) { - Map> leftRight = Arrays.stream(sampleIndexes). - collect(Collectors.partitioningBy((smpl) -> leftCats.get(mapping.get((int)values[smpl])))); - - List left = leftRight.get(true); - int leftSize = left.size(); - double leftImpurity = calc.apply(left.stream().mapToDouble(s -> labels[s])); - - List right = leftRight.get(false); - int rightSize = right.size(); - double rightImpurity = calc.apply(right.stream().mapToDouble(s -> labels[s])); - - int totalSize = leftSize + rightSize; - - // Result of this call will be sent back to trainer node, we do not need vectors inside of sent data. - CategoricalSplitInfo res = new CategoricalSplitInfo<>(intervalIdx, - new CategoricalRegionInfo(leftImpurity, null), // cats can be computed on the last step. - new CategoricalRegionInfo(rightImpurity, null), - leftCats); - - res.setInfoGain(impurity - (double)leftSize / totalSize * leftImpurity - (double)rightSize / totalSize * rightImpurity); - return res; - } - - /** - * Get a stream of subsets given categories count. - * - * @param catsCnt categories count. - * @return Stream of subsets given categories count. - */ - private Stream powerSet(int catsCnt) { - Iterable iterable = () -> new PSI(catsCnt); - return StreamSupport.stream(iterable.spliterator(), false); - } - - /** {@inheritDoc} */ - @Override public SplitInfo findBestSplit(RegionProjection regionPrj, double[] values, - double[] labels, int regIdx) { - Map mapping = mapping(regionPrj.data().cats()); - - return powerSet(regionPrj.data().cats().length()). - map(s -> split(s, regIdx, mapping, regionPrj.sampleIndexes(), values, labels, regionPrj.data().impurity())). - max(Comparator.comparingDouble(SplitInfo::infoGain)). - orElse(null); - } - - /** {@inheritDoc} */ - @Override public RegionProjection createInitialRegion(Integer[] sampleIndexes, - double[] values, double[] labels) { - BitSet set = new BitSet(); - set.set(0, catsCnt); - - Double impurity = calc.apply(Arrays.stream(labels)); - - return new RegionProjection<>(sampleIndexes, new CategoricalRegionInfo(impurity, set), 0); - } - - /** {@inheritDoc} */ - @Override public SparseBitSet calculateOwnershipBitSet(RegionProjection regionPrj, - double[] values, - CategoricalSplitInfo s) { - SparseBitSet res = new SparseBitSet(); - Arrays.stream(regionPrj.sampleIndexes()).forEach(smpl -> res.set(smpl, s.bitSet().get((int)values[smpl]))); - return res; - } - - /** {@inheritDoc} */ - @Override public IgniteBiTuple performSplit(SparseBitSet bs, - RegionProjection reg, CategoricalRegionInfo leftData, CategoricalRegionInfo rightData) { - return performSplitGeneric(bs, null, reg, leftData, rightData); - } - - /** {@inheritDoc} */ - @Override public IgniteBiTuple performSplitGeneric( - SparseBitSet bs, double[] values, RegionProjection reg, RegionInfo leftData, - RegionInfo rightData) { - int depth = reg.depth(); - - int lSize = bs.cardinality(); - int rSize = reg.sampleIndexes().length - lSize; - IgniteBiTuple lrSamples = splitByBitSet(lSize, rSize, reg.sampleIndexes(), bs); - BitSet leftCats = calculateCats(lrSamples.get1(), values); - CategoricalRegionInfo lInfo = new CategoricalRegionInfo(leftData.impurity(), leftCats); - - // TODO: IGNITE-5892 Check how it will work with sparse data. - BitSet rightCats = calculateCats(lrSamples.get2(), values); - CategoricalRegionInfo rInfo = new CategoricalRegionInfo(rightData.impurity(), rightCats); - - RegionProjection rPrj = new RegionProjection<>(lrSamples.get2(), rInfo, depth + 1); - RegionProjection lPrj = new RegionProjection<>(lrSamples.get1(), lInfo, depth + 1); - return new IgniteBiTuple<>(lPrj, rPrj); - } - - /** - * Powerset iterator. Iterates not over the whole powerset, but on half of it. - */ - private static class PSI implements Iterator { - - /** Current subset number. */ - private int i = 1; // We are not interested in {emptyset, set} split and therefore start from 1. - - /** Size of set, subsets of which we iterate over. */ - final int size; - - /** - * @param bitCnt Size of set, subsets of which we iterate over. - */ - PSI(int bitCnt) { - this.size = 1 << (bitCnt - 1); - } - - /** {@inheritDoc} */ - @Override public boolean hasNext() { - return i < size; - } - - /** {@inheritDoc} */ - @Override public BitSet next() { - BitSet res = BitSet.valueOf(new long[] {i}); - i++; - return res; - } - } - - /** */ - private Map mapping(BitSet bs) { - int bn = 0; - Map res = new HashMap<>(); - - int i = 0; - while ((bn = bs.nextSetBit(bn)) != -1) { - res.put(bn, i); - i++; - bn++; - } - - return res; - } - - /** Get set of categories of given samples */ - private BitSet calculateCats(Integer[] sampleIndexes, double[] values) { - BitSet res = new BitSet(); - - for (int smpl : sampleIndexes) - res.set((int)values[smpl]); - - return res; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/ContinuousFeatureProcessor.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/ContinuousFeatureProcessor.java deleted file mode 100644 index 4117993aa68a2..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/ContinuousFeatureProcessor.java +++ /dev/null @@ -1,111 +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.trees.trainers.columnbased.vectors; - -import com.zaxxer.sparsebits.SparseBitSet; -import java.util.Arrays; -import java.util.Comparator; -import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.ml.trees.ContinuousRegionInfo; -import org.apache.ignite.ml.trees.ContinuousSplitCalculator; -import org.apache.ignite.ml.trees.RegionInfo; -import org.apache.ignite.ml.trees.trainers.columnbased.RegionProjection; - -import static org.apache.ignite.ml.trees.trainers.columnbased.vectors.FeatureVectorProcessorUtils.splitByBitSet; - -/** - * Container of projection of samples on continuous feature. - * - * @param Information about regions. Designed to contain information which will make computations of impurity - * optimal. - */ -public class ContinuousFeatureProcessor implements - FeatureProcessor> { - /** ContinuousSplitCalculator used for calculating of best split of each region. */ - private final ContinuousSplitCalculator calc; - - /** - * @param splitCalc Calculator used for calculating splits. - */ - public ContinuousFeatureProcessor(ContinuousSplitCalculator splitCalc) { - this.calc = splitCalc; - } - - /** {@inheritDoc} */ - @Override public SplitInfo findBestSplit(RegionProjection ri, double[] values, double[] labels, int regIdx) { - SplitInfo res = calc.splitRegion(ri.sampleIndexes(), values, labels, regIdx, ri.data()); - - if (res == null) - return null; - - double lWeight = (double)res.leftData.getSize() / ri.sampleIndexes().length; - double rWeight = (double)res.rightData.getSize() / ri.sampleIndexes().length; - - double infoGain = ri.data().impurity() - lWeight * res.leftData().impurity() - rWeight * res.rightData().impurity(); - res.setInfoGain(infoGain); - - return res; - } - - /** {@inheritDoc} */ - @Override public RegionProjection createInitialRegion(Integer[] samples, double[] values, double[] labels) { - Arrays.sort(samples, Comparator.comparingDouble(s -> values[s])); - return new RegionProjection<>(samples, calc.calculateRegionInfo(Arrays.stream(labels), samples.length), 0); - } - - /** {@inheritDoc} */ - @Override public SparseBitSet calculateOwnershipBitSet(RegionProjection reg, double[] values, - ContinuousSplitInfo s) { - SparseBitSet res = new SparseBitSet(); - - for (int i = 0; i < s.leftData().getSize(); i++) - res.set(reg.sampleIndexes()[i]); - - return res; - } - - /** {@inheritDoc} */ - @Override public IgniteBiTuple performSplit(SparseBitSet bs, - RegionProjection reg, D leftData, D rightData) { - int lSize = leftData.getSize(); - int rSize = rightData.getSize(); - int depth = reg.depth(); - - IgniteBiTuple lrSamples = splitByBitSet(lSize, rSize, reg.sampleIndexes(), bs); - - RegionProjection left = new RegionProjection<>(lrSamples.get1(), leftData, depth + 1); - RegionProjection right = new RegionProjection<>(lrSamples.get2(), rightData, depth + 1); - - return new IgniteBiTuple<>(left, right); - } - - /** {@inheritDoc} */ - @Override public IgniteBiTuple performSplitGeneric(SparseBitSet bs, - double[] labels, RegionProjection reg, RegionInfo leftData, RegionInfo rightData) { - int lSize = bs.cardinality(); - int rSize = reg.sampleIndexes().length - lSize; - int depth = reg.depth(); - - IgniteBiTuple lrSamples = splitByBitSet(lSize, rSize, reg.sampleIndexes(), bs); - - D ld = calc.calculateRegionInfo(Arrays.stream(lrSamples.get1()).mapToDouble(s -> labels[s]), lSize); - D rd = calc.calculateRegionInfo(Arrays.stream(lrSamples.get2()).mapToDouble(s -> labels[s]), rSize); - - return new IgniteBiTuple<>(new RegionProjection<>(lrSamples.get1(), ld, depth + 1), new RegionProjection<>(lrSamples.get2(), rd, depth + 1)); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/ContinuousSplitInfo.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/ContinuousSplitInfo.java deleted file mode 100644 index 8b45cb580ce36..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/ContinuousSplitInfo.java +++ /dev/null @@ -1,71 +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.trees.trainers.columnbased.vectors; - -import org.apache.ignite.ml.trees.RegionInfo; -import org.apache.ignite.ml.trees.nodes.ContinuousSplitNode; -import org.apache.ignite.ml.trees.nodes.SplitNode; - -/** - * Information about split of continuous region. - * - * @param Class encapsulating information about the region. - */ -public class ContinuousSplitInfo extends SplitInfo { - /** - * Threshold used for split. - * Samples with values less or equal than this go to left region, others go to the right region. - */ - private final double threshold; - - /** - * @param regionIdx Index of region being split. - * @param threshold Threshold used for split. Samples with values less or equal than this go to left region, others - * go to the right region. - * @param leftData Information about left subregion. - * @param rightData Information about right subregion. - */ - public ContinuousSplitInfo(int regionIdx, double threshold, D leftData, D rightData) { - super(regionIdx, leftData, rightData); - this.threshold = threshold; - } - - /** {@inheritDoc} */ - @Override public SplitNode createSplitNode(int featureIdx) { - return new ContinuousSplitNode(threshold, featureIdx); - } - - /** - * Threshold used for splits. - * Samples with values less or equal than this go to left region, others go to the right region. - */ - public double threshold() { - return threshold; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return "ContinuousSplitInfo [" + - "threshold=" + threshold + - ", infoGain=" + infoGain + - ", regionIdx=" + regionIdx + - ", leftData=" + leftData + - ", rightData=" + rightData + - ']'; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/FeatureProcessor.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/FeatureProcessor.java deleted file mode 100644 index 56508e5e37f19..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/FeatureProcessor.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.ml.trees.trainers.columnbased.vectors; - -import com.zaxxer.sparsebits.SparseBitSet; -import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.ml.trees.RegionInfo; -import org.apache.ignite.ml.trees.trainers.columnbased.ColumnDecisionTreeTrainer; -import org.apache.ignite.ml.trees.trainers.columnbased.RegionProjection; - -/** - * Base interface for feature processors used in {@link ColumnDecisionTreeTrainer} - * - * @param Class representing data of regions resulted from split. - * @param Class representing data of split. - */ -public interface FeatureProcessor> { - /** - * Finds best split by this feature among all splits of all regions. - * - * @return best split by this feature among all splits of all regions. - */ - SplitInfo findBestSplit(RegionProjection regionPrj, double[] values, double[] labels, int regIdx); - - /** - * Creates initial region from samples. - * - * @param samples samples. - * @return region. - */ - RegionProjection createInitialRegion(Integer[] samples, double[] values, double[] labels); - - /** - * Calculates the bitset mapping each data point to left (corresponding bit is set) or right subregion. - * - * @param s data used for calculating the split. - * @return Bitset mapping each data point to left (corresponding bit is set) or right subregion. - */ - SparseBitSet calculateOwnershipBitSet(RegionProjection regionPrj, double[] values, S s); - - /** - * Splits given region using bitset which maps data point to left or right subregion. - * This method is present for the vectors of the same type to be able to pass between them information about regions - * and therefore used iff the optimal split is received on feature of the same type. - * - * @param bs Bitset which maps data point to left or right subregion. - * @param leftData Data of the left subregion. - * @param rightData Data of the right subregion. - * @return This feature vector. - */ - IgniteBiTuple performSplit(SparseBitSet bs, RegionProjection reg, D leftData, - D rightData); - - /** - * Splits given region using bitset which maps data point to left or right subregion. This method is used iff the - * optimal split is received on feature of different type, therefore information about regions is limited to the - * {@link RegionInfo} class which is base for all classes used to represent region data. - * - * @param bs Bitset which maps data point to left or right subregion. - * @param leftData Data of the left subregion. - * @param rightData Data of the right subregion. - * @return This feature vector. - */ - IgniteBiTuple performSplitGeneric(SparseBitSet bs, double[] values, - RegionProjection reg, RegionInfo leftData, - RegionInfo rightData); -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/FeatureVectorProcessorUtils.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/FeatureVectorProcessorUtils.java deleted file mode 100644 index 69ff019917501..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/FeatureVectorProcessorUtils.java +++ /dev/null @@ -1,57 +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.trees.trainers.columnbased.vectors; - -import com.zaxxer.sparsebits.SparseBitSet; -import org.apache.ignite.lang.IgniteBiTuple; - -/** Utility class for feature vector processors. */ -public class FeatureVectorProcessorUtils { - /** - * Split target array into two (left and right) arrays by bitset. - * - * @param lSize Left array size; - * @param rSize Right array size. - * @param samples Arrays to split size. - * @param bs Bitset specifying split. - * @return BiTuple containing result of split. - */ - public static IgniteBiTuple splitByBitSet(int lSize, int rSize, Integer[] samples, - SparseBitSet bs) { - Integer[] lArr = new Integer[lSize]; - Integer[] rArr = new Integer[rSize]; - - int lc = 0; - int rc = 0; - - for (int i = 0; i < lSize + rSize; i++) { - int si = samples[i]; - - if (bs.get(si)) { - lArr[lc] = si; - lc++; - } - else { - rArr[rc] = si; - rc++; - } - } - - return new IgniteBiTuple<>(lArr, rArr); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/SampleInfo.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/SampleInfo.java deleted file mode 100644 index 8aa4f79c0b18c..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/SampleInfo.java +++ /dev/null @@ -1,80 +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.trees.trainers.columnbased.vectors; - -import java.io.Externalizable; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; - -/** - * Information about given sample within given fixed feature. - */ -public class SampleInfo implements Externalizable { - /** Value of projection of this sample on given fixed feature. */ - private double val; - - /** Sample index. */ - private int sampleIdx; - - /** - * @param val Value of projection of this sample on given fixed feature. - * @param sampleIdx Sample index. - */ - public SampleInfo(double val, int sampleIdx) { - this.val = val; - this.sampleIdx = sampleIdx; - } - - /** - * No-op constructor used for serialization/deserialization. - */ - public SampleInfo() { - // No-op. - } - - /** - * Get the value of projection of this sample on given fixed feature. - * - * @return Value of projection of this sample on given fixed feature. - */ - public double val() { - return val; - } - - /** - * Get the sample index. - * - * @return Sample index. - */ - public int sampleInd() { - return sampleIdx; - } - - /** {@inheritDoc} */ - @Override public void writeExternal(ObjectOutput out) throws IOException { - out.writeDouble(val); - out.writeInt(sampleIdx); - } - - /** {@inheritDoc} */ - @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - val = in.readDouble(); - sampleIdx = in.readInt(); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/SplitInfo.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/SplitInfo.java deleted file mode 100644 index 124e82f938316..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/SplitInfo.java +++ /dev/null @@ -1,106 +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.trees.trainers.columnbased.vectors; - -import org.apache.ignite.ml.trees.RegionInfo; -import org.apache.ignite.ml.trees.nodes.SplitNode; - -/** - * Class encapsulating information about the split. - * - * @param Class representing information of left and right subregions. - */ -public abstract class SplitInfo { - /** Information gain of this split. */ - protected double infoGain; - - /** Index of the region to split. */ - protected final int regionIdx; - - /** Data of left subregion. */ - protected final D leftData; - - /** Data of right subregion. */ - protected final D rightData; - - /** - * Construct the split info. - * - * @param regionIdx Index of the region to split. - * @param leftData Data of left subregion. - * @param rightData Data of right subregion. - */ - public SplitInfo(int regionIdx, D leftData, D rightData) { - this.regionIdx = regionIdx; - this.leftData = leftData; - this.rightData = rightData; - } - - /** - * Index of region to split. - * - * @return Index of region to split. - */ - public int regionIndex() { - return regionIdx; - } - - /** - * Information gain of the split. - * - * @return Information gain of the split. - */ - public double infoGain() { - return infoGain; - } - - /** - * Data of right subregion. - * - * @return Data of right subregion. - */ - public D rightData() { - return rightData; - } - - /** - * Data of left subregion. - * - * @return Data of left subregion. - */ - public D leftData() { - return leftData; - } - - /** - * Create SplitNode from this split info. - * - * @param featureIdx Index of feature by which goes split. - * @return SplitNode from this split info. - */ - public abstract SplitNode createSplitNode(int featureIdx); - - /** - * Set information gain. - * - * @param infoGain Information gain. - */ - public void setInfoGain(double infoGain) { - this.infoGain = infoGain; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/package-info.java deleted file mode 100644 index 0dea204b530aa..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trees/trainers/columnbased/vectors/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. - */ - -/** - * - * Contains feature containers needed by column based decision tree trainers. - */ -package org.apache.ignite.ml.trees.trainers.columnbased.vectors; \ No newline at end of file diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java index e22a3a5fb4c38..9900f854be9f4 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java @@ -28,7 +28,7 @@ import org.apache.ignite.ml.regressions.RegressionsTestSuite; import org.apache.ignite.ml.svm.SVMTestSuite; import org.apache.ignite.ml.trainers.group.TrainersGroupTestSuite; -import org.apache.ignite.ml.trees.DecisionTreesTestSuite; +import org.apache.ignite.ml.tree.DecisionTreeTestSuite; import org.junit.runner.RunWith; import org.junit.runners.Suite; @@ -41,7 +41,7 @@ RegressionsTestSuite.class, SVMTestSuite.class, ClusteringTestSuite.class, - DecisionTreesTestSuite.class, + DecisionTreeTestSuite.class, KNNTestSuite.class, LocalModelsTest.class, MLPTestSuite.class, diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MnistMLPTestUtil.java b/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MnistMLPTestUtil.java index e624004081a0c..d68b3553c2b2b 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MnistMLPTestUtil.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MnistMLPTestUtil.java @@ -25,11 +25,10 @@ import java.util.stream.Stream; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; -import org.apache.ignite.ml.trees.performance.ColumnDecisionTreeTrainerBenchmark; import org.apache.ignite.ml.util.MnistUtils; /** */ -class MnistMLPTestUtil { +public class MnistMLPTestUtil { /** Name of the property specifying path to training set images. */ private static final String PROP_TRAINING_IMAGES = "mnist.training.images"; @@ -62,7 +61,7 @@ static IgniteBiTuple, Stream loadTrainingSet(int cnt) throws IOException { + public static List loadTrainingSet(int cnt) throws IOException { Properties props = loadMNISTProperties(); return MnistUtils.mnistAsList(props.getProperty(PROP_TRAINING_IMAGES), props.getProperty(PROP_TRAINING_LABELS), new Random(123L), cnt); } @@ -74,7 +73,7 @@ static List loadTrainingSet(int cnt) throws IOExce * @return List of MNIST images. * @throws IOException In case of exception. */ - static List loadTestSet(int cnt) throws IOException { + public static List loadTestSet(int cnt) throws IOException { Properties props = loadMNISTProperties(); return MnistUtils.mnistAsList(props.getProperty(PROP_TEST_IMAGES), props.getProperty(PROP_TEST_LABELS), new Random(123L), cnt); } @@ -83,7 +82,7 @@ static List loadTestSet(int cnt) throws IOExceptio private static Properties loadMNISTProperties() throws IOException { Properties res = new Properties(); - InputStream is = ColumnDecisionTreeTrainerBenchmark.class.getClassLoader().getResourceAsStream("manualrun/trees/columntrees.manualrun.properties"); + InputStream is = MnistMLPTestUtil.class.getClassLoader().getResourceAsStream("manualrun/trees/columntrees.manualrun.properties"); res.load(is); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerIntegrationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerIntegrationTest.java new file mode 100644 index 0000000000000..94bca3f83e2f6 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerIntegrationTest.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.tree; + +import java.util.Arrays; +import java.util.Random; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests for {@link DecisionTreeClassificationTrainer} that require to start the whole Ignite infrastructure. + */ +public class DecisionTreeClassificationTrainerIntegrationTest extends GridCommonAbstractTest { + /** 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()); + } + + /** */ + public void testFit() { + int size = 100; + + CacheConfiguration trainingSetCacheCfg = new CacheConfiguration<>(); + trainingSetCacheCfg.setAffinity(new RendezvousAffinityFunction(false, 10)); + trainingSetCacheCfg.setName("TRAINING_SET"); + + IgniteCache data = ignite.createCache(trainingSetCacheCfg); + + Random rnd = new Random(0); + for (int i = 0; i < size; i++) { + double x = rnd.nextDouble() - 0.5; + data.put(i, new double[]{x, x > 0 ? 1 : 0}); + } + + DecisionTreeClassificationTrainer trainer = new DecisionTreeClassificationTrainer(1, 0); + + DecisionTreeNode tree = trainer.fit( + new CacheBasedDatasetBuilder<>(ignite, data), + (k, v) -> Arrays.copyOf(v, v.length - 1), + (k, v) -> v[v.length - 1] + ); + + assertTrue(tree instanceof DecisionTreeConditionalNode); + + DecisionTreeConditionalNode node = (DecisionTreeConditionalNode) tree; + + assertEquals(0, node.getThreshold(), 1e-3); + + assertTrue(node.getThenNode() instanceof DecisionTreeLeafNode); + assertTrue(node.getElseNode() instanceof DecisionTreeLeafNode); + + DecisionTreeLeafNode thenNode = (DecisionTreeLeafNode) node.getThenNode(); + DecisionTreeLeafNode elseNode = (DecisionTreeLeafNode) node.getElseNode(); + + assertEquals(1, thenNode.getVal(), 1e-10); + assertEquals(0, elseNode.getVal(), 1e-10); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerTest.java new file mode 100644 index 0000000000000..2599bfe2b17e1 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerTest.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.ml.tree; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static junit.framework.TestCase.assertEquals; +import static junit.framework.TestCase.assertTrue; + +/** + * Tests for {@link DecisionTreeClassificationTrainer}. + */ +@RunWith(Parameterized.class) +public class DecisionTreeClassificationTrainerTest { + /** Number of parts to be tested. */ + private static final int[] partsToBeTested = new int[] {1, 2, 3, 4, 5, 7}; + + /** Number of partitions. */ + @Parameterized.Parameter + public int parts; + + @Parameterized.Parameters(name = "Data divided on {0} partitions") + public static Iterable data() { + List res = new ArrayList<>(); + for (int part : partsToBeTested) + res.add(new Integer[] {part}); + + return res; + } + + /** */ + @Test + public void testFit() { + int size = 100; + + Map data = new HashMap<>(); + + Random rnd = new Random(0); + for (int i = 0; i < size; i++) { + double x = rnd.nextDouble() - 0.5; + data.put(i, new double[]{x, x > 0 ? 1 : 0}); + } + + DecisionTreeClassificationTrainer trainer = new DecisionTreeClassificationTrainer(1, 0); + + DecisionTreeNode tree = trainer.fit( + new LocalDatasetBuilder<>(data, parts), + (k, v) -> Arrays.copyOf(v, v.length - 1), + (k, v) -> v[v.length - 1] + ); + + assertTrue(tree instanceof DecisionTreeConditionalNode); + + DecisionTreeConditionalNode node = (DecisionTreeConditionalNode) tree; + + assertEquals(0, node.getThreshold(), 1e-3); + + assertTrue(node.getThenNode() instanceof DecisionTreeLeafNode); + assertTrue(node.getElseNode() instanceof DecisionTreeLeafNode); + + DecisionTreeLeafNode thenNode = (DecisionTreeLeafNode) node.getThenNode(); + DecisionTreeLeafNode elseNode = (DecisionTreeLeafNode) node.getElseNode(); + + assertEquals(1, thenNode.getVal(), 1e-10); + assertEquals(0, elseNode.getVal(), 1e-10); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerIntegrationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerIntegrationTest.java new file mode 100644 index 0000000000000..754ff20f8046e --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerIntegrationTest.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.tree; + +import java.util.Arrays; +import java.util.Random; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests for {@link DecisionTreeRegressionTrainer} that require to start the whole Ignite infrastructure. + */ +public class DecisionTreeRegressionTrainerIntegrationTest extends GridCommonAbstractTest { + /** 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()); + } + + /** */ + public void testFit() { + int size = 100; + + CacheConfiguration trainingSetCacheCfg = new CacheConfiguration<>(); + trainingSetCacheCfg.setAffinity(new RendezvousAffinityFunction(false, 10)); + trainingSetCacheCfg.setName("TRAINING_SET"); + + IgniteCache data = ignite.createCache(trainingSetCacheCfg); + + Random rnd = new Random(0); + for (int i = 0; i < size; i++) { + double x = rnd.nextDouble() - 0.5; + data.put(i, new double[]{x, x > 0 ? 1 : 0}); + } + + DecisionTreeRegressionTrainer trainer = new DecisionTreeRegressionTrainer(1, 0); + + DecisionTreeNode tree = trainer.fit( + new CacheBasedDatasetBuilder<>(ignite, data), + (k, v) -> Arrays.copyOf(v, v.length - 1), + (k, v) -> v[v.length - 1] + ); + + assertTrue(tree instanceof DecisionTreeConditionalNode); + + DecisionTreeConditionalNode node = (DecisionTreeConditionalNode) tree; + + assertEquals(0, node.getThreshold(), 1e-3); + + assertTrue(node.getThenNode() instanceof DecisionTreeLeafNode); + assertTrue(node.getElseNode() instanceof DecisionTreeLeafNode); + + DecisionTreeLeafNode thenNode = (DecisionTreeLeafNode) node.getThenNode(); + DecisionTreeLeafNode elseNode = (DecisionTreeLeafNode) node.getElseNode(); + + assertEquals(1, thenNode.getVal(), 1e-10); + assertEquals(0, elseNode.getVal(), 1e-10); + } +} 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 new file mode 100644 index 0000000000000..3bdbf60675a3f --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerTest.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.ml.tree; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static junit.framework.TestCase.assertEquals; +import static junit.framework.TestCase.assertTrue; + +/** + * Tests for {@link DecisionTreeRegressionTrainer}. + */ +@RunWith(Parameterized.class) +public class DecisionTreeRegressionTrainerTest { + /** Number of parts to be tested. */ + private static final int[] partsToBeTested = new int[] {1, 2, 3, 4, 5, 7}; + + /** Number of partitions. */ + @Parameterized.Parameter + public int parts; + + @Parameterized.Parameters(name = "Data divided on {0} partitions") + public static Iterable data() { + List res = new ArrayList<>(); + for (int part : partsToBeTested) + res.add(new Integer[] {part}); + + return res; + } + + /** */ + @Test + public void testFit() { + int size = 100; + + Map data = new HashMap<>(); + + Random rnd = new Random(0); + for (int i = 0; i < size; i++) { + double x = rnd.nextDouble() - 0.5; + data.put(i, new double[]{x, x > 0 ? 1 : 0}); + } + + DecisionTreeRegressionTrainer trainer = new DecisionTreeRegressionTrainer(1, 0); + + DecisionTreeNode tree = trainer.fit( + new LocalDatasetBuilder<>(data, parts), + (k, v) -> Arrays.copyOf(v, v.length - 1), + (k, v) -> v[v.length - 1] + ); + + assertTrue(tree instanceof DecisionTreeConditionalNode); + + DecisionTreeConditionalNode node = (DecisionTreeConditionalNode) tree; + + assertEquals(0, node.getThreshold(), 1e-3); + + assertTrue(node.getThenNode() instanceof DecisionTreeLeafNode); + assertTrue(node.getElseNode() instanceof DecisionTreeLeafNode); + + DecisionTreeLeafNode thenNode = (DecisionTreeLeafNode) node.getThenNode(); + DecisionTreeLeafNode elseNode = (DecisionTreeLeafNode) node.getElseNode(); + + assertEquals(1, thenNode.getVal(), 1e-10); + assertEquals(0, elseNode.getVal(), 1e-10); + } +} 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 new file mode 100644 index 0000000000000..2cbb486c3df5f --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeTestSuite.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.tree; + +import org.apache.ignite.ml.tree.data.DecisionTreeDataTest; +import org.apache.ignite.ml.tree.impurity.gini.GiniImpurityMeasureCalculatorTest; +import org.apache.ignite.ml.tree.impurity.gini.GiniImpurityMeasureTest; +import org.apache.ignite.ml.tree.impurity.mse.MSEImpurityMeasureCalculatorTest; +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.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({ + DecisionTreeClassificationTrainerTest.class, + DecisionTreeRegressionTrainerTest.class, + DecisionTreeClassificationTrainerIntegrationTest.class, + DecisionTreeRegressionTrainerIntegrationTest.class, + DecisionTreeDataTest.class, + GiniImpurityMeasureCalculatorTest.class, + GiniImpurityMeasureTest.class, + MSEImpurityMeasureCalculatorTest.class, + MSEImpurityMeasureTest.class, + StepFunctionTest.class, + SimpleStepFunctionCompressorTest.class +}) +public class DecisionTreeTestSuite { +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/data/DecisionTreeDataTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/data/DecisionTreeDataTest.java new file mode 100644 index 0000000000000..0c89d4e11d05d --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/data/DecisionTreeDataTest.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.data; + +import org.junit.Test; + +import static org.junit.Assert.assertArrayEquals; + +/** + * Tests for {@link DecisionTreeData}. + */ +public class DecisionTreeDataTest { + /** */ + @Test + public void testFilter() { + double[][] features = new double[][]{{0}, {1}, {2}, {3}, {4}, {5}}; + double[] labels = new double[]{0, 1, 2, 3, 4, 5}; + + DecisionTreeData data = new DecisionTreeData(features, labels); + DecisionTreeData filteredData = data.filter(obj -> obj[0] > 2); + + assertArrayEquals(new double[][]{{3}, {4}, {5}}, filteredData.getFeatures()); + assertArrayEquals(new double[]{3, 4, 5}, filteredData.getLabels(), 1e-10); + } + + /** */ + @Test + public void testSort() { + double[][] features = new double[][]{{4, 1}, {3, 3}, {2, 0}, {1, 4}, {0, 2}}; + double[] labels = new double[]{0, 1, 2, 3, 4}; + + DecisionTreeData data = new DecisionTreeData(features, labels); + + data.sort(0); + + assertArrayEquals(new double[][]{{0, 2}, {1, 4}, {2, 0}, {3, 3}, {4, 1}}, features); + assertArrayEquals(new double[]{4, 3, 2, 1, 0}, labels, 1e-10); + + data.sort(1); + + assertArrayEquals(new double[][]{{2, 0}, {4, 1}, {0, 2}, {3, 3}, {1, 4}}, features); + assertArrayEquals(new double[]{2, 0, 4, 1, 3}, labels, 1e-10); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/gini/GiniImpurityMeasureCalculatorTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/gini/GiniImpurityMeasureCalculatorTest.java new file mode 100644 index 0000000000000..afd81e882491d --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/gini/GiniImpurityMeasureCalculatorTest.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.ml.tree.impurity.gini; + +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.ml.tree.data.DecisionTreeData; +import org.apache.ignite.ml.tree.impurity.util.StepFunction; +import org.junit.Test; + +import static junit.framework.TestCase.assertEquals; +import static org.junit.Assert.assertArrayEquals; + +/** + * Tests for {@link GiniImpurityMeasureCalculator}. + */ +public class GiniImpurityMeasureCalculatorTest { + /** */ + @Test + public void testCalculate() { + double[][] data = new double[][]{{0, 1}, {1, 0}, {2, 2}, {3, 3}}; + double[] labels = new double[]{0, 1, 1, 1}; + + Map encoder = new HashMap<>(); + encoder.put(0.0, 0); + encoder.put(1.0, 1); + GiniImpurityMeasureCalculator calculator = new GiniImpurityMeasureCalculator(encoder); + + StepFunction[] impurity = calculator.calculate(new DecisionTreeData(data, labels)); + + assertEquals(2, impurity.length); + + // Check Gini calculated for the first column. + assertArrayEquals(new double[]{Double.NEGATIVE_INFINITY, 0, 1, 2, 3}, impurity[0].getX(), 1e-10); + assertEquals(-2.500, impurity[0].getY()[0].impurity(), 1e-3); + assertEquals(-4.000, impurity[0].getY()[1].impurity(),1e-3); + assertEquals(-3.000, impurity[0].getY()[2].impurity(),1e-3); + assertEquals(-2.666, impurity[0].getY()[3].impurity(),1e-3); + assertEquals(-2.500, impurity[0].getY()[4].impurity(),1e-3); + + // Check Gini calculated for the second column. + assertArrayEquals(new double[]{Double.NEGATIVE_INFINITY, 0, 1, 2, 3}, impurity[1].getX(), 1e-10); + assertEquals(-2.500, impurity[1].getY()[0].impurity(),1e-3); + assertEquals(-2.666, impurity[1].getY()[1].impurity(),1e-3); + assertEquals(-3.000, impurity[1].getY()[2].impurity(),1e-3); + assertEquals(-2.666, impurity[1].getY()[3].impurity(),1e-3); + assertEquals(-2.500, impurity[1].getY()[4].impurity(),1e-3); + } + + /** */ + @Test + public void testCalculateWithRepeatedData() { + double[][] data = new double[][]{{0}, {1}, {2}, {2}, {3}}; + double[] labels = new double[]{0, 1, 1, 1, 1}; + + Map encoder = new HashMap<>(); + encoder.put(0.0, 0); + encoder.put(1.0, 1); + GiniImpurityMeasureCalculator calculator = new GiniImpurityMeasureCalculator(encoder); + + StepFunction[] impurity = calculator.calculate(new DecisionTreeData(data, labels)); + + assertEquals(1, impurity.length); + + // Check Gini calculated for the first column. + assertArrayEquals(new double[]{Double.NEGATIVE_INFINITY, 0, 1, 2, 3}, impurity[0].getX(), 1e-10); + assertEquals(-3.400, impurity[0].getY()[0].impurity(), 1e-3); + assertEquals(-5.000, impurity[0].getY()[1].impurity(),1e-3); + assertEquals(-4.000, impurity[0].getY()[2].impurity(),1e-3); + assertEquals(-3.500, impurity[0].getY()[3].impurity(),1e-3); + assertEquals(-3.400, impurity[0].getY()[4].impurity(),1e-3); + } + + /** */ + @Test + public void testGetLabelCode() { + Map encoder = new HashMap<>(); + encoder.put(0.0, 0); + encoder.put(1.0, 1); + encoder.put(2.0, 2); + + GiniImpurityMeasureCalculator calculator = new GiniImpurityMeasureCalculator(encoder); + + assertEquals(0, calculator.getLabelCode(0.0)); + assertEquals(1, calculator.getLabelCode(1.0)); + assertEquals(2, calculator.getLabelCode(2.0)); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/gini/GiniImpurityMeasureTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/gini/GiniImpurityMeasureTest.java new file mode 100644 index 0000000000000..35c456aa51adf --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/gini/GiniImpurityMeasureTest.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.ml.tree.impurity.gini; + +import java.util.Random; +import org.junit.Test; + +import static junit.framework.TestCase.assertEquals; + +/** + * Tests for {@link GiniImpurityMeasure}. + */ +public class GiniImpurityMeasureTest { + /** */ + @Test + public void testImpurityOnEmptyData() { + long[] left = new long[]{0, 0, 0}; + long[] right = new long[]{0, 0, 0}; + + GiniImpurityMeasure impurity = new GiniImpurityMeasure(left, right); + + assertEquals(0.0, impurity.impurity(), 1e-10); + } + + /** */ + @Test + public void testImpurityLeftPart() { + long[] left = new long[]{3, 0, 0}; + long[] right = new long[]{0, 0, 0}; + + GiniImpurityMeasure impurity = new GiniImpurityMeasure(left, right); + + assertEquals(-3, impurity.impurity(), 1e-10); + } + + /** */ + @Test + public void testImpurityRightPart() { + long[] left = new long[]{0, 0, 0}; + long[] right = new long[]{3, 0, 0}; + + GiniImpurityMeasure impurity = new GiniImpurityMeasure(left, right); + + assertEquals(-3, impurity.impurity(), 1e-10); + } + + /** */ + @Test + public void testImpurityLeftAndRightPart() { + long[] left = new long[]{3, 0, 0}; + long[] right = new long[]{0, 3, 0}; + + GiniImpurityMeasure impurity = new GiniImpurityMeasure(left, right); + + assertEquals(-6, impurity.impurity(), 1e-10); + } + + /** */ + @Test + public void testAdd() { + Random rnd = new Random(0); + + GiniImpurityMeasure a = new GiniImpurityMeasure( + new long[]{randCnt(rnd), randCnt(rnd), randCnt(rnd)}, + new long[]{randCnt(rnd), randCnt(rnd), randCnt(rnd)} + ); + + + GiniImpurityMeasure b = new GiniImpurityMeasure( + new long[]{randCnt(rnd), randCnt(rnd), randCnt(rnd)}, + new long[]{randCnt(rnd), randCnt(rnd), randCnt(rnd)} + ); + + GiniImpurityMeasure c = a.add(b); + + assertEquals(a.getLeft()[0] + b.getLeft()[0], c.getLeft()[0]); + assertEquals(a.getLeft()[1] + b.getLeft()[1], c.getLeft()[1]); + assertEquals(a.getLeft()[2] + b.getLeft()[2], c.getLeft()[2]); + + assertEquals(a.getRight()[0] + b.getRight()[0], c.getRight()[0]); + assertEquals(a.getRight()[1] + b.getRight()[1], c.getRight()[1]); + assertEquals(a.getRight()[2] + b.getRight()[2], c.getRight()[2]); + } + + /** */ + @Test + public void testSubtract() { + Random rnd = new Random(0); + + GiniImpurityMeasure a = new GiniImpurityMeasure( + new long[]{randCnt(rnd), randCnt(rnd), randCnt(rnd)}, + new long[]{randCnt(rnd), randCnt(rnd), randCnt(rnd)} + ); + + + GiniImpurityMeasure b = new GiniImpurityMeasure( + new long[]{randCnt(rnd), randCnt(rnd), randCnt(rnd)}, + new long[]{randCnt(rnd), randCnt(rnd), randCnt(rnd)} + ); + + GiniImpurityMeasure c = a.subtract(b); + + assertEquals(a.getLeft()[0] - b.getLeft()[0], c.getLeft()[0]); + assertEquals(a.getLeft()[1] - b.getLeft()[1], c.getLeft()[1]); + assertEquals(a.getLeft()[2] - b.getLeft()[2], c.getLeft()[2]); + + assertEquals(a.getRight()[0] - b.getRight()[0], c.getRight()[0]); + assertEquals(a.getRight()[1] - b.getRight()[1], c.getRight()[1]); + assertEquals(a.getRight()[2] - b.getRight()[2], c.getRight()[2]); + } + + /** Generates random count. */ + private long randCnt(Random rnd) { + return Math.abs(rnd.nextInt()); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/mse/MSEImpurityMeasureCalculatorTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/mse/MSEImpurityMeasureCalculatorTest.java new file mode 100644 index 0000000000000..510c18fe3750e --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/mse/MSEImpurityMeasureCalculatorTest.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.impurity.mse; + +import org.apache.ignite.ml.tree.data.DecisionTreeData; +import org.apache.ignite.ml.tree.impurity.util.StepFunction; +import org.junit.Test; + +import static junit.framework.TestCase.assertEquals; +import static org.junit.Assert.assertArrayEquals; + +/** + * Tests for {@link MSEImpurityMeasureCalculator}. + */ +public class MSEImpurityMeasureCalculatorTest { + /** */ + @Test + public void testCalculate() { + double[][] data = new double[][]{{0, 2}, {1, 1}, {2, 0}, {3, 3}}; + double[] labels = new double[]{1, 2, 2, 1}; + + MSEImpurityMeasureCalculator calculator = new MSEImpurityMeasureCalculator(); + + StepFunction[] impurity = calculator.calculate(new DecisionTreeData(data, labels)); + + assertEquals(2, impurity.length); + + // Test MSE calculated for the first column. + assertArrayEquals(new double[]{Double.NEGATIVE_INFINITY, 0, 1, 2, 3}, impurity[0].getX(), 1e-10); + assertEquals(1.000, impurity[0].getY()[0].impurity(), 1e-3); + assertEquals(0.666, impurity[0].getY()[1].impurity(),1e-3); + assertEquals(1.000, impurity[0].getY()[2].impurity(),1e-3); + assertEquals(0.666, impurity[0].getY()[3].impurity(),1e-3); + assertEquals(1.000, impurity[0].getY()[4].impurity(),1e-3); + + // Test MSE calculated for the second column. + assertArrayEquals(new double[]{Double.NEGATIVE_INFINITY, 0, 1, 2, 3}, impurity[1].getX(), 1e-10); + assertEquals(1.000, impurity[1].getY()[0].impurity(),1e-3); + assertEquals(0.666, impurity[1].getY()[1].impurity(),1e-3); + assertEquals(0.000, impurity[1].getY()[2].impurity(),1e-3); + assertEquals(0.666, impurity[1].getY()[3].impurity(),1e-3); + assertEquals(1.000, impurity[1].getY()[4].impurity(),1e-3); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/mse/MSEImpurityMeasureTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/mse/MSEImpurityMeasureTest.java new file mode 100644 index 0000000000000..3d11d9d7d95f5 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/mse/MSEImpurityMeasureTest.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.ml.tree.impurity.mse; + +import java.util.Random; +import org.junit.Test; + +import static junit.framework.TestCase.assertEquals; + +/** + * Tests for {@link MSEImpurityMeasure}. + */ +public class MSEImpurityMeasureTest { + /** */ + @Test + public void testImpurityOnEmptyData() { + MSEImpurityMeasure impurity = new MSEImpurityMeasure(0, 0, 0, 0, 0, 0); + + assertEquals(0.0, impurity.impurity(), 1e-10); + } + + /** */ + @Test + public void testImpurityLeftPart() { + // Test on left part [1, 2, 2, 1, 1, 1]. + MSEImpurityMeasure impurity = new MSEImpurityMeasure(8, 12, 6, 0, 0, 0); + + assertEquals(1.333, impurity.impurity(), 1e-3); + } + + /** */ + @Test + public void testImpurityRightPart() { + // Test on right part [1, 2, 2, 1, 1, 1]. + MSEImpurityMeasure impurity = new MSEImpurityMeasure(0, 0, 0, 8, 12, 6); + + assertEquals(1.333, impurity.impurity(), 1e-3); + } + + /** */ + @Test + public void testImpurityLeftAndRightPart() { + // Test on left part [1, 2, 2] and right part [1, 1, 1]. + MSEImpurityMeasure impurity = new MSEImpurityMeasure(5, 9, 3, 3, 3, 3); + + assertEquals(0.666, impurity.impurity(), 1e-3); + } + + /** */ + @Test + public void testAdd() { + Random rnd = new Random(0); + + MSEImpurityMeasure a = new MSEImpurityMeasure( + rnd.nextDouble(), rnd.nextDouble(), rnd.nextInt(), rnd.nextDouble(), rnd.nextDouble(), rnd.nextInt() + ); + + MSEImpurityMeasure b = new MSEImpurityMeasure( + rnd.nextDouble(), rnd.nextDouble(), rnd.nextInt(), rnd.nextDouble(), rnd.nextDouble(), rnd.nextInt() + ); + + MSEImpurityMeasure c = a.add(b); + + assertEquals(a.getLeftY() + b.getLeftY(), c.getLeftY(), 1e-10); + assertEquals(a.getLeftY2() + b.getLeftY2(), c.getLeftY2(), 1e-10); + assertEquals(a.getLeftCnt() + b.getLeftCnt(), c.getLeftCnt()); + assertEquals(a.getRightY() + b.getRightY(), c.getRightY(), 1e-10); + assertEquals(a.getRightY2() + b.getRightY2(), c.getRightY2(), 1e-10); + assertEquals(a.getRightCnt() + b.getRightCnt(), c.getRightCnt()); + } + + /** */ + @Test + public void testSubtract() { + Random rnd = new Random(0); + + MSEImpurityMeasure a = new MSEImpurityMeasure( + rnd.nextDouble(), rnd.nextDouble(), rnd.nextInt(), rnd.nextDouble(), rnd.nextDouble(), rnd.nextInt() + ); + + MSEImpurityMeasure b = new MSEImpurityMeasure( + rnd.nextDouble(), rnd.nextDouble(), rnd.nextInt(), rnd.nextDouble(), rnd.nextDouble(), rnd.nextInt() + ); + + MSEImpurityMeasure c = a.subtract(b); + + assertEquals(a.getLeftY() - b.getLeftY(), c.getLeftY(), 1e-10); + assertEquals(a.getLeftY2() - b.getLeftY2(), c.getLeftY2(), 1e-10); + assertEquals(a.getLeftCnt() - b.getLeftCnt(), c.getLeftCnt()); + assertEquals(a.getRightY() - b.getRightY(), c.getRightY(), 1e-10); + assertEquals(a.getRightY2() - b.getRightY2(), c.getRightY2(), 1e-10); + assertEquals(a.getRightCnt() - b.getRightCnt(), c.getRightCnt()); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/util/SimpleStepFunctionCompressorTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/util/SimpleStepFunctionCompressorTest.java new file mode 100644 index 0000000000000..001404fe52126 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/util/SimpleStepFunctionCompressorTest.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.tree.impurity.util; + +import org.junit.Test; + +import static org.junit.Assert.assertArrayEquals; + +/** + * Tests for {@link SimpleStepFunctionCompressor}. + */ +public class SimpleStepFunctionCompressorTest { + /** */ + @Test + public void testCompressSmallFunction() { + StepFunction function = new StepFunction<>( + new double[]{1, 2, 3, 4}, + TestImpurityMeasure.asTestImpurityMeasures(1, 2, 3, 4) + ); + + SimpleStepFunctionCompressor compressor = new SimpleStepFunctionCompressor<>(5, 0, 0); + + StepFunction resFunction = compressor.compress(function); + + assertArrayEquals(new double[]{1, 2, 3, 4}, resFunction.getX(), 1e-10); + assertArrayEquals(TestImpurityMeasure.asTestImpurityMeasures(1, 2, 3, 4), resFunction.getY()); + } + + /** */ + @Test + public void testCompressIncreasingFunction() { + StepFunction function = new StepFunction<>( + new double[]{1, 2, 3, 4, 5}, + TestImpurityMeasure.asTestImpurityMeasures(1, 2, 3, 4, 5) + ); + + SimpleStepFunctionCompressor compressor = new SimpleStepFunctionCompressor<>(1, 0.4, 0); + + StepFunction resFunction = compressor.compress(function); + + assertArrayEquals(new double[]{1, 3, 5}, resFunction.getX(), 1e-10); + assertArrayEquals(TestImpurityMeasure.asTestImpurityMeasures(1, 3, 5), resFunction.getY()); + } + + /** */ + @Test + public void testCompressDecreasingFunction() { + StepFunction function = new StepFunction<>( + new double[]{1, 2, 3, 4, 5}, + TestImpurityMeasure.asTestImpurityMeasures(5, 4, 3, 2, 1) + ); + + SimpleStepFunctionCompressor compressor = new SimpleStepFunctionCompressor<>(1, 0, 0.4); + + StepFunction resFunction = compressor.compress(function); + + assertArrayEquals(new double[]{1, 3, 5}, resFunction.getX(), 1e-10); + assertArrayEquals(TestImpurityMeasure.asTestImpurityMeasures(5, 3, 1), resFunction.getY()); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/util/StepFunctionTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/util/StepFunctionTest.java new file mode 100644 index 0000000000000..2a0279cec0eef --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/util/StepFunctionTest.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.ml.tree.impurity.util; + +import org.junit.Test; + +import static org.junit.Assert.assertArrayEquals; + +/** + * Tests for {@link StepFunction}. + */ +public class StepFunctionTest { + /** */ + @Test + public void testAddIncreasingFunctions() { + StepFunction a = new StepFunction<>( + new double[]{1, 3, 5}, + TestImpurityMeasure.asTestImpurityMeasures(1, 2, 3) + ); + + StepFunction b = new StepFunction<>( + new double[]{0, 2, 4}, + TestImpurityMeasure.asTestImpurityMeasures(1, 2, 3) + ); + + StepFunction c = a.add(b); + + assertArrayEquals(new double[]{0, 1, 2, 3, 4, 5}, c.getX(), 1e-10); + assertArrayEquals( + TestImpurityMeasure.asTestImpurityMeasures(1, 2, 3, 4, 5, 6), + c.getY() + ); + } + + /** */ + @Test + public void testAddDecreasingFunctions() { + StepFunction a = new StepFunction<>( + new double[]{1, 3, 5}, + TestImpurityMeasure.asTestImpurityMeasures(3, 2, 1) + ); + + StepFunction b = new StepFunction<>( + new double[]{0, 2, 4}, + TestImpurityMeasure.asTestImpurityMeasures(3, 2, 1) + ); + + StepFunction c = a.add(b); + + assertArrayEquals(new double[]{0, 1, 2, 3, 4, 5}, c.getX(), 1e-10); + assertArrayEquals( + TestImpurityMeasure.asTestImpurityMeasures(3, 6, 5, 4, 3, 2), + c.getY() + ); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/util/TestImpurityMeasure.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/util/TestImpurityMeasure.java new file mode 100644 index 0000000000000..c0d1911a586e8 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/impurity/util/TestImpurityMeasure.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.tree.impurity.util; + +import java.util.Objects; +import org.apache.ignite.ml.tree.impurity.ImpurityMeasure; + +/** + * Utils class used as impurity measure in tests. + */ +class TestImpurityMeasure implements ImpurityMeasure { + /** */ + private static final long serialVersionUID = 2414020770162797847L; + + /** Impurity. */ + private final double impurity; + + /** + * Constructs a new instance of test impurity measure. + * + * @param impurity Impurity. + */ + private TestImpurityMeasure(double impurity) { + this.impurity = impurity; + } + + /** + * Convert doubles to array of test impurity measures. + * + * @param impurity Impurity as array of doubles. + * @return Test impurity measure objects as array. + */ + static TestImpurityMeasure[] asTestImpurityMeasures(double... impurity) { + TestImpurityMeasure[] res = new TestImpurityMeasure[impurity.length]; + + for (int i = 0; i < impurity.length; i++) + res[i] = new TestImpurityMeasure(impurity[i]); + + return res; + } + + /** {@inheritDoc} */ + @Override public double impurity() { + return impurity; + } + + /** {@inheritDoc} */ + @Override public TestImpurityMeasure add(TestImpurityMeasure measure) { + return new TestImpurityMeasure(impurity + measure.impurity); + } + + /** {@inheritDoc} */ + @Override public TestImpurityMeasure subtract(TestImpurityMeasure measure) { + return new TestImpurityMeasure(impurity - measure.impurity); + } + + /** */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + TestImpurityMeasure measure = (TestImpurityMeasure)o; + + return Double.compare(measure.impurity, impurity) == 0; + } + + /** */ + @Override public int hashCode() { + + return Objects.hash(impurity); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTIntegrationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTIntegrationTest.java new file mode 100644 index 0000000000000..b259ec9700de5 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTIntegrationTest.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.ml.tree.performance; + +import java.io.IOException; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; +import org.apache.ignite.ml.nn.performance.MnistMLPTestUtil; +import org.apache.ignite.ml.tree.DecisionTreeClassificationTrainer; +import org.apache.ignite.ml.tree.DecisionTreeNode; +import org.apache.ignite.ml.tree.impurity.util.SimpleStepFunctionCompressor; +import org.apache.ignite.ml.util.MnistUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests {@link DecisionTreeClassificationTrainer} on the MNIST dataset that require to start the whole Ignite + * infrastructure. For manual run. + */ +public class DecisionTreeMNISTIntegrationTest extends GridCommonAbstractTest { + /** 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()); + } + + /** Tests on the MNIST dataset. For manual run. */ + public void testMNIST() throws IOException { + CacheConfiguration trainingSetCacheCfg = new CacheConfiguration<>(); + trainingSetCacheCfg.setAffinity(new RendezvousAffinityFunction(false, 10)); + trainingSetCacheCfg.setName("MNIST_TRAINING_SET"); + + IgniteCache trainingSet = ignite.createCache(trainingSetCacheCfg); + + int i = 0; + for (MnistUtils.MnistLabeledImage e : MnistMLPTestUtil.loadTrainingSet(60_000)) + trainingSet.put(i++, e); + + DecisionTreeClassificationTrainer trainer = new DecisionTreeClassificationTrainer( + 8, + 0, + new SimpleStepFunctionCompressor<>()); + + DecisionTreeNode mdl = trainer.fit( + new CacheBasedDatasetBuilder<>(ignite, trainingSet), + (k, v) -> v.getPixels(), + (k, v) -> (double) v.getLabel() + ); + + int correctAnswers = 0; + int incorrectAnswers = 0; + + for (MnistUtils.MnistLabeledImage e : MnistMLPTestUtil.loadTestSet(10_000)) { + double res = mdl.apply(e.getPixels()); + + if (res == e.getLabel()) + correctAnswers++; + else + incorrectAnswers++; + } + + double accuracy = 1.0 * correctAnswers / (correctAnswers + incorrectAnswers); + + assertTrue(accuracy > 0.8); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTTest.java new file mode 100644 index 0000000000000..6dbd44c5919b3 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTTest.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.performance; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; +import org.apache.ignite.ml.nn.performance.MnistMLPTestUtil; +import org.apache.ignite.ml.tree.DecisionTreeClassificationTrainer; +import org.apache.ignite.ml.tree.DecisionTreeNode; +import org.apache.ignite.ml.tree.impurity.util.SimpleStepFunctionCompressor; +import org.apache.ignite.ml.util.MnistUtils; +import org.junit.Test; + +import static junit.framework.TestCase.assertTrue; + +/** + * Tests {@link DecisionTreeClassificationTrainer} on the MNIST dataset using locally stored data. For manual run. + */ +public class DecisionTreeMNISTTest { + /** Tests on the MNIST dataset. For manual run. */ + @Test + public void testMNIST() throws IOException { + Map trainingSet = new HashMap<>(); + + int i = 0; + for (MnistUtils.MnistLabeledImage e : MnistMLPTestUtil.loadTrainingSet(60_000)) + trainingSet.put(i++, e); + + + DecisionTreeClassificationTrainer trainer = new DecisionTreeClassificationTrainer( + 8, + 0, + new SimpleStepFunctionCompressor<>()); + + DecisionTreeNode mdl = trainer.fit( + new LocalDatasetBuilder<>(trainingSet, 10), + (k, v) -> v.getPixels(), + (k, v) -> (double) v.getLabel() + ); + + int correctAnswers = 0; + int incorrectAnswers = 0; + + for (MnistUtils.MnistLabeledImage e : MnistMLPTestUtil.loadTestSet(10_000)) { + double res = mdl.apply(e.getPixels()); + + if (res == e.getLabel()) + correctAnswers++; + else + incorrectAnswers++; + } + + double accuracy = 1.0 * correctAnswers / (correctAnswers + incorrectAnswers); + + assertTrue(accuracy > 0.8); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trees/BaseDecisionTreeTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/trees/BaseDecisionTreeTest.java deleted file mode 100644 index 65f0ae49b919f..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/trees/BaseDecisionTreeTest.java +++ /dev/null @@ -1,70 +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.trees; - -import java.util.Arrays; -import org.apache.ignite.Ignite; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; -import org.apache.ignite.ml.structures.LabeledVectorDouble; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; - -/** - * Base class for decision trees test. - */ -public class BaseDecisionTreeTest extends GridCommonAbstractTest { - /** Count of nodes. */ - private static final int NODE_COUNT = 4; - - /** Grid instance. */ - protected Ignite ignite; - - /** - * Default constructor. - */ - public BaseDecisionTreeTest() { - super(false); - } - - /** - * {@inheritDoc} - */ - @Override protected void beforeTest() throws Exception { - ignite = grid(NODE_COUNT); - } - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - for (int i = 1; i <= NODE_COUNT; i++) - startGrid(i); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() throws Exception { - stopAllGrids(); - } - - /** - * Convert double array to {@link LabeledVectorDouble} - * - * @param arr Array for conversion. - * @return LabeledVectorDouble. - */ - protected static LabeledVectorDouble asLabeledVector(double arr[]) { - return new LabeledVectorDouble<>(new DenseLocalOnHeapVector(Arrays.copyOf(arr, arr.length - 1)), arr[arr.length - 1]); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trees/ColumnDecisionTreeTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/trees/ColumnDecisionTreeTrainerTest.java deleted file mode 100644 index b090f43ace881..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/trees/ColumnDecisionTreeTrainerTest.java +++ /dev/null @@ -1,191 +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.trees; - -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.stream.Collectors; -import java.util.stream.DoubleStream; -import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.internal.util.typedef.X; -import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.ml.math.StorageConstants; -import org.apache.ignite.ml.math.Tracer; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; -import org.apache.ignite.ml.structures.LabeledVectorDouble; -import org.apache.ignite.ml.trees.models.DecisionTreeModel; -import org.apache.ignite.ml.trees.trainers.columnbased.ColumnDecisionTreeTrainer; -import org.apache.ignite.ml.trees.trainers.columnbased.ColumnDecisionTreeTrainerInput; -import org.apache.ignite.ml.trees.trainers.columnbased.MatrixColumnDecisionTreeTrainerInput; -import org.apache.ignite.ml.trees.trainers.columnbased.contsplitcalcs.ContinuousSplitCalculators; -import org.apache.ignite.ml.trees.trainers.columnbased.regcalcs.RegionCalculators; - -/** Tests behaviour of ColumnDecisionTreeTrainer. */ -public class ColumnDecisionTreeTrainerTest extends BaseDecisionTreeTest { - /** - * Test {@link ColumnDecisionTreeTrainerTest} for mixed (continuous and categorical) data with Gini impurity. - */ - public void testCacheMixedGini() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - int totalPts = 1 << 10; - int featCnt = 2; - - HashMap catsInfo = new HashMap<>(); - catsInfo.put(1, 3); - - Random rnd = new Random(12349L); - - SplitDataGenerator gen = new SplitDataGenerator<>( - featCnt, catsInfo, () -> new DenseLocalOnHeapVector(featCnt + 1), rnd). - split(0, 1, new int[] {0, 2}). - split(1, 0, -10.0); - - testByGen(totalPts, catsInfo, gen, ContinuousSplitCalculators.GINI.apply(ignite), RegionCalculators.GINI, RegionCalculators.MEAN, rnd); - } - - /** - * Test {@link ColumnDecisionTreeTrainerTest} for mixed (continuous and categorical) data with Variance impurity. - */ - public void testCacheMixed() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - int totalPts = 1 << 10; - int featCnt = 2; - - HashMap catsInfo = new HashMap<>(); - catsInfo.put(1, 3); - - Random rnd = new Random(12349L); - - SplitDataGenerator gen = new SplitDataGenerator<>( - featCnt, catsInfo, () -> new DenseLocalOnHeapVector(featCnt + 1), rnd). - split(0, 1, new int[] {0, 2}). - split(1, 0, -10.0); - - testByGen(totalPts, catsInfo, gen, ContinuousSplitCalculators.VARIANCE, RegionCalculators.VARIANCE, RegionCalculators.MEAN, rnd); - } - - /** - * Test {@link ColumnDecisionTreeTrainerTest} for continuous data with Variance impurity. - */ - public void testCacheCont() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - int totalPts = 1 << 10; - int featCnt = 12; - - HashMap catsInfo = new HashMap<>(); - - Random rnd = new Random(12349L); - - SplitDataGenerator gen = new SplitDataGenerator<>( - featCnt, catsInfo, () -> new DenseLocalOnHeapVector(featCnt + 1), rnd). - split(0, 0, -10.0). - split(1, 0, 0.0). - split(1, 1, 2.0). - split(3, 7, 50.0); - - testByGen(totalPts, catsInfo, gen, ContinuousSplitCalculators.VARIANCE, RegionCalculators.VARIANCE, RegionCalculators.MEAN, rnd); - } - - /** - * Test {@link ColumnDecisionTreeTrainerTest} for continuous data with Gini impurity. - */ - public void testCacheContGini() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - int totalPts = 1 << 10; - int featCnt = 12; - - HashMap catsInfo = new HashMap<>(); - - Random rnd = new Random(12349L); - - SplitDataGenerator gen = new SplitDataGenerator<>( - featCnt, catsInfo, () -> new DenseLocalOnHeapVector(featCnt + 1), rnd). - split(0, 0, -10.0). - split(1, 0, 0.0). - split(1, 1, 2.0). - split(3, 7, 50.0); - - testByGen(totalPts, catsInfo, gen, ContinuousSplitCalculators.GINI.apply(ignite), RegionCalculators.GINI, RegionCalculators.MEAN, rnd); - } - - /** - * Test {@link ColumnDecisionTreeTrainerTest} for categorical data with Variance impurity. - */ - public void testCacheCat() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - int totalPts = 1 << 10; - int featCnt = 12; - - HashMap catsInfo = new HashMap<>(); - catsInfo.put(5, 7); - - Random rnd = new Random(12349L); - - SplitDataGenerator gen = new SplitDataGenerator<>( - featCnt, catsInfo, () -> new DenseLocalOnHeapVector(featCnt + 1), rnd). - split(0, 5, new int[] {0, 2, 5}); - - testByGen(totalPts, catsInfo, gen, ContinuousSplitCalculators.VARIANCE, RegionCalculators.VARIANCE, RegionCalculators.MEAN, rnd); - } - - /** */ - private void testByGen(int totalPts, HashMap catsInfo, - SplitDataGenerator gen, - IgniteFunction> calc, - IgniteFunction> catImpCalc, - IgniteFunction regCalc, Random rnd) { - - List> lst = gen. - points(totalPts, (i, rn) -> i). - collect(Collectors.toList()); - - int featCnt = gen.featuresCnt(); - - Collections.shuffle(lst, rnd); - - SparseDistributedMatrix m = new SparseDistributedMatrix(totalPts, featCnt + 1, StorageConstants.COLUMN_STORAGE_MODE, StorageConstants.RANDOM_ACCESS_MODE); - - Map> byRegion = new HashMap<>(); - - int i = 0; - for (IgniteBiTuple bt : lst) { - byRegion.putIfAbsent(bt.get1(), new LinkedList<>()); - byRegion.get(bt.get1()).add(asLabeledVector(bt.get2().getStorage().data())); - m.setRow(i, bt.get2().getStorage().data()); - i++; - } - - ColumnDecisionTreeTrainer trainer = - new ColumnDecisionTreeTrainer<>(3, calc, catImpCalc, regCalc, ignite); - - DecisionTreeModel mdl = trainer.train(new MatrixColumnDecisionTreeTrainerInput(m, catsInfo)); - - byRegion.keySet().forEach(k -> { - LabeledVectorDouble sp = byRegion.get(k).get(0); - Tracer.showAscii(sp.features()); - X.println("Actual and predicted vectors [act=" + sp.label() + " " + ", pred=" + mdl.apply(sp.features()) + "]"); - assert mdl.apply(sp.features()) == sp.doubleLabel(); - }); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trees/DecisionTreesTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/trees/DecisionTreesTestSuite.java deleted file mode 100644 index 3343503d97b73..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/trees/DecisionTreesTestSuite.java +++ /dev/null @@ -1,33 +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.trees; - -import org.junit.runner.RunWith; -import org.junit.runners.Suite; - -/** - * Test suite for all tests located in org.apache.ignite.ml.trees package - */ -@RunWith(Suite.class) -@Suite.SuiteClasses({ - ColumnDecisionTreeTrainerTest.class, - GiniSplitCalculatorTest.class, - VarianceSplitCalculatorTest.class -}) -public class DecisionTreesTestSuite { -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trees/GiniSplitCalculatorTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/trees/GiniSplitCalculatorTest.java deleted file mode 100644 index c92b4f54f33a3..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/trees/GiniSplitCalculatorTest.java +++ /dev/null @@ -1,141 +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.trees; - -import java.util.stream.DoubleStream; -import org.apache.ignite.ml.trees.trainers.columnbased.contsplitcalcs.GiniSplitCalculator; -import org.apache.ignite.ml.trees.trainers.columnbased.vectors.SplitInfo; -import org.junit.Test; - -/** - * Test of {@link GiniSplitCalculator}. - */ -public class GiniSplitCalculatorTest { - /** Test calculation of region info consisting from one point. */ - @Test - public void testCalculateRegionInfoSimple() { - double labels[] = new double[] {0.0}; - - assert new GiniSplitCalculator(labels).calculateRegionInfo(DoubleStream.of(labels), 0).impurity() == 0.0; - } - - /** Test calculation of region info consisting from two distinct classes. */ - @Test - public void testCalculateRegionInfoTwoClasses() { - double labels[] = new double[] {0.0, 1.0}; - - assert new GiniSplitCalculator(labels).calculateRegionInfo(DoubleStream.of(labels), 0).impurity() == 0.5; - } - - /** Test calculation of region info consisting from three distinct classes. */ - @Test - public void testCalculateRegionInfoThreeClasses() { - double labels[] = new double[] {0.0, 1.0, 2.0}; - - assert Math.abs(new GiniSplitCalculator(labels).calculateRegionInfo(DoubleStream.of(labels), 0).impurity() - 2.0 / 3) < 1E-5; - } - - /** Test calculation of split of region consisting from one point. */ - @Test - public void testSplitSimple() { - double labels[] = new double[] {0.0}; - double values[] = new double[] {0.0}; - Integer[] samples = new Integer[] {0}; - - int cnts[] = new int[] {1}; - - GiniSplitCalculator.GiniData data = new GiniSplitCalculator.GiniData(0.0, 1, cnts, 1); - - assert new GiniSplitCalculator(labels).splitRegion(samples, values, labels, 0, data) == null; - } - - /** Test calculation of split of region consisting from two points. */ - @Test - public void testSplitTwoClassesTwoPoints() { - double labels[] = new double[] {0.0, 1.0}; - double values[] = new double[] {0.0, 1.0}; - Integer[] samples = new Integer[] {0, 1}; - - int cnts[] = new int[] {1, 1}; - - GiniSplitCalculator.GiniData data = new GiniSplitCalculator.GiniData(0.5, 2, cnts, 1.0 * 1.0 + 1.0 * 1.0); - - SplitInfo split = new GiniSplitCalculator(labels).splitRegion(samples, values, labels, 0, data); - - assert split.leftData().impurity() == 0; - assert split.leftData().counts()[0] == 1; - assert split.leftData().counts()[1] == 0; - assert split.leftData().getSize() == 1; - - assert split.rightData().impurity() == 0; - assert split.rightData().counts()[0] == 0; - assert split.rightData().counts()[1] == 1; - assert split.rightData().getSize() == 1; - } - - /** Test calculation of split of region consisting from four distinct values. */ - @Test - public void testSplitTwoClassesFourPoints() { - double labels[] = new double[] {0.0, 0.0, 1.0, 1.0}; - double values[] = new double[] {0.0, 1.0, 2.0, 3.0}; - - Integer[] samples = new Integer[] {0, 1, 2, 3}; - - int[] cnts = new int[] {2, 2}; - - GiniSplitCalculator.GiniData data = new GiniSplitCalculator.GiniData(0.5, 4, cnts, 2.0 * 2.0 + 2.0 * 2.0); - - SplitInfo split = new GiniSplitCalculator(labels).splitRegion(samples, values, labels, 0, data); - - assert split.leftData().impurity() == 0; - assert split.leftData().counts()[0] == 2; - assert split.leftData().counts()[1] == 0; - assert split.leftData().getSize() == 2; - - assert split.rightData().impurity() == 0; - assert split.rightData().counts()[0] == 0; - assert split.rightData().counts()[1] == 2; - assert split.rightData().getSize() == 2; - } - - /** Test calculation of split of region consisting from three distinct values. */ - @Test - public void testSplitThreePoints() { - double labels[] = new double[] {0.0, 1.0, 2.0}; - double values[] = new double[] {0.0, 1.0, 2.0}; - Integer[] samples = new Integer[] {0, 1, 2}; - - int[] cnts = new int[] {1, 1, 1}; - - GiniSplitCalculator.GiniData data = new GiniSplitCalculator.GiniData(2.0 / 3, 3, cnts, 1.0 * 1.0 + 1.0 * 1.0 + 1.0 * 1.0); - - SplitInfo split = new GiniSplitCalculator(labels).splitRegion(samples, values, labels, 0, data); - - assert split.leftData().impurity() == 0.0; - assert split.leftData().counts()[0] == 1; - assert split.leftData().counts()[1] == 0; - assert split.leftData().counts()[2] == 0; - assert split.leftData().getSize() == 1; - - assert split.rightData().impurity() == 0.5; - assert split.rightData().counts()[0] == 0; - assert split.rightData().counts()[1] == 1; - assert split.rightData().counts()[2] == 1; - assert split.rightData().getSize() == 2; - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trees/SplitDataGenerator.java b/modules/ml/src/test/java/org/apache/ignite/ml/trees/SplitDataGenerator.java deleted file mode 100644 index 279e6851c52b5..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/trees/SplitDataGenerator.java +++ /dev/null @@ -1,390 +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.trees; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.BitSet; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.function.BiFunction; -import java.util.function.Function; -import java.util.function.Supplier; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import java.util.stream.Stream; -import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.exceptions.MathIllegalArgumentException; -import org.apache.ignite.ml.util.Utils; - -/** - * Utility class for generating data which has binary tree split structure. - * - * @param - */ -public class SplitDataGenerator { - /** */ - private static final double DELTA = 100.0; - - /** Map of the form of (is categorical -> list of region indexes). */ - private final Map> di; - - /** List of regions. */ - private final List regs; - - /** Data of bounds of regions. */ - private final Map> boundsData; - - /** Random numbers generator. */ - private final Random rnd; - - /** Supplier of vectors. */ - private final Supplier supplier; - - /** Features count. */ - private final int featCnt; - - /** - * Create SplitDataGenerator. - * - * @param featCnt Features count. - * @param catFeaturesInfo Information about categorical features in form of map (feature index -> categories - * count). - * @param supplier Supplier of vectors. - * @param rnd Random numbers generator. - */ - public SplitDataGenerator(int featCnt, Map catFeaturesInfo, Supplier supplier, Random rnd) { - regs = new LinkedList<>(); - boundsData = new HashMap<>(); - this.rnd = rnd; - this.supplier = supplier; - this.featCnt = featCnt; - - // Divide indexes into indexes of categorical coordinates and indexes of continuous coordinates. - di = IntStream.range(0, featCnt). - boxed(). - collect(Collectors.partitioningBy(catFeaturesInfo::containsKey)); - - // Categorical coordinates info. - Map catCoords = new HashMap<>(); - di.get(true).forEach(i -> { - BitSet bs = new BitSet(); - bs.set(0, catFeaturesInfo.get(i)); - catCoords.put(i, new CatCoordInfo(bs)); - }); - - // Continuous coordinates info. - Map contCoords = new HashMap<>(); - di.get(false).forEach(i -> { - contCoords.put(i, new ContCoordInfo()); - boundsData.put(i, new IgniteBiTuple<>(-1.0, 1.0)); - }); - - Region firstReg = new Region(catCoords, contCoords, 0); - regs.add(firstReg); - } - - /** - * Categorical coordinate info. - */ - private static class CatCoordInfo implements Serializable { - /** - * Defines categories which are included in this region - */ - private final BitSet bs; - - /** - * Construct CatCoordInfo. - * - * @param bs Bitset. - */ - CatCoordInfo(BitSet bs) { - this.bs = bs; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return "CatCoordInfo [" + - "bs=" + bs + - ']'; - } - } - - /** - * Continuous coordinate info. - */ - private static class ContCoordInfo implements Serializable { - /** - * Left (min) bound of region. - */ - private double left; - - /** - * Right (max) bound of region. - */ - private double right; - - /** - * Construct ContCoordInfo. - */ - ContCoordInfo() { - left = Double.NEGATIVE_INFINITY; - right = Double.POSITIVE_INFINITY; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return "ContCoordInfo [" + - "left=" + left + - ", right=" + right + - ']'; - } - } - - /** - * Class representing information about region. - */ - private static class Region implements Serializable { - /** - * Information about categorical coordinates restrictions of this region in form of - * (coordinate index -> restriction) - */ - private final Map catCoords; - - /** - * Information about continuous coordinates restrictions of this region in form of - * (coordinate index -> restriction) - */ - private final Map contCoords; - - /** - * Region should contain {@code 1/2^twoPow * totalPoints} points. - */ - private int twoPow; - - /** - * Construct region by information about restrictions on coordinates (features) values. - * - * @param catCoords Restrictions on categorical coordinates. - * @param contCoords Restrictions on continuous coordinates - * @param twoPow Region should contain {@code 1/2^twoPow * totalPoints} points. - */ - Region(Map catCoords, Map contCoords, int twoPow) { - this.catCoords = catCoords; - this.contCoords = contCoords; - this.twoPow = twoPow; - } - - /** */ - int divideBy() { - return 1 << twoPow; - } - - /** */ - void incTwoPow() { - twoPow++; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return "Region [" + - "catCoords=" + catCoords + - ", contCoords=" + contCoords + - ", twoPow=" + twoPow + - ']'; - } - - /** - * Generate continuous coordinate for this region. - * - * @param coordIdx Coordinate index. - * @param boundsData Data with bounds - * @param rnd Random numbers generator. - * @return Categorical coordinate value. - */ - double generateContCoord(int coordIdx, Map> boundsData, - Random rnd) { - ContCoordInfo cci = contCoords.get(coordIdx); - double left = cci.left; - double right = cci.right; - - if (left == Double.NEGATIVE_INFINITY) - left = boundsData.get(coordIdx).get1() - DELTA; - - if (right == Double.POSITIVE_INFINITY) - right = boundsData.get(coordIdx).get2() + DELTA; - - double size = right - left; - - return left + rnd.nextDouble() * size; - } - - /** - * Generate categorical coordinate value for this region. - * - * @param coordIdx Coordinate index. - * @param rnd Random numbers generator. - * @return Categorical coordinate value. - */ - double generateCatCoord(int coordIdx, Random rnd) { - // Pick random bit. - BitSet bs = catCoords.get(coordIdx).bs; - int j = rnd.nextInt(bs.length()); - - int i = 0; - int bn = 0; - int bnp = 0; - - while ((bn = bs.nextSetBit(bn)) != -1 && i <= j) { - i++; - bnp = bn; - bn++; - } - - return bnp; - } - - /** - * Generate points for this region. - * - * @param ptsCnt Count of points to generate. - * @param val Label for all points in this region. - * @param boundsData Data about bounds of continuous coordinates. - * @param catCont Data about which categories can be in this region in the form (coordinate index -> list of - * categories indexes). - * @param s Vectors supplier. - * @param rnd Random numbers generator. - * @param Type of vectors. - * @return Stream of generated points for this region. - */ - Stream generatePoints(int ptsCnt, double val, - Map> boundsData, Map> catCont, - Supplier s, - Random rnd) { - return IntStream.range(0, ptsCnt / divideBy()).mapToObj(i -> { - V v = s.get(); - int coordsCnt = v.size(); - catCont.get(false).forEach(ci -> v.setX(ci, generateContCoord(ci, boundsData, rnd))); - catCont.get(true).forEach(ci -> v.setX(ci, generateCatCoord(ci, rnd))); - - v.setX(coordsCnt - 1, val); - return v; - }); - } - } - - /** - * Split region by continuous coordinate.using given threshold. - * - * @param regIdx Region index. - * @param coordIdx Coordinate index. - * @param threshold Threshold. - * @return {@code this}. - */ - public SplitDataGenerator split(int regIdx, int coordIdx, double threshold) { - Region regToSplit = regs.get(regIdx); - ContCoordInfo cci = regToSplit.contCoords.get(coordIdx); - - double left = cci.left; - double right = cci.right; - - if (threshold < left || threshold > right) - throw new MathIllegalArgumentException("Threshold is out of region bounds."); - - regToSplit.incTwoPow(); - - Region newReg = Utils.copy(regToSplit); - newReg.contCoords.get(coordIdx).left = threshold; - - regs.add(regIdx + 1, newReg); - cci.right = threshold; - - IgniteBiTuple bounds = boundsData.get(coordIdx); - double min = bounds.get1(); - double max = bounds.get2(); - boundsData.put(coordIdx, new IgniteBiTuple<>(Math.min(threshold, min), Math.max(max, threshold))); - - return this; - } - - /** - * Split region by categorical coordinate. - * - * @param regIdx Region index. - * @param coordIdx Coordinate index. - * @param cats Categories allowed for the left sub region. - * @return {@code this}. - */ - public SplitDataGenerator split(int regIdx, int coordIdx, int[] cats) { - BitSet subset = new BitSet(); - Arrays.stream(cats).forEach(subset::set); - Region regToSplit = regs.get(regIdx); - CatCoordInfo cci = regToSplit.catCoords.get(coordIdx); - - BitSet ssc = (BitSet)subset.clone(); - BitSet set = cci.bs; - ssc.and(set); - if (ssc.length() != subset.length()) - throw new MathIllegalArgumentException("Splitter set is not a subset of a parent subset."); - - ssc.xor(set); - set.and(subset); - - regToSplit.incTwoPow(); - Region newReg = Utils.copy(regToSplit); - newReg.catCoords.put(coordIdx, new CatCoordInfo(ssc)); - - regs.add(regIdx + 1, newReg); - - return this; - } - - /** - * Get stream of points generated by this generator. - * - * @param ptsCnt Points count. - */ - public Stream> points(int ptsCnt, BiFunction f) { - regs.forEach(System.out::println); - - return IntStream.range(0, regs.size()). - boxed(). - map(i -> regs.get(i).generatePoints(ptsCnt, f.apply((double)i, rnd), boundsData, di, supplier, rnd).map(v -> new IgniteBiTuple<>(i, v))).flatMap(Function.identity()); - } - - /** - * Count of regions. - * - * @return Count of regions. - */ - public int regsCount() { - return regs.size(); - } - - /** - * Get features count. - * - * @return Features count. - */ - public int featuresCnt() { - return featCnt; - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trees/VarianceSplitCalculatorTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/trees/VarianceSplitCalculatorTest.java deleted file mode 100644 index d67cbc6005e2f..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/trees/VarianceSplitCalculatorTest.java +++ /dev/null @@ -1,84 +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.trees; - -import java.util.stream.DoubleStream; -import org.apache.ignite.ml.trees.trainers.columnbased.contsplitcalcs.VarianceSplitCalculator; -import org.apache.ignite.ml.trees.trainers.columnbased.vectors.SplitInfo; -import org.junit.Test; - -/** - * Test for {@link VarianceSplitCalculator}. - */ -public class VarianceSplitCalculatorTest { - /** Test calculation of region info consisting from one point. */ - @Test - public void testCalculateRegionInfoSimple() { - double labels[] = new double[] {0.0}; - - assert new VarianceSplitCalculator().calculateRegionInfo(DoubleStream.of(labels), 1).impurity() == 0.0; - } - - /** Test calculation of region info consisting from two classes. */ - @Test - public void testCalculateRegionInfoTwoClasses() { - double labels[] = new double[] {0.0, 1.0}; - - assert new VarianceSplitCalculator().calculateRegionInfo(DoubleStream.of(labels), 2).impurity() == 0.25; - } - - /** Test calculation of region info consisting from three classes. */ - @Test - public void testCalculateRegionInfoThreeClasses() { - double labels[] = new double[] {1.0, 2.0, 3.0}; - - assert Math.abs(new VarianceSplitCalculator().calculateRegionInfo(DoubleStream.of(labels), 3).impurity() - 2.0 / 3) < 1E-10; - } - - /** Test calculation of split of region consisting from one point. */ - @Test - public void testSplitSimple() { - double labels[] = new double[] {0.0}; - double values[] = new double[] {0.0}; - Integer[] samples = new Integer[] {0}; - - VarianceSplitCalculator.VarianceData data = new VarianceSplitCalculator.VarianceData(0.0, 1, 0.0); - - assert new VarianceSplitCalculator().splitRegion(samples, values, labels, 0, data) == null; - } - - /** Test calculation of split of region consisting from two classes. */ - @Test - public void testSplitTwoClassesTwoPoints() { - double labels[] = new double[] {0.0, 1.0}; - double values[] = new double[] {0.0, 1.0}; - Integer[] samples = new Integer[] {0, 1}; - - VarianceSplitCalculator.VarianceData data = new VarianceSplitCalculator.VarianceData(0.25, 2, 0.5); - - SplitInfo split = new VarianceSplitCalculator().splitRegion(samples, values, labels, 0, data); - - assert split.leftData().impurity() == 0; - assert split.leftData().mean() == 0; - assert split.leftData().getSize() == 1; - - assert split.rightData().impurity() == 0; - assert split.rightData().mean() == 1; - assert split.rightData().getSize() == 1; - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trees/performance/ColumnDecisionTreeTrainerBenchmark.java b/modules/ml/src/test/java/org/apache/ignite/ml/trees/performance/ColumnDecisionTreeTrainerBenchmark.java deleted file mode 100644 index 21fd692366a43..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/trees/performance/ColumnDecisionTreeTrainerBenchmark.java +++ /dev/null @@ -1,456 +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.trees.performance; - -import it.unimi.dsi.fastutil.ints.Int2DoubleOpenHashMap; -import java.io.IOException; -import java.io.InputStream; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Random; -import java.util.UUID; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.DoubleStream; -import java.util.stream.IntStream; -import java.util.stream.Stream; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteDataStreamer; -import org.apache.ignite.Ignition; -import org.apache.ignite.cache.CacheAtomicityMode; -import org.apache.ignite.cache.CacheMode; -import org.apache.ignite.cache.CacheWriteSynchronizationMode; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.processors.cache.GridCacheProcessor; -import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.internal.util.typedef.X; -import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.ml.Model; -import org.apache.ignite.ml.estimators.Estimators; -import org.apache.ignite.ml.math.StorageConstants; -import org.apache.ignite.ml.math.Tracer; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.distributed.keys.impl.SparseMatrixKey; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.functions.IgniteTriFunction; -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.ml.math.impls.storage.matrix.SparseDistributedMatrixStorage; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; -import org.apache.ignite.ml.structures.LabeledVectorDouble; -import org.apache.ignite.ml.trees.BaseDecisionTreeTest; -import org.apache.ignite.ml.trees.SplitDataGenerator; -import org.apache.ignite.ml.trees.models.DecisionTreeModel; -import org.apache.ignite.ml.trees.trainers.columnbased.BiIndex; -import org.apache.ignite.ml.trees.trainers.columnbased.BiIndexedCacheColumnDecisionTreeTrainerInput; -import org.apache.ignite.ml.trees.trainers.columnbased.ColumnDecisionTreeTrainer; -import org.apache.ignite.ml.trees.trainers.columnbased.MatrixColumnDecisionTreeTrainerInput; -import org.apache.ignite.ml.trees.trainers.columnbased.caches.ContextCache; -import org.apache.ignite.ml.trees.trainers.columnbased.caches.FeaturesCache; -import org.apache.ignite.ml.trees.trainers.columnbased.caches.ProjectionsCache; -import org.apache.ignite.ml.trees.trainers.columnbased.caches.SplitCache; -import org.apache.ignite.ml.trees.trainers.columnbased.contsplitcalcs.ContinuousSplitCalculators; -import org.apache.ignite.ml.trees.trainers.columnbased.contsplitcalcs.GiniSplitCalculator; -import org.apache.ignite.ml.trees.trainers.columnbased.contsplitcalcs.VarianceSplitCalculator; -import org.apache.ignite.ml.trees.trainers.columnbased.regcalcs.RegionCalculators; -import org.apache.ignite.ml.util.MnistUtils; -import org.apache.ignite.stream.StreamTransformer; -import org.apache.ignite.testframework.junits.IgniteTestResources; -import org.apache.log4j.Level; -import org.junit.Assert; - -/** - * Various benchmarks for hand runs. - */ -public class ColumnDecisionTreeTrainerBenchmark extends BaseDecisionTreeTest { - /** Name of the property specifying path to training set images. */ - private static final String PROP_TRAINING_IMAGES = "mnist.training.images"; - - /** Name of property specifying path to training set labels. */ - private static final String PROP_TRAINING_LABELS = "mnist.training.labels"; - - /** Name of property specifying path to test set images. */ - private static final String PROP_TEST_IMAGES = "mnist.test.images"; - - /** Name of property specifying path to test set labels. */ - private static final String PROP_TEST_LABELS = "mnist.test.labels"; - - /** Function to approximate. */ - private static final Function f1 = v -> v.get(0) * v.get(0) + 2 * Math.sin(v.get(1)) + v.get(2); - - /** {@inheritDoc} */ - @Override protected long getTestTimeout() { - return 6000000; - } - - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName, - IgniteTestResources rsrcs) throws Exception { - IgniteConfiguration configuration = super.getConfiguration(igniteInstanceName, rsrcs); - // We do not need any extra event types. - configuration.setIncludeEventTypes(); - configuration.setPeerClassLoadingEnabled(false); - - resetLog4j(Level.INFO, false, GridCacheProcessor.class.getPackage().getName()); - - return configuration; - } - - /** - * This test is for manual run only. - * To run this test rename this method so it starts from 'test'. - */ - public void tstCacheMixed() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - int ptsPerReg = 150; - int featCnt = 10; - - HashMap catsInfo = new HashMap<>(); - catsInfo.put(1, 3); - - Random rnd = new Random(12349L); - - SplitDataGenerator gen = new SplitDataGenerator<>( - featCnt, catsInfo, () -> new DenseLocalOnHeapVector(featCnt + 1), rnd). - split(0, 1, new int[] {0, 2}). - split(1, 0, -10.0). - split(0, 0, 0.0); - - testByGenStreamerLoad(ptsPerReg, catsInfo, gen, rnd); - } - - /** - * Run decision tree classifier on MNIST using bi-indexed cache as a storage for dataset. - * To run this test rename this method so it starts from 'test'. - * - * @throws IOException In case of loading MNIST dataset errors. - */ - public void tstMNISTBiIndexedCache() throws IOException { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - - int ptsCnt = 40_000; - int featCnt = 28 * 28; - - Properties props = loadMNISTProperties(); - - Stream trainingMnistStream = MnistUtils.mnistAsStream(props.getProperty(PROP_TRAINING_IMAGES), props.getProperty(PROP_TRAINING_LABELS), new Random(123L), ptsCnt); - Stream testMnistStream = MnistUtils.mnistAsStream(props.getProperty(PROP_TEST_IMAGES), props.getProperty(PROP_TEST_LABELS), new Random(123L), 10_000); - - IgniteCache cache = createBiIndexedCache(); - - loadVectorsIntoBiIndexedCache(cache.getName(), trainingMnistStream.iterator(), featCnt + 1); - - ColumnDecisionTreeTrainer trainer = - new ColumnDecisionTreeTrainer<>(10, ContinuousSplitCalculators.GINI.apply(ignite), RegionCalculators.GINI, RegionCalculators.MOST_COMMON, ignite); - - X.println("Training started."); - long before = System.currentTimeMillis(); - DecisionTreeModel mdl = trainer.train(new BiIndexedCacheColumnDecisionTreeTrainerInput(cache, new HashMap<>(), ptsCnt, featCnt)); - X.println("Training finished in " + (System.currentTimeMillis() - before)); - - IgniteTriFunction, Stream>, Function, Double> mse = Estimators.errorsPercentage(); - Double accuracy = mse.apply(mdl, testMnistStream.map(v -> new IgniteBiTuple<>(v.viewPart(0, featCnt), v.getX(featCnt))), Function.identity()); - X.println("Errors percentage: " + accuracy); - - Assert.assertEquals(0, SplitCache.getOrCreate(ignite).size()); - Assert.assertEquals(0, FeaturesCache.getOrCreate(ignite).size()); - Assert.assertEquals(0, ContextCache.getOrCreate(ignite).size()); - Assert.assertEquals(0, ProjectionsCache.getOrCreate(ignite).size()); - } - - /** - * Run decision tree classifier on MNIST using sparse distributed matrix as a storage for dataset. - * To run this test rename this method so it starts from 'test'. - * - * @throws IOException In case of loading MNIST dataset errors. - */ - public void tstMNISTSparseDistributedMatrix() throws IOException { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - - int ptsCnt = 30_000; - int featCnt = 28 * 28; - - Properties props = loadMNISTProperties(); - - Stream trainingMnistStream = MnistUtils.mnistAsStream(props.getProperty(PROP_TRAINING_IMAGES), props.getProperty(PROP_TRAINING_LABELS), new Random(123L), ptsCnt); - Stream testMnistStream = MnistUtils.mnistAsStream(props.getProperty(PROP_TEST_IMAGES), props.getProperty(PROP_TEST_LABELS), new Random(123L), 10_000); - - SparseDistributedMatrix m = new SparseDistributedMatrix(ptsCnt, featCnt + 1, StorageConstants.COLUMN_STORAGE_MODE, StorageConstants.RANDOM_ACCESS_MODE); - - SparseDistributedMatrixStorage sto = (SparseDistributedMatrixStorage)m.getStorage(); - - loadVectorsIntoSparseDistributedMatrixCache(sto.cache().getName(), sto.getUUID(), trainingMnistStream.iterator(), featCnt + 1); - - ColumnDecisionTreeTrainer trainer = - new ColumnDecisionTreeTrainer<>(10, ContinuousSplitCalculators.GINI.apply(ignite), RegionCalculators.GINI, RegionCalculators.MOST_COMMON, ignite); - - X.println("Training started"); - long before = System.currentTimeMillis(); - DecisionTreeModel mdl = trainer.train(new MatrixColumnDecisionTreeTrainerInput(m, new HashMap<>())); - X.println("Training finished in " + (System.currentTimeMillis() - before)); - - IgniteTriFunction, Stream>, Function, Double> mse = Estimators.errorsPercentage(); - Double accuracy = mse.apply(mdl, testMnistStream.map(v -> new IgniteBiTuple<>(v.viewPart(0, featCnt), v.getX(featCnt))), Function.identity()); - X.println("Errors percentage: " + accuracy); - - Assert.assertEquals(0, SplitCache.getOrCreate(ignite).size()); - Assert.assertEquals(0, FeaturesCache.getOrCreate(ignite).size()); - Assert.assertEquals(0, ContextCache.getOrCreate(ignite).size()); - Assert.assertEquals(0, ProjectionsCache.getOrCreate(ignite).size()); - } - - /** Load properties for MNIST tests. */ - private static Properties loadMNISTProperties() throws IOException { - Properties res = new Properties(); - - InputStream is = ColumnDecisionTreeTrainerBenchmark.class.getClassLoader().getResourceAsStream("manualrun/trees/columntrees.manualrun.properties"); - - res.load(is); - - return res; - } - - /** */ - private void testByGenStreamerLoad(int ptsPerReg, HashMap catsInfo, - SplitDataGenerator gen, Random rnd) { - - List> lst = gen. - points(ptsPerReg, (i, rn) -> i). - collect(Collectors.toList()); - - int featCnt = gen.featuresCnt(); - - Collections.shuffle(lst, rnd); - - int numRegs = gen.regsCount(); - - SparseDistributedMatrix m = new SparseDistributedMatrix(numRegs * ptsPerReg, featCnt + 1, StorageConstants.COLUMN_STORAGE_MODE, StorageConstants.RANDOM_ACCESS_MODE); - - IgniteFunction regCalc = s -> s.average().orElse(0.0); - - Map> byRegion = new HashMap<>(); - - SparseDistributedMatrixStorage sto = (SparseDistributedMatrixStorage)m.getStorage(); - long before = System.currentTimeMillis(); - X.println("Batch loading started..."); - loadVectorsIntoSparseDistributedMatrixCache(sto.cache().getName(), sto.getUUID(), gen. - points(ptsPerReg, (i, rn) -> i).map(IgniteBiTuple::get2).iterator(), featCnt + 1); - X.println("Batch loading took " + (System.currentTimeMillis() - before) + " ms."); - - for (IgniteBiTuple bt : lst) { - byRegion.putIfAbsent(bt.get1(), new LinkedList<>()); - byRegion.get(bt.get1()).add(asLabeledVector(bt.get2().getStorage().data())); - } - - ColumnDecisionTreeTrainer trainer = - new ColumnDecisionTreeTrainer<>(2, ContinuousSplitCalculators.VARIANCE, RegionCalculators.VARIANCE, regCalc, ignite); - - before = System.currentTimeMillis(); - DecisionTreeModel mdl = trainer.train(new MatrixColumnDecisionTreeTrainerInput(m, catsInfo)); - - X.println("Training took: " + (System.currentTimeMillis() - before) + " ms."); - - byRegion.keySet().forEach(k -> { - LabeledVectorDouble sp = byRegion.get(k).get(0); - Tracer.showAscii(sp.features()); - X.println("Predicted value and label [pred=" + mdl.apply(sp.features()) + ", label=" + sp.doubleLabel() + "]"); - assert mdl.apply(sp.features()) == sp.doubleLabel(); - }); - } - - /** - * Test decision tree regression. - * To run this test rename this method so it starts from 'test'. - */ - public void tstF1() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - int ptsCnt = 10000; - Map ranges = new HashMap<>(); - - ranges.put(0, new double[] {-100.0, 100.0}); - ranges.put(1, new double[] {-100.0, 100.0}); - ranges.put(2, new double[] {-100.0, 100.0}); - - int featCnt = 100; - double[] defRng = {-1.0, 1.0}; - - Vector[] trainVectors = vecsFromRanges(ranges, featCnt, defRng, new Random(123L), ptsCnt, f1); - - SparseDistributedMatrix m = new SparseDistributedMatrix(ptsCnt, featCnt + 1, StorageConstants.COLUMN_STORAGE_MODE, StorageConstants.RANDOM_ACCESS_MODE); - - SparseDistributedMatrixStorage sto = (SparseDistributedMatrixStorage)m.getStorage(); - - loadVectorsIntoSparseDistributedMatrixCache(sto.cache().getName(), sto.getUUID(), Arrays.stream(trainVectors).iterator(), featCnt + 1); - - IgniteFunction regCalc = s -> s.average().orElse(0.0); - - ColumnDecisionTreeTrainer trainer = - new ColumnDecisionTreeTrainer<>(10, ContinuousSplitCalculators.VARIANCE, RegionCalculators.VARIANCE, regCalc, ignite); - - X.println("Training started."); - long before = System.currentTimeMillis(); - DecisionTreeModel mdl = trainer.train(new MatrixColumnDecisionTreeTrainerInput(m, new HashMap<>())); - X.println("Training finished in: " + (System.currentTimeMillis() - before) + " ms."); - - Vector[] testVectors = vecsFromRanges(ranges, featCnt, defRng, new Random(123L), 20, f1); - - IgniteTriFunction, Stream>, Function, Double> mse = Estimators.MSE(); - Double accuracy = mse.apply(mdl, Arrays.stream(testVectors).map(v -> new IgniteBiTuple<>(v.viewPart(0, featCnt), v.getX(featCnt))), Function.identity()); - X.println("MSE: " + accuracy); - } - - /** - * Load vectors into sparse distributed matrix. - * - * @param cacheName Name of cache where matrix is stored. - * @param uuid UUID of matrix. - * @param iter Iterator over vectors. - * @param vectorSize size of vectors. - */ - private void loadVectorsIntoSparseDistributedMatrixCache(String cacheName, UUID uuid, - Iterator iter, int vectorSize) { - try (IgniteDataStreamer> streamer = - Ignition.localIgnite().dataStreamer(cacheName)) { - int sampleIdx = 0; - streamer.allowOverwrite(true); - - streamer.receiver(StreamTransformer.from((e, arg) -> { - Map val = e.getValue(); - - if (val == null) - val = new Int2DoubleOpenHashMap(); - - val.putAll((Map)arg[0]); - - e.setValue(val); - - return null; - })); - - // Feature index -> (sample index -> value) - Map> batch = new HashMap<>(); - IntStream.range(0, vectorSize).forEach(i -> batch.put(i, new HashMap<>())); - int batchSize = 1000; - - while (iter.hasNext()) { - org.apache.ignite.ml.math.Vector next = iter.next(); - - for (int i = 0; i < vectorSize; i++) - batch.get(i).put(sampleIdx, next.getX(i)); - - X.println("Sample index: " + sampleIdx); - if (sampleIdx % batchSize == 0) { - batch.keySet().forEach(fi -> streamer.addData(new SparseMatrixKey(fi, uuid, fi), batch.get(fi))); - IntStream.range(0, vectorSize).forEach(i -> batch.put(i, new HashMap<>())); - } - sampleIdx++; - } - if (sampleIdx % batchSize != 0) { - batch.keySet().forEach(fi -> streamer.addData(new SparseMatrixKey(fi, uuid, fi), batch.get(fi))); - IntStream.range(0, vectorSize).forEach(i -> batch.put(i, new HashMap<>())); - } - } - } - - /** - * Load vectors into bi-indexed cache. - * - * @param cacheName Name of cache. - * @param iter Iterator over vectors. - * @param vectorSize size of vectors. - */ - private void loadVectorsIntoBiIndexedCache(String cacheName, - Iterator iter, int vectorSize) { - try (IgniteDataStreamer streamer = - Ignition.localIgnite().dataStreamer(cacheName)) { - int sampleIdx = 0; - - streamer.perNodeBufferSize(10000); - - while (iter.hasNext()) { - org.apache.ignite.ml.math.Vector next = iter.next(); - - for (int i = 0; i < vectorSize; i++) - streamer.addData(new BiIndex(sampleIdx, i), next.getX(i)); - - sampleIdx++; - - if (sampleIdx % 1000 == 0) - System.out.println("Loaded: " + sampleIdx + " vectors."); - } - } - } - - /** - * Create bi-indexed cache for tests. - * - * @return Bi-indexed cache. - */ - private IgniteCache createBiIndexedCache() { - CacheConfiguration cfg = new CacheConfiguration<>(); - - // Write to primary. - cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC); - - // Atomic transactions only. - cfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); - - // No eviction. - cfg.setEvictionPolicy(null); - - // No copying of values. - cfg.setCopyOnRead(false); - - // Cache is partitioned. - cfg.setCacheMode(CacheMode.PARTITIONED); - - cfg.setBackups(0); - - cfg.setName("TMP_BI_INDEXED_CACHE"); - - return Ignition.localIgnite().getOrCreateCache(cfg); - } - - /** */ - private Vector[] vecsFromRanges(Map ranges, int featCnt, double[] defRng, Random rnd, int ptsCnt, - Function f) { - int vs = featCnt + 1; - DenseLocalOnHeapVector[] res = new DenseLocalOnHeapVector[ptsCnt]; - for (int pt = 0; pt < ptsCnt; pt++) { - DenseLocalOnHeapVector v = new DenseLocalOnHeapVector(vs); - for (int i = 0; i < featCnt; i++) { - double[] range = ranges.getOrDefault(i, defRng); - double from = range[0]; - double to = range[1]; - double rng = to - from; - - v.setX(i, rnd.nextDouble() * rng); - } - v.setX(featCnt, f.apply(v)); - res[pt] = v; - } - - return res; - } -} diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/trees/IgniteColumnDecisionTreeGiniBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/trees/IgniteColumnDecisionTreeGiniBenchmark.java deleted file mode 100644 index f8a7c0834d825..0000000000000 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/trees/IgniteColumnDecisionTreeGiniBenchmark.java +++ /dev/null @@ -1,70 +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.yardstick.ml.trees; - -import java.util.HashMap; -import java.util.Map; -import org.apache.ignite.Ignite; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; -import org.apache.ignite.ml.trees.trainers.columnbased.contsplitcalcs.ContinuousSplitCalculators; -import org.apache.ignite.ml.trees.trainers.columnbased.regcalcs.RegionCalculators; -import org.apache.ignite.resources.IgniteInstanceResource; -import org.apache.ignite.thread.IgniteThread; -import org.apache.ignite.yardstick.IgniteAbstractBenchmark; - -/** - * Ignite benchmark that performs ML Grid operations. - */ -@SuppressWarnings("unused") -public class IgniteColumnDecisionTreeGiniBenchmark extends IgniteAbstractBenchmark { - /** */ - @IgniteInstanceResource - private Ignite ignite; - - /** {@inheritDoc} */ - @Override public boolean test(Map ctx) throws Exception { - // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread - // because we create ignite cache internally. - IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), - this.getClass().getSimpleName(), new Runnable() { - /** {@inheritDoc} */ - @Override public void run() { - // IMPL NOTE originally taken from ColumnDecisionTreeTrainerTest#testCacheMixedGini - int totalPts = 1 << 10; - int featCnt = 2; - - HashMap catsInfo = new HashMap<>(); - catsInfo.put(1, 3); - - SplitDataGenerator gen = new SplitDataGenerator<>( - featCnt, catsInfo, () -> new DenseLocalOnHeapVector(featCnt + 1)). - split(0, 1, new int[] {0, 2}). - split(1, 0, -10.0); - - gen.testByGen(totalPts, ContinuousSplitCalculators.GINI.apply(ignite), - RegionCalculators.GINI, RegionCalculators.MEAN, ignite); - } - }); - - igniteThread.start(); - - igniteThread.join(); - - return true; - } -} diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/trees/IgniteColumnDecisionTreeVarianceBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/trees/IgniteColumnDecisionTreeVarianceBenchmark.java deleted file mode 100644 index f9d417f47051e..0000000000000 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/trees/IgniteColumnDecisionTreeVarianceBenchmark.java +++ /dev/null @@ -1,71 +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.yardstick.ml.trees; - -import java.util.HashMap; -import java.util.Map; -import org.apache.ignite.Ignite; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; -import org.apache.ignite.ml.trees.trainers.columnbased.contsplitcalcs.ContinuousSplitCalculators; -import org.apache.ignite.ml.trees.trainers.columnbased.regcalcs.RegionCalculators; -import org.apache.ignite.resources.IgniteInstanceResource; -import org.apache.ignite.thread.IgniteThread; -import org.apache.ignite.yardstick.IgniteAbstractBenchmark; - -/** - * Ignite benchmark that performs ML Grid operations. - */ -@SuppressWarnings("unused") -public class IgniteColumnDecisionTreeVarianceBenchmark extends IgniteAbstractBenchmark { - /** */ - @IgniteInstanceResource - private Ignite ignite; - - /** {@inheritDoc} */ - @Override public boolean test(Map ctx) throws Exception { - // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread - // because we create ignite cache internally. - IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), - this.getClass().getSimpleName(), new Runnable() { - /** {@inheritDoc} */ - @Override public void run() { - // IMPL NOTE originally taken from ColumnDecisionTreeTrainerTest#testCacheMixed - int totalPts = 1 << 10; - int featCnt = 2; - - HashMap catsInfo = new HashMap<>(); - catsInfo.put(1, 3); - - SplitDataGenerator gen - = new SplitDataGenerator<>( - featCnt, catsInfo, () -> new DenseLocalOnHeapVector(featCnt + 1)). - split(0, 1, new int[] {0, 2}). - split(1, 0, -10.0); - - gen.testByGen(totalPts, - ContinuousSplitCalculators.VARIANCE, RegionCalculators.VARIANCE, RegionCalculators.MEAN, ignite); - } - }); - - igniteThread.start(); - - igniteThread.join(); - - return true; - } -} diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/trees/SplitDataGenerator.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/trees/SplitDataGenerator.java deleted file mode 100644 index f9117f4bb38be..0000000000000 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/trees/SplitDataGenerator.java +++ /dev/null @@ -1,426 +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.yardstick.ml.trees; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.BitSet; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.function.BiFunction; -import java.util.function.Function; -import java.util.function.Supplier; -import java.util.stream.Collectors; -import java.util.stream.DoubleStream; -import java.util.stream.IntStream; -import java.util.stream.Stream; -import org.apache.ignite.Ignite; -import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.ml.math.StorageConstants; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.exceptions.MathIllegalArgumentException; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; -import org.apache.ignite.ml.structures.LabeledVectorDouble; -import org.apache.ignite.ml.trees.ContinuousRegionInfo; -import org.apache.ignite.ml.trees.ContinuousSplitCalculator; -import org.apache.ignite.ml.trees.models.DecisionTreeModel; -import org.apache.ignite.ml.trees.trainers.columnbased.ColumnDecisionTreeTrainer; -import org.apache.ignite.ml.trees.trainers.columnbased.ColumnDecisionTreeTrainerInput; -import org.apache.ignite.ml.trees.trainers.columnbased.MatrixColumnDecisionTreeTrainerInput; -import org.apache.ignite.ml.util.Utils; - -/** */ -class SplitDataGenerator { - /** */ - private static final Random rnd = new Random(12349L); - - /** */ - private static final double DELTA = 100.0; - - /** Map of the form of (is categorical -> list of region indexes). */ - private final Map> di; - - /** List of regions. */ - private final List regs; - - /** Data of bounds of regions. */ - private final Map> boundsData; - - /** */ - private final Map catFeaturesInfo; - - /** Supplier of vectors. */ - private final Supplier supplier; - - /** Features count. */ - private final int featCnt; - - /** - * Create SplitDataGenerator. - * - * @param featCnt Features count. - * @param catFeaturesInfo Information about categorical features in form of map (feature index -> categories - * count). - * @param supplier Supplier of vectors. - */ - SplitDataGenerator(int featCnt, Map catFeaturesInfo, Supplier supplier) { - regs = new LinkedList<>(); - boundsData = new HashMap<>(); - this.supplier = supplier; - this.featCnt = featCnt; - this.catFeaturesInfo = catFeaturesInfo; - - // Divide indexes into indexes of categorical coordinates and indexes of continuous coordinates. - di = IntStream.range(0, featCnt). - boxed(). - collect(Collectors.partitioningBy(catFeaturesInfo::containsKey)); - - // Categorical coordinates info. - Map catCoords = new HashMap<>(); - di.get(true).forEach(i -> { - BitSet bs = new BitSet(); - bs.set(0, catFeaturesInfo.get(i)); - catCoords.put(i, new CatCoordInfo(bs)); - }); - - // Continuous coordinates info. - Map contCoords = new HashMap<>(); - di.get(false).forEach(i -> { - contCoords.put(i, new ContCoordInfo()); - boundsData.put(i, new IgniteBiTuple<>(-1.0, 1.0)); - }); - - Region firstReg = new Region(catCoords, contCoords, 0); - regs.add(firstReg); - } - - /** */ - void testByGen(int totalPts, - IgniteFunction> calc, - IgniteFunction> catImpCalc, - IgniteFunction regCalc, Ignite ignite) { - - List> lst = points(totalPts, (i, rn) -> i).collect(Collectors.toList()); - - Collections.shuffle(lst, rnd); - - SparseDistributedMatrix m = new SparseDistributedMatrix(totalPts, - featCnt + 1, StorageConstants.COLUMN_STORAGE_MODE, StorageConstants.RANDOM_ACCESS_MODE); - - Map> byRegion = new HashMap<>(); - - int i = 0; - for (IgniteBiTuple bt : lst) { - byRegion.putIfAbsent(bt.get1(), new LinkedList<>()); - byRegion.get(bt.get1()).add(asLabeledVector(bt.get2().getStorage().data())); - m.setRow(i, bt.get2().getStorage().data()); - i++; - } - - ColumnDecisionTreeTrainer trainer = - new ColumnDecisionTreeTrainer<>(3, calc, catImpCalc, regCalc, ignite); - - DecisionTreeModel mdl = trainer.train(new MatrixColumnDecisionTreeTrainerInput(m, catFeaturesInfo)); - - byRegion.keySet().forEach(k -> mdl.apply(byRegion.get(k).get(0).features())); - } - - /** - * Split region by continuous coordinate using given threshold. - * - * @param regIdx Region index. - * @param coordIdx Coordinate index. - * @param threshold Threshold. - * @return {@code this}. - */ - SplitDataGenerator split(int regIdx, int coordIdx, double threshold) { - Region regToSplit = regs.get(regIdx); - ContCoordInfo cci = regToSplit.contCoords.get(coordIdx); - - double left = cci.left; - double right = cci.right; - - if (threshold < left || threshold > right) - throw new MathIllegalArgumentException("Threshold is out of region bounds."); - - regToSplit.incTwoPow(); - - Region newReg = Utils.copy(regToSplit); - newReg.contCoords.get(coordIdx).left = threshold; - - regs.add(regIdx + 1, newReg); - cci.right = threshold; - - IgniteBiTuple bounds = boundsData.get(coordIdx); - double min = bounds.get1(); - double max = bounds.get2(); - boundsData.put(coordIdx, new IgniteBiTuple<>(Math.min(threshold, min), Math.max(max, threshold))); - - return this; - } - - /** - * Split region by categorical coordinate. - * - * @param regIdx Region index. - * @param coordIdx Coordinate index. - * @param cats Categories allowed for the left sub region. - * @return {@code this}. - */ - SplitDataGenerator split(int regIdx, int coordIdx, int[] cats) { - BitSet subset = new BitSet(); - Arrays.stream(cats).forEach(subset::set); - Region regToSplit = regs.get(regIdx); - CatCoordInfo cci = regToSplit.catCoords.get(coordIdx); - - BitSet ssc = (BitSet)subset.clone(); - BitSet set = cci.bs; - ssc.and(set); - if (ssc.length() != subset.length()) - throw new MathIllegalArgumentException("Splitter set is not a subset of a parent subset."); - - ssc.xor(set); - set.and(subset); - - regToSplit.incTwoPow(); - Region newReg = Utils.copy(regToSplit); - newReg.catCoords.put(coordIdx, new CatCoordInfo(ssc)); - - regs.add(regIdx + 1, newReg); - - return this; - } - - /** - * Get stream of points generated by this generator. - * - * @param ptsCnt Points count. - */ - private Stream> points(int ptsCnt, BiFunction f) { - return IntStream.range(0, regs.size()). - boxed(). - map(i -> regs.get(i).generatePoints(ptsCnt, f.apply((double)i, rnd), boundsData, di, supplier, rnd) - .map(v -> new IgniteBiTuple<>(i, v))).flatMap(Function.identity()); - } - - /** - * Convert double array to {@link LabeledVectorDouble} - * - * @param arr Array for conversion. - * @return LabeledVectorDouble. - */ - private static LabeledVectorDouble asLabeledVector(double arr[]) { - return new LabeledVectorDouble<>(new DenseLocalOnHeapVector( - Arrays.copyOf(arr, arr.length - 1)), arr[arr.length - 1]); - } - - /** - * Categorical coordinate info. - */ - private static class CatCoordInfo implements Serializable { - /** - * Defines categories which are included in this region - */ - private final BitSet bs; - - /** - * Construct CatCoordInfo. - * - * @param bs Bitset. - */ - CatCoordInfo(BitSet bs) { - this.bs = bs; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return "CatCoordInfo [" + - "bs=" + bs + - ']'; - } - } - - /** - * Continuous coordinate info. - */ - private static class ContCoordInfo implements Serializable { - /** - * Left (min) bound of region. - */ - private double left; - - /** - * Right (max) bound of region. - */ - private double right; - - /** - * Construct ContCoordInfo. - */ - ContCoordInfo() { - left = Double.NEGATIVE_INFINITY; - right = Double.POSITIVE_INFINITY; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return "ContCoordInfo [" + - "left=" + left + - ", right=" + right + - ']'; - } - } - - /** - * Class representing information about region. - */ - private static class Region implements Serializable { - /** - * Information about categorical coordinates restrictions of this region in form of - * (coordinate index -> restriction) - */ - private final Map catCoords; - - /** - * Information about continuous coordinates restrictions of this region in form of - * (coordinate index -> restriction) - */ - private final Map contCoords; - - /** - * Region should contain {@code 1/2^twoPow * totalPoints} points. - */ - private int twoPow; - - /** - * Construct region by information about restrictions on coordinates (features) values. - * - * @param catCoords Restrictions on categorical coordinates. - * @param contCoords Restrictions on continuous coordinates - * @param twoPow Region should contain {@code 1/2^twoPow * totalPoints} points. - */ - Region(Map catCoords, Map contCoords, int twoPow) { - this.catCoords = catCoords; - this.contCoords = contCoords; - this.twoPow = twoPow; - } - - /** */ - int divideBy() { - return 1 << twoPow; - } - - /** */ - void incTwoPow() { - twoPow++; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return "Region [" + - "catCoords=" + catCoords + - ", contCoords=" + contCoords + - ", twoPow=" + twoPow + - ']'; - } - - /** - * Generate continuous coordinate for this region. - * - * @param coordIdx Coordinate index. - * @param boundsData Data with bounds - * @param rnd Random numbers generator. - * @return Categorical coordinate value. - */ - double generateContCoord(int coordIdx, Map> boundsData, - Random rnd) { - ContCoordInfo cci = contCoords.get(coordIdx); - double left = cci.left; - double right = cci.right; - - if (left == Double.NEGATIVE_INFINITY) - left = boundsData.get(coordIdx).get1() - DELTA; - - if (right == Double.POSITIVE_INFINITY) - right = boundsData.get(coordIdx).get2() + DELTA; - - double size = right - left; - - return left + rnd.nextDouble() * size; - } - - /** - * Generate categorical coordinate value for this region. - * - * @param coordIdx Coordinate index. - * @param rnd Random numbers generator. - * @return Categorical coordinate value. - */ - double generateCatCoord(int coordIdx, Random rnd) { - // Pick random bit. - BitSet bs = catCoords.get(coordIdx).bs; - int j = rnd.nextInt(bs.length()); - - int i = 0; - int bn = 0; - int bnp = 0; - - while ((bn = bs.nextSetBit(bn)) != -1 && i <= j) { - i++; - bnp = bn; - bn++; - } - - return bnp; - } - - /** - * Generate points for this region. - * - * @param ptsCnt Count of points to generate. - * @param val Label for all points in this region. - * @param boundsData Data about bounds of continuous coordinates. - * @param catCont Data about which categories can be in this region in the form (coordinate index -> list of - * categories indexes). - * @param s Vectors supplier. - * @param rnd Random numbers generator. - * @param Type of vectors. - * @return Stream of generated points for this region. - */ - Stream generatePoints(int ptsCnt, double val, - Map> boundsData, Map> catCont, - Supplier s, - Random rnd) { - return IntStream.range(0, ptsCnt / divideBy()).mapToObj(i -> { - V v = s.get(); - int coordsCnt = v.size(); - catCont.get(false).forEach(ci -> v.setX(ci, generateContCoord(ci, boundsData, rnd))); - catCont.get(true).forEach(ci -> v.setX(ci, generateCatCoord(ci, rnd))); - - v.setX(coordsCnt - 1, val); - return v; - }); - } - } -} diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/trees/package-info.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/trees/package-info.java deleted file mode 100644 index fc379a607bcf9..0000000000000 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/trees/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. - */ - -/** - * - * ML Grid decision tree benchmarks. - */ -package org.apache.ignite.yardstick.ml.trees; \ No newline at end of file From 1ea17c821f51d8725290021599a60cbbdfd2ee25 Mon Sep 17 00:00:00 2001 From: Sergey Chugunov Date: Wed, 11 Apr 2018 12:28:40 +0300 Subject: [PATCH 0021/1463] IGNITE-8216 Fixed javadoc for release build --- parent/pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/parent/pom.xml b/parent/pom.xml index 16a9395c874d8..3decc16612374 100644 --- a/parent/pom.xml +++ b/parent/pom.xml @@ -403,6 +403,10 @@ SPI: Event Storage org.apache.ignite.spi.eventstorage* + + Communication Failure Detection + org.apache.ignite.failure + Segmentation Detection org.apache.ignite.plugin.segmentation From 6bc937575d5dfbe1d2c05865c8df0cabc6ace90c Mon Sep 17 00:00:00 2001 From: zaleslaw Date: Wed, 11 Apr 2018 12:31:48 +0300 Subject: [PATCH 0022/1463] IGNITE-7830: Knn Lin Reg with new datasets this closes #3583 (cherry picked from commit a4653b7) --- .../org/apache/ignite/ml/knn/KNNUtils.java | 59 ++++++++ .../KNNClassificationTrainer.java | 23 +-- .../ml/knn/regression/KNNRegressionModel.java | 87 +++++++++++ .../knn/regression/KNNRegressionTrainer.java | 40 +++++ .../ml/knn/regression/package-info.java | 22 +++ .../ignite/ml/knn/KNNRegressionTest.java | 143 ++++++++++++++++++ .../apache/ignite/ml/knn/KNNTestSuite.java | 1 + 7 files changed, 354 insertions(+), 21 deletions(-) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/knn/KNNUtils.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionModel.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionTrainer.java create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/package-info.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.java diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/KNNUtils.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/KNNUtils.java new file mode 100644 index 0000000000000..88fa70f8c9ea2 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/KNNUtils.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.knn; + +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.dataset.PartitionDataBuilder; +import org.apache.ignite.ml.knn.partitions.KNNPartitionContext; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.structures.LabeledDataset; +import org.apache.ignite.ml.structures.LabeledVector; +import org.apache.ignite.ml.structures.partition.LabeledDatasetPartitionDataBuilderOnHeap; +import org.jetbrains.annotations.Nullable; + +/** + * Helper class for KNNRegression. + */ +public class KNNUtils { + /** + * Builds dataset. + * + * @param datasetBuilder Dataset builder. + * @param featureExtractor Feature extractor. + * @param lbExtractor Label extractor. + * @return Dataset. + */ + @Nullable public static Dataset> buildDataset(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + PartitionDataBuilder> partDataBuilder + = new LabeledDatasetPartitionDataBuilderOnHeap<>( + featureExtractor, + lbExtractor + ); + + Dataset> dataset = null; + + if (datasetBuilder != null) { + dataset = datasetBuilder.build( + (upstream, upstreamSize) -> new KNNPartitionContext(), + partDataBuilder + ); + } + return dataset; + } +} 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 357047f81313f..c0c8e6593d3e4 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 @@ -17,14 +17,9 @@ package org.apache.ignite.ml.knn.classification; -import org.apache.ignite.ml.dataset.Dataset; import org.apache.ignite.ml.dataset.DatasetBuilder; -import org.apache.ignite.ml.dataset.PartitionDataBuilder; -import org.apache.ignite.ml.knn.partitions.KNNPartitionContext; +import org.apache.ignite.ml.knn.KNNUtils; import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.structures.LabeledDataset; -import org.apache.ignite.ml.structures.partition.LabeledDatasetPartitionDataBuilderOnHeap; -import org.apache.ignite.ml.structures.LabeledVector; import org.apache.ignite.ml.trainers.SingleLabelDatasetTrainer; /** @@ -41,20 +36,6 @@ public class KNNClassificationTrainer implements SingleLabelDatasetTrainer KNNClassificationModel fit(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { - PartitionDataBuilder> partDataBuilder - = new LabeledDatasetPartitionDataBuilderOnHeap<>( - featureExtractor, - lbExtractor - ); - - Dataset> dataset = null; - - if (datasetBuilder != null) { - dataset = datasetBuilder.build( - (upstream, upstreamSize) -> new KNNPartitionContext(), - partDataBuilder - ); - } - return new KNNClassificationModel<>(dataset); + return new KNNClassificationModel<>(KNNUtils.buildDataset(datasetBuilder, featureExtractor, lbExtractor)); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionModel.java new file mode 100644 index 0000000000000..cabc1438e1d77 --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionModel.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.knn.regression; + +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.knn.classification.KNNClassificationModel; +import org.apache.ignite.ml.knn.partitions.KNNPartitionContext; +import org.apache.ignite.ml.math.Vector; +import org.apache.ignite.ml.math.exceptions.UnsupportedOperationException; +import org.apache.ignite.ml.structures.LabeledDataset; +import org.apache.ignite.ml.structures.LabeledVector; + +import java.util.List; + +/** + * This class provides kNN Multiple Linear Regression or Locally [weighted] regression (Simple and Weighted versions). + * + *

    This is an instance-based learning method.

    + * + *
      + *
    • Local means using nearby points (i.e. a nearest neighbors approach).
    • + *
    • Weighted means we value points based upon how far away they are.
    • + *
    • Regression means approximating a function.
    • + *
    + */ +public class KNNRegressionModel extends KNNClassificationModel { + /** + * Builds the model via prepared dataset. + * @param dataset Specially prepared object to run algorithm over it. + */ + public KNNRegressionModel(Dataset> dataset) { + super(dataset); + } + + /** {@inheritDoc} */ + @Override public Double apply(Vector v) { + List neighbors = findKNearestNeighbors(v); + + return predictYBasedOn(neighbors, v); + } + + /** */ + private double predictYBasedOn(List neighbors, Vector v) { + switch (stgy) { + case SIMPLE: + return simpleRegression(neighbors); + case WEIGHTED: + return weightedRegression(neighbors, v); + default: + throw new UnsupportedOperationException("Strategy " + stgy.name() + " is not supported"); + } + } + + /** */ + private double weightedRegression(List neighbors, Vector v) { + double sum = 0.0; + double div = 0.0; + for (LabeledVector neighbor : neighbors) { + double distance = distanceMeasure.compute(v, neighbor.features()); + sum += neighbor.label() * distance; + div += distance; + } + return sum / div; + } + + /** */ + private double simpleRegression(List neighbors) { + double sum = 0.0; + for (LabeledVector neighbor : neighbors) + sum += neighbor.label(); + return sum / (double)k; + } +} \ No newline at end of file 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 new file mode 100644 index 0000000000000..2d13cd50417df --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionTrainer.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.ml.knn.regression; + +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.knn.KNNUtils; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; + +/** + * kNN algorithm trainer to solve regression task. + */ +public class KNNRegressionTrainer{ + /** + * Trains model based on the specified data. + * + * @param datasetBuilder Dataset builder. + * @param featureExtractor Feature extractor. + * @param lbExtractor Label extractor. + * @return Model. + */ + public KNNRegressionModel fit(DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + return new KNNRegressionModel<>(KNNUtils.buildDataset(datasetBuilder, featureExtractor, lbExtractor)); + } +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/package-info.java new file mode 100644 index 0000000000000..82e71929e1c9f --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/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 helper classes for kNN regression algorithms. + */ +package org.apache.ignite.ml.knn.regression; 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 new file mode 100644 index 0000000000000..66dbca9ff038e --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.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.ml.knn; + +import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; +import org.apache.ignite.ml.knn.classification.KNNStrategy; +import org.apache.ignite.ml.knn.regression.KNNRegressionModel; +import org.apache.ignite.ml.knn.regression.KNNRegressionTrainer; +import org.apache.ignite.ml.math.Vector; +import org.apache.ignite.ml.math.distances.EuclideanDistance; +import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; +import org.junit.Assert; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +/** + * Tests for {@link KNNRegressionTrainer}. + */ +public class KNNRegressionTest extends BaseKNNTest { + /** */ + private double[] y; + + /** */ + private double[][] x; + + /** */ + public void testSimpleRegressionWithOneNeighbour() { + IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); + + 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 knnMdl = (KNNRegressionModel) trainer.fit( + new LocalDatasetBuilder<>(data, 2), + (k, v) -> Arrays.copyOfRange(v, 1, v.length), + (k, v) -> v[0] + ).withK(1) + .withDistanceMeasure(new EuclideanDistance()) + .withStrategy(KNNStrategy.SIMPLE); + + Vector vector = new DenseLocalOnHeapVector(new double[] {0, 0, 0, 5.0, 0.0}); + System.out.println(knnMdl.apply(vector)); + Assert.assertEquals(15, knnMdl.apply(vector), 1E-12); + } + + /** */ + public void testLongly() { + + IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); + + Map data = new HashMap<>(); + data.put(0, new double[] {60323, 83.0, 234289, 2356, 1590, 107608, 1947}); + data.put(1, new double[] {61122, 88.5, 259426, 2325, 1456, 108632, 1948}); + data.put(2, new double[] {60171, 88.2, 258054, 3682, 1616, 109773, 1949}); + data.put(3, new double[] {61187, 89.5, 284599, 3351, 1650, 110929, 1950}); + data.put(4, new double[] {63221, 96.2, 328975, 2099, 3099, 112075, 1951}); + data.put(5, new double[] {63639, 98.1, 346999, 1932, 3594, 113270, 1952}); + data.put(6, new double[] {64989, 99.0, 365385, 1870, 3547, 115094, 1953}); + data.put(7, new double[] {63761, 100.0, 363112, 3578, 3350, 116219, 1954}); + data.put(8, new double[] {66019, 101.2, 397469, 2904, 3048, 117388, 1955}); + data.put(9, new double[] {68169, 108.4, 442769, 2936, 2798, 120445, 1957}); + data.put(10, new double[] {66513, 110.8, 444546, 4681, 2637, 121950, 1958}); + data.put(11, new double[] {68655, 112.6, 482704, 3813, 2552, 123366, 1959}); + data.put(12, new double[] {69564, 114.2, 502601, 3931, 2514, 125368, 1960}); + data.put(13, new double[] {69331, 115.7, 518173, 4806, 2572, 127852, 1961}); + data.put(14, new double[] {70551, 116.9, 554894, 4007, 2827, 130081, 1962}); + + KNNRegressionTrainer trainer = new KNNRegressionTrainer(); + + KNNRegressionModel knnMdl = (KNNRegressionModel) trainer.fit( + new LocalDatasetBuilder<>(data, 2), + (k, v) -> Arrays.copyOfRange(v, 1, v.length), + (k, v) -> v[0] + ).withK(3) + .withDistanceMeasure(new EuclideanDistance()) + .withStrategy(KNNStrategy.SIMPLE); + + Vector vector = new DenseLocalOnHeapVector(new double[] {104.6, 419180, 2822, 2857, 118734, 1956}); + System.out.println(knnMdl.apply(vector)); + Assert.assertEquals(67857, knnMdl.apply(vector), 2000); + } + + /** */ + public void testLonglyWithWeightedStrategy() { + IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); + + Map data = new HashMap<>(); + data.put(0, new double[] {60323, 83.0, 234289, 2356, 1590, 107608, 1947}); + data.put(1, new double[] {61122, 88.5, 259426, 2325, 1456, 108632, 1948}); + data.put(2, new double[] {60171, 88.2, 258054, 3682, 1616, 109773, 1949}); + data.put(3, new double[] {61187, 89.5, 284599, 3351, 1650, 110929, 1950}); + data.put(4, new double[] {63221, 96.2, 328975, 2099, 3099, 112075, 1951}); + data.put(5, new double[] {63639, 98.1, 346999, 1932, 3594, 113270, 1952}); + data.put(6, new double[] {64989, 99.0, 365385, 1870, 3547, 115094, 1953}); + data.put(7, new double[] {63761, 100.0, 363112, 3578, 3350, 116219, 1954}); + data.put(8, new double[] {66019, 101.2, 397469, 2904, 3048, 117388, 1955}); + data.put(9, new double[] {68169, 108.4, 442769, 2936, 2798, 120445, 1957}); + data.put(10, new double[] {66513, 110.8, 444546, 4681, 2637, 121950, 1958}); + data.put(11, new double[] {68655, 112.6, 482704, 3813, 2552, 123366, 1959}); + data.put(12, new double[] {69564, 114.2, 502601, 3931, 2514, 125368, 1960}); + data.put(13, new double[] {69331, 115.7, 518173, 4806, 2572, 127852, 1961}); + data.put(14, new double[] {70551, 116.9, 554894, 4007, 2827, 130081, 1962}); + + KNNRegressionTrainer trainer = new KNNRegressionTrainer(); + + KNNRegressionModel knnMdl = (KNNRegressionModel) trainer.fit( + new LocalDatasetBuilder<>(data, 2), + (k, v) -> Arrays.copyOfRange(v, 1, v.length), + (k, v) -> v[0] + ).withK(3) + .withDistanceMeasure(new EuclideanDistance()) + .withStrategy(KNNStrategy.SIMPLE); + + Vector vector = new DenseLocalOnHeapVector(new double[] {104.6, 419180, 2822, 2857, 118734, 1956}); + System.out.println(knnMdl.apply(vector)); + Assert.assertEquals(67857, knnMdl.apply(vector), 2000); + } +} \ No newline at end of file diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNTestSuite.java index 95ebec5a2859b..55ef24e700005 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNTestSuite.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNTestSuite.java @@ -26,6 +26,7 @@ @RunWith(Suite.class) @Suite.SuiteClasses({ KNNClassificationTest.class, + KNNRegressionTest.class, LabeledDatasetTest.class }) public class KNNTestSuite { From b96271866aeb09b434e2b95a457655a44d913254 Mon Sep 17 00:00:00 2001 From: Alexander Kalinin Date: Wed, 11 Apr 2018 17:09:41 +0700 Subject: [PATCH 0023/1463] IGNITE-4091 Web Console: Refactored using of internal Angular API. (cherry picked from commit 74d2545) --- .../web-console/frontend/app/app.config.js | 14 +++--- .../modal-import-models/component.js | 4 +- .../app/components/page-profile/controller.js | 4 +- .../frontend/app/modules/ace.module.js | 47 ++++++++++--------- .../services/AngularStrapSelect.decorator.js | 5 +- .../services/AngularStrapTooltip.decorator.js | 8 ++-- .../app/services/FormUtils.service.js | 3 +- 7 files changed, 45 insertions(+), 40 deletions(-) diff --git a/modules/web-console/frontend/app/app.config.js b/modules/web-console/frontend/app/app.config.js index 9d8dc99cec52a..e2bc057fc4ece 100644 --- a/modules/web-console/frontend/app/app.config.js +++ b/modules/web-console/frontend/app/app.config.js @@ -43,7 +43,7 @@ igniteConsoleCfg.config(['$animateProvider', ($animateProvider) => { // AngularStrap modal popup configuration. igniteConsoleCfg.config(['$modalProvider', ($modalProvider) => { - angular.extend($modalProvider.defaults, { + Object.assign($modalProvider.defaults, { animation: 'am-fade-and-scale', placement: 'center', html: true @@ -52,7 +52,7 @@ igniteConsoleCfg.config(['$modalProvider', ($modalProvider) => { // AngularStrap popover configuration. igniteConsoleCfg.config(['$popoverProvider', ($popoverProvider) => { - angular.extend($popoverProvider.defaults, { + Object.assign($popoverProvider.defaults, { trigger: 'manual', placement: 'right', container: 'body', @@ -62,7 +62,7 @@ igniteConsoleCfg.config(['$popoverProvider', ($popoverProvider) => { // AngularStrap tooltips configuration. igniteConsoleCfg.config(['$tooltipProvider', ($tooltipProvider) => { - angular.extend($tooltipProvider.defaults, { + Object.assign($tooltipProvider.defaults, { container: 'body', delay: {show: 150, hide: 150}, placement: 'right', @@ -73,7 +73,7 @@ igniteConsoleCfg.config(['$tooltipProvider', ($tooltipProvider) => { // AngularStrap select (combobox) configuration. igniteConsoleCfg.config(['$selectProvider', ($selectProvider) => { - angular.extend($selectProvider.defaults, { + Object.assign($selectProvider.defaults, { container: 'body', maxLength: '5', allText: 'Select All', @@ -87,7 +87,7 @@ igniteConsoleCfg.config(['$selectProvider', ($selectProvider) => { // AngularStrap alerts configuration. igniteConsoleCfg.config(['$alertProvider', ($alertProvider) => { - angular.extend($alertProvider.defaults, { + Object.assign($alertProvider.defaults, { container: 'body', placement: 'top-right', duration: '5', @@ -99,7 +99,7 @@ igniteConsoleCfg.config(['$alertProvider', ($alertProvider) => { // AngularStrap dropdowns () configuration. igniteConsoleCfg.config(['$dropdownProvider', ($dropdownProvider) => { - angular.extend($dropdownProvider.defaults, { + Object.assign($dropdownProvider.defaults, { templateUrl: dropdownTemplateUrl, animation: '' }); @@ -107,7 +107,7 @@ igniteConsoleCfg.config(['$dropdownProvider', ($dropdownProvider) => { // AngularStrap dropdowns () configuration. igniteConsoleCfg.config(['$datepickerProvider', ($datepickerProvider) => { - angular.extend($datepickerProvider.defaults, { + Object.assign($datepickerProvider.defaults, { autoclose: true, iconLeft: 'icon-datepicker-left', iconRight: 'icon-datepicker-right' diff --git a/modules/web-console/frontend/app/components/page-configure/components/modal-import-models/component.js b/modules/web-console/frontend/app/components/page-configure/components/modal-import-models/component.js index 7f852b02cd3ec..813c998f0e00e 100644 --- a/modules/web-console/frontend/app/components/page-configure/components/modal-import-models/component.js +++ b/modules/web-console/frontend/app/components/page-configure/components/modal-import-models/component.js @@ -84,7 +84,7 @@ const DFLT_REPLICATED_CACHE = { const CACHE_TEMPLATES = [DFLT_PARTITIONED_CACHE, DFLT_REPLICATED_CACHE]; export class ModalImportModels { - /** + /** * Cluster ID to import models into * @type {string} */ @@ -771,7 +771,7 @@ export class ModalImportModels { // Prepare caches for generation. if (table.action === IMPORT_DM_NEW_CACHE) { - const newCache = angular.copy(this.loadedCaches[table.cacheOrTemplate]); + const newCache = _.cloneDeep(this.loadedCaches[table.cacheOrTemplate]); batchAction.newCache = newCache; diff --git a/modules/web-console/frontend/app/components/page-profile/controller.js b/modules/web-console/frontend/app/components/page-profile/controller.js index 05fe1183001c9..c67a603d6a72b 100644 --- a/modules/web-console/frontend/app/components/page-profile/controller.js +++ b/modules/web-console/frontend/app/components/page-profile/controller.js @@ -15,6 +15,8 @@ * limitations under the License. */ +import _ from 'lodash'; + export default class PageProfileController { static $inject = [ '$rootScope', '$scope', '$http', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteFocus', 'IgniteConfirm', 'IgniteCountries', 'User' @@ -28,7 +30,7 @@ export default class PageProfileController { this.ui = {}; this.User.read() - .then((user) => this.ui.user = angular.copy(user)); + .then((user) => this.ui.user = _.cloneDeep(user)); this.ui.countries = this.Countries.getAll(); } diff --git a/modules/web-console/frontend/app/modules/ace.module.js b/modules/web-console/frontend/app/modules/ace.module.js index a28536a0233fa..6a6e70a45f83e 100644 --- a/modules/web-console/frontend/app/modules/ace.module.js +++ b/modules/web-console/frontend/app/modules/ace.module.js @@ -16,12 +16,13 @@ */ import angular from 'angular'; +import _ from 'lodash'; angular .module('ignite-console.ace', []) .constant('igniteAceConfig', {}) .directive('igniteAce', ['igniteAceConfig', (aceConfig) => { - if (angular.isUndefined(window.ace)) + if (_.isUndefined(window.ace)) throw new Error('ignite-ace need ace to work... (o rly?)'); /** @@ -43,7 +44,7 @@ angular */ const setOptions = (acee, session, opts) => { // Sets the ace worker path, if running from concatenated or minified source. - if (angular.isDefined(opts.workerPath)) { + if (!_.isUndefined(opts.workerPath)) { const config = window.ace.acequire('ace/config'); config.set('workerPath', opts.workerPath); @@ -53,26 +54,26 @@ angular _.forEach(opts.require, (n) => window.ace.acequire(n)); // Boolean options. - if (angular.isDefined(opts.showGutter)) + if (!_.isUndefined(opts.showGutter)) acee.renderer.setShowGutter(opts.showGutter); - if (angular.isDefined(opts.useWrapMode)) + if (!_.isUndefined(opts.useWrapMode)) session.setUseWrapMode(opts.useWrapMode); - if (angular.isDefined(opts.showInvisibles)) + if (!_.isUndefined(opts.showInvisibles)) acee.renderer.setShowInvisibles(opts.showInvisibles); - if (angular.isDefined(opts.showIndentGuides)) + if (!_.isUndefined(opts.showIndentGuides)) acee.renderer.setDisplayIndentGuides(opts.showIndentGuides); - if (angular.isDefined(opts.useSoftTabs)) + if (!_.isUndefined(opts.useSoftTabs)) session.setUseSoftTabs(opts.useSoftTabs); - if (angular.isDefined(opts.showPrintMargin)) + if (!_.isUndefined(opts.showPrintMargin)) acee.setShowPrintMargin(opts.showPrintMargin); // Commands. - if (angular.isDefined(opts.disableSearch) && opts.disableSearch) { + if (!_.isUndefined(opts.disableSearch) && opts.disableSearch) { acee.commands.addCommands([{ name: 'unfind', bindKey: { @@ -85,21 +86,21 @@ angular } // Base options. - if (angular.isString(opts.theme)) + if (_.isString(opts.theme)) acee.setTheme('ace/theme/' + opts.theme); - if (angular.isString(opts.mode)) + if (_.isString(opts.mode)) session.setMode('ace/mode/' + opts.mode); - if (angular.isDefined(opts.firstLineNumber)) { - if (angular.isNumber(opts.firstLineNumber)) + if (!_.isUndefined(opts.firstLineNumber)) { + if (_.isNumber(opts.firstLineNumber)) session.setOption('firstLineNumber', opts.firstLineNumber); - else if (angular.isFunction(opts.firstLineNumber)) + else if (_.isFunction(opts.firstLineNumber)) session.setOption('firstLineNumber', opts.firstLineNumber()); } // Advanced options. - if (angular.isDefined(opts.advanced)) { + if (!_.isUndefined(opts.advanced)) { for (const key in opts.advanced) { if (opts.advanced.hasOwnProperty(key)) { // Create a javascript object with the key and value. @@ -112,7 +113,7 @@ angular } // Advanced options for the renderer. - if (angular.isDefined(opts.rendererOptions)) { + if (!_.isUndefined(opts.rendererOptions)) { for (const key in opts.rendererOptions) { if (opts.rendererOptions.hasOwnProperty(key)) { // Create a javascript object with the key and value. @@ -126,7 +127,7 @@ angular // onLoad callbacks. _.forEach(opts.callbacks, (cb) => { - if (angular.isFunction(cb)) + if (_.isFunction(cb)) cb(acee); }); }; @@ -147,7 +148,7 @@ angular * * @type object */ - let opts = angular.extend({}, options, scope.$eval(attrs.igniteAce)); + let opts = Object.assign({}, options, scope.$eval(attrs.igniteAce)); /** * ACE editor. @@ -191,9 +192,9 @@ angular !scope.$$phase && !scope.$root.$$phase) scope.$eval(() => ngModel.$setViewValue(newValue)); - if (angular.isDefined(callback)) { + if (!_.isUndefined(callback)) { scope.$evalAsync(() => { - if (angular.isFunction(callback)) + if (_.isFunction(callback)) callback([e, acee]); else throw new Error('ignite-ace use a function as callback'); @@ -210,10 +211,10 @@ angular form && form.$removeControl(ngModel); ngModel.$formatters.push((value) => { - if (angular.isUndefined(value) || value === null) + if (_.isUndefined(value) || value === null) return ''; - if (angular.isObject(value) || angular.isArray(value)) + if (_.isObject(value) || _.isArray(value)) throw new Error('ignite-ace cannot use an object or an array as a model'); return value; @@ -229,7 +230,7 @@ angular if (current === previous) return; - opts = angular.extend({}, options, scope.$eval(attrs.igniteAce)); + opts = Object.assign({}, options, scope.$eval(attrs.igniteAce)); opts.callbacks = [opts.onLoad]; diff --git a/modules/web-console/frontend/app/services/AngularStrapSelect.decorator.js b/modules/web-console/frontend/app/services/AngularStrapSelect.decorator.js index 39f7ccdb6bec2..32fa167f2d1af 100644 --- a/modules/web-console/frontend/app/services/AngularStrapSelect.decorator.js +++ b/modules/web-console/frontend/app/services/AngularStrapSelect.decorator.js @@ -16,6 +16,7 @@ */ import angular from 'angular'; +import _ from 'lodash'; /** * Special decorator that fix problem in AngularStrap selectAll / deselectAll methods. @@ -27,12 +28,12 @@ export default angular.module('mgcrea.ngStrap.select') const delegate = $delegate(element, controller, config); // Common vars. - const options = angular.extend({}, $delegate.defaults, config); + const options = Object.assign({}, $delegate.defaults, config); const scope = delegate.$scope; const valueByIndex = (index) => { - if (angular.isUndefined(scope.$matches[index])) + if (_.isUndefined(scope.$matches[index])) return null; return scope.$matches[index].value; diff --git a/modules/web-console/frontend/app/services/AngularStrapTooltip.decorator.js b/modules/web-console/frontend/app/services/AngularStrapTooltip.decorator.js index d01a45075b794..fa59f32765111 100644 --- a/modules/web-console/frontend/app/services/AngularStrapTooltip.decorator.js +++ b/modules/web-console/frontend/app/services/AngularStrapTooltip.decorator.js @@ -16,7 +16,7 @@ */ import angular from 'angular'; -import flow from 'lodash/flow'; +import _ from 'lodash'; /** * Decorator that fix problem in AngularStrap $tooltip. @@ -62,7 +62,7 @@ export default angular scope.$emit(options.prefixEvent + '.hide.before', $tooltip); - if (angular.isDefined(options.onBeforeHide) && angular.isFunction(options.onBeforeHide)) + if (!_.isUndefined(options.onBeforeHide) && _.isFunction(options.onBeforeHide)) options.onBeforeHide($tooltip); $tooltip.$isShown = scope.$isShown = false; @@ -82,8 +82,8 @@ export default angular const $tooltip = $delegate(el, config); $tooltip.$referenceElement = el; - $tooltip.destroy = flow($tooltip.destroy, () => $tooltip.$referenceElement = null); - $tooltip.$applyPlacement = flow($tooltip.$applyPlacement, () => { + $tooltip.destroy = _.flow($tooltip.destroy, () => $tooltip.$referenceElement = null); + $tooltip.$applyPlacement = _.flow($tooltip.$applyPlacement, () => { if (!$tooltip.$element) return; diff --git a/modules/web-console/frontend/app/services/FormUtils.service.js b/modules/web-console/frontend/app/services/FormUtils.service.js index f22d4bc6c0af8..da1d73700d59b 100644 --- a/modules/web-console/frontend/app/services/FormUtils.service.js +++ b/modules/web-console/frontend/app/services/FormUtils.service.js @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +import _ from 'lodash'; export default ['IgniteFormUtils', ['$window', 'IgniteFocus', ($window, Focus) => { function ensureActivePanel(ui, pnl, focusId) { @@ -41,7 +42,7 @@ export default ['IgniteFormUtils', ['$window', 'IgniteFocus', ($window, Focus) = if (!activePanels || activePanels.length < 1) ui.activePanels = [idx]; else if (!_.includes(activePanels, idx)) { - const newActivePanels = angular.copy(activePanels); + const newActivePanels = _.cloneDeep(activePanels); newActivePanels.push(idx); From 2be10fdcde3c529b866f85a5b74ff3d471c5c9c7 Mon Sep 17 00:00:00 2001 From: Ilya Kasnacheev Date: Wed, 11 Apr 2018 19:32:52 +0700 Subject: [PATCH 0024/1463] IGNITE-8106 Collect suppressed exceptions from causes. - Fixes #3735. Signed-off-by: Alexey Kuznetsov (cherry picked from commit 98ef925) --- .../GridChangeStateCommandHandler.java | 3 +- .../ignite/internal/util/typedef/X.java | 37 +++++++++++++++---- .../visor/util/VisorExceptionWrapper.java | 11 +++--- .../tcp/TcpCommunicationSpi.java | 2 +- .../GridSuppressedExceptionSelfTest.java | 23 +++++++++++- 5 files changed, 59 insertions(+), 17 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeStateCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeStateCommandHandler.java index 7bb13d9b41e84..619be34546cd7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeStateCommandHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeStateCommandHandler.java @@ -27,6 +27,7 @@ import org.apache.ignite.internal.processors.rest.request.GridRestChangeStateRequest; import org.apache.ignite.internal.processors.rest.request.GridRestRequest; import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.SB; import org.apache.ignite.internal.util.typedef.internal.U; @@ -78,7 +79,7 @@ public GridChangeStateCommandHandler(GridKernalContext ctx) { sb.a(e.getMessage()).a("\n").a("suppressed: \n"); - for (Throwable t:e.getSuppressed()) + for (Throwable t : X.getSuppressedList(e)) sb.a(t.getMessage()).a("\n"); res.setError(sb.toString()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java b/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java index 395de237386ef..1a43daa09c918 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java @@ -469,14 +469,12 @@ public static boolean hasSuppressed(@Nullable Throwable t, @Nullable Class getSuppressedList(@Nullable Throwable t) { + List result = new ArrayList<>(); + + if (t == null) + return result; + + do { + for (Throwable suppressed : t.getSuppressed()) { + result.add(suppressed); + + result.addAll(getSuppressedList(suppressed)); + } + } while ((t = t.getCause()) != null); + + return result; + } + /** * A way to get the entire nested stack-trace of an throwable. * @@ -889,4 +910,4 @@ public static double parseDouble(@Nullable String s, double dflt) { return dflt; } } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java index 15e9557004bb2..ba52c5fd8b190 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java @@ -17,7 +17,8 @@ package org.apache.ignite.internal.visor.util; -import org.apache.ignite.internal.util.typedef.F; +import java.util.List; +import org.apache.ignite.internal.util.typedef.X; /** * Exception wrapper for safe for transferring to Visor. @@ -56,12 +57,10 @@ public VisorExceptionWrapper(Throwable cause) { if (cause.getCause() != null) initCause(new VisorExceptionWrapper(cause.getCause())); - Throwable[] suppressed = cause.getSuppressed(); + List suppressed = X.getSuppressedList(cause); - if (!F.isEmpty(suppressed)) { - for (Throwable sup : suppressed) - addSuppressed(new VisorExceptionWrapper(sup)); - } + for (Throwable sup : suppressed) + addSuppressed(new VisorExceptionWrapper(sup)); } /** 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 9e7b59235db80..df37dff1f6f10 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 @@ -3476,7 +3476,7 @@ else if (X.hasCause(e, SocketTimeoutException.class)) ctx.failNode(node.id(), "TcpCommunicationSpi failed to establish connection to node [" + "rmtNode=" + node + ", errs=" + errs + - ", connectErrs=" + Arrays.toString(errs.getSuppressed()) + ']'); + ", connectErrs=" + X.getSuppressedList(errs) + ']'); } } diff --git a/modules/core/src/test/java/org/apache/ignite/GridSuppressedExceptionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/GridSuppressedExceptionSelfTest.java index 6e32249b54f4c..55e54fb884fac 100644 --- a/modules/core/src/test/java/org/apache/ignite/GridSuppressedExceptionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/GridSuppressedExceptionSelfTest.java @@ -18,6 +18,7 @@ package org.apache.ignite; import java.io.IOException; +import java.util.List; import junit.framework.TestCase; import org.apache.ignite.internal.util.typedef.X; @@ -67,6 +68,26 @@ public void testXHasCause() throws Exception { } } + /** + * @throws Exception If failed. + */ + public void testXGetSuppressedList() throws Exception { + IgniteCheckedException me = prepareMultiException(); + + assertEquals(3, X.getSuppressedList(me).size()); + + RuntimeException e = new RuntimeException(); + e.addSuppressed(me); + + List suppresseds = X.getSuppressedList(e); + + assertEquals(4, suppresseds.size()); + + assertEquals("Test message.", suppresseds.get(0).getMessage()); + for (int i = 1; i <= 3; i++) + assertEquals("Demo exception.", suppresseds.get(1).getMessage()); + } + /** * @throws Exception If failed. */ @@ -116,4 +137,4 @@ private void generateException(int calls, Throwable cause) throws IgniteCheckedE else generateException(calls - 1, cause); } -} \ No newline at end of file +} From 7f463be9ff6c5d2d84b0aacc888159ffe68bf269 Mon Sep 17 00:00:00 2001 From: Alexander Paschenko Date: Wed, 11 Apr 2018 16:20:16 +0300 Subject: [PATCH 0025/1463] IGNITE-8204: SQL: fixed hangs when lazy flag is enabled. This closes #3785. --- .../query/h2/twostep/GridMapQueryExecutor.java | 7 +++++++ .../query/h2/twostep/MapQueryLazyWorker.java | 13 +++++++++++-- 2 files changed, 18 insertions(+), 2 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 9b1e4faa4346c..930ada22f523e 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 @@ -233,6 +233,13 @@ else if (msg instanceof GridH2DmlRequest) } } + /** + * @return Busy lock for lazy workers to guard their operations with. + */ + GridSpinBusyLock busyLock() { + return busyLock; + } + /** * @param node Node. * @param msg Message. diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorker.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorker.java index 59c050ffc2bbf..98f3df98260fc 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorker.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorker.java @@ -80,8 +80,17 @@ public MapQueryLazyWorker(@Nullable String instanceName, MapQueryLazyWorkerKey k while (!isCancelled()) { Runnable task = tasks.take(); - if (task != null) - task.run(); + if (task != null) { + if (!exec.busyLock().enterBusy()) + return; + + try { + task.run(); + } + finally { + exec.busyLock().leaveBusy(); + } + } } } finally { From 7eee6e24736d8c0958e0107471c47ef4035037d8 Mon Sep 17 00:00:00 2001 From: Alexey Kukushkin Date: Wed, 11 Apr 2018 16:29:07 +0300 Subject: [PATCH 0026/1463] IGNITE-8221: Security for thin clients. --- .../apache/ignite/IgniteSystemProperties.java | 6 +++ .../client/ClientAuthenticationException.java | 2 +- .../client/ClientAuthorizationException.java | 46 +++++++++++++++++++ .../internal/client/thin/ClientChannel.java | 3 +- .../client/thin/TcpClientChannel.java | 39 ++++++++-------- .../IgniteAuthenticationProcessor.java | 5 +- .../processors/cache/GridCacheProcessor.java | 32 +++++++++++++ .../processors/cache/GridCacheUtils.java | 5 ++ .../client/ClientConnectionContext.java | 45 +++++++++++++++++- .../platform/client/ClientRequest.java | 29 ++++++++++++ .../platform/client/ClientStatus.java | 3 ++ .../cache/ClientCacheClearKeyRequest.java | 3 ++ .../cache/ClientCacheClearKeysRequest.java | 3 ++ .../client/cache/ClientCacheClearRequest.java | 3 ++ .../cache/ClientCacheContainsKeyRequest.java | 3 ++ .../cache/ClientCacheContainsKeysRequest.java | 3 ++ ...ntCacheCreateWithConfigurationRequest.java | 6 ++- .../ClientCacheCreateWithNameRequest.java | 3 ++ .../cache/ClientCacheDestroyRequest.java | 3 ++ .../cache/ClientCacheGetAllRequest.java | 3 ++ .../ClientCacheGetAndPutIfAbsentRequest.java | 3 ++ .../cache/ClientCacheGetAndPutRequest.java | 3 ++ .../cache/ClientCacheGetAndRemoveRequest.java | 3 ++ .../ClientCacheGetAndReplaceRequest.java | 3 ++ ...heGetOrCreateWithConfigurationRequest.java | 6 ++- ...ClientCacheGetOrCreateWithNameRequest.java | 3 ++ .../client/cache/ClientCacheGetRequest.java | 3 ++ .../cache/ClientCacheGetSizeRequest.java | 3 ++ .../cache/ClientCachePutAllRequest.java | 3 ++ .../cache/ClientCachePutIfAbsentRequest.java | 3 ++ .../client/cache/ClientCachePutRequest.java | 3 ++ .../cache/ClientCacheRemoveAllRequest.java | 3 ++ .../ClientCacheRemoveIfEqualsRequest.java | 3 ++ .../cache/ClientCacheRemoveKeyRequest.java | 3 ++ .../cache/ClientCacheRemoveKeysRequest.java | 3 ++ .../ClientCacheReplaceIfEqualsRequest.java | 3 ++ .../cache/ClientCacheReplaceRequest.java | 3 ++ .../client/cache/ClientCacheRequest.java | 32 +++++++++++++ .../cache/ClientCacheScanQueryRequest.java | 3 ++ .../ClientCacheSqlFieldsQueryRequest.java | 1 + .../cache/ClientCacheSqlQueryRequest.java | 1 + .../security/AuthenticationContext.java | 40 ++++++++++++++++ .../plugin/security/SecurityPermission.java | 11 ++++- .../ignite/spi/discovery/tcp/ServerImpl.java | 12 ++++- 44 files changed, 371 insertions(+), 28 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/client/ClientAuthorizationException.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 9da123e01fb35..ac7947b7e45aa 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -834,6 +834,12 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_WAL_FSYNC_WITH_DEDICATED_WORKER = "IGNITE_WAL_FSYNC_WITH_DEDICATED_WORKER"; + /** + * When set to {@code true}, on-heap cache cannot be enabled - see + * {@link CacheConfiguration#setOnheapCacheEnabled(boolean)}. + * Default is {@code false}. + */ + public static final String IGNITE_DISABLE_ONHEAP_CACHE = "IGNITE_DISABLE_ONHEAP_CACHE"; /** * When set to {@code false}, loaded pages implementation is switched to previous version of implementation, * FullPageIdTable. {@code True} value enables 'Robin Hood hashing: backward shift deletion'. diff --git a/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java b/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java index dc39c7a0ab780..0c24db8880a4f 100644 --- a/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java +++ b/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java @@ -18,7 +18,7 @@ package org.apache.ignite.client; /** - * Indicates Ignite server the client is connected to closed the connection and no longer available. + * Indicates user name or password is invalid. */ public class ClientAuthenticationException extends ClientException { /** Serial version uid. */ diff --git a/modules/core/src/main/java/org/apache/ignite/client/ClientAuthorizationException.java b/modules/core/src/main/java/org/apache/ignite/client/ClientAuthorizationException.java new file mode 100644 index 0000000000000..cacede67bb356 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/client/ClientAuthorizationException.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.client; + +/** + * Indicates user has no permission to perform operation. + */ +public class ClientAuthorizationException extends ClientException { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Message. */ + private static final String MSG = "User is not authorized to perform this operation"; + + /** + * Default constructor. + */ + public ClientAuthorizationException() { + super(MSG); + } + + /** + * Constructs a new exception with the specified cause and a detail + * message of (cause==null ? null : cause.toString()). + * + * @param cause the cause. + */ + public ClientAuthorizationException(Throwable cause) { + super(MSG, cause); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientChannel.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientChannel.java index 71502a4760483..eb62c808225f9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientChannel.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientChannel.java @@ -22,6 +22,7 @@ import org.apache.ignite.internal.binary.streams.BinaryInputStream; import org.apache.ignite.internal.binary.streams.BinaryOutputStream; import org.apache.ignite.client.ClientConnectionException; +import org.apache.ignite.client.ClientAuthorizationException; /** * Processing thin client requests and responses. @@ -41,5 +42,5 @@ interface ClientChannel extends AutoCloseable { * @return Received operation payload or {@code null} if response has no payload. */ public T receive(ClientOperation op, long reqId, Function payloadReader) - throws ClientConnectionException; + throws ClientConnectionException, ClientAuthorizationException; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java index 404793a83f338..8e8294f1ef6f2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java @@ -50,6 +50,7 @@ import javax.net.ssl.TrustManagerFactory; import javax.net.ssl.X509TrustManager; import org.apache.ignite.client.ClientAuthenticationException; +import org.apache.ignite.client.ClientAuthorizationException; import org.apache.ignite.client.ClientConnectionException; import org.apache.ignite.client.SslMode; import org.apache.ignite.client.SslProtocol; @@ -62,6 +63,7 @@ import org.apache.ignite.internal.binary.streams.BinaryInputStream; import org.apache.ignite.internal.binary.streams.BinaryOffheapOutputStream; import org.apache.ignite.internal.binary.streams.BinaryOutputStream; +import org.apache.ignite.internal.processors.platform.client.ClientStatus; /** * Implements {@link ClientChannel} over TCP. @@ -138,7 +140,8 @@ class TcpClientChannel implements ClientChannel { /** {@inheritDoc} */ public T receive(ClientOperation op, long reqId, Function payloadReader) - throws ClientConnectionException { + throws ClientConnectionException, ClientAuthorizationException { + final int MIN_RES_SIZE = 8 + 4; // minimal response size: long (8 bytes) ID + int (4 bytes) status int resSize = new BinaryHeapInputStream(read(4)).readInt(); @@ -163,7 +166,12 @@ public T receive(ClientOperation op, long reqId, Function(); + else + users.clear(); for (User u : initUsrs.usrs) users.put(u.name(), u); 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 7edac7374fbe9..3aa6603a47d96 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 @@ -149,6 +149,8 @@ import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.apache.ignite.mxbean.CacheGroupMetricsMXBean; import org.apache.ignite.mxbean.IgniteMBeanAware; +import org.apache.ignite.plugin.security.SecurityException; +import org.apache.ignite.plugin.security.SecurityPermission; import org.apache.ignite.spi.IgniteNodeValidationResult; import org.apache.ignite.spi.discovery.DiscoveryDataBag; import org.apache.ignite.spi.discovery.DiscoveryDataBag.GridDiscoveryData; @@ -1126,6 +1128,9 @@ private void startCache(GridCacheAdapter cache, QuerySchema schema) throws CacheConfiguration cfg = cacheCtx.config(); + if (cacheCtx.userCache()) + authorizeCacheCreate(cacheCtx.name(), cfg); + // Intentionally compare Boolean references using '!=' below to check if the flag has been explicitly set. if (cfg.isStoreKeepBinary() && cfg.isStoreKeepBinary() != CacheConfiguration.DFLT_STORE_KEEP_BINARY && !(ctx.config().getMarshaller() instanceof BinaryMarshaller)) @@ -3151,6 +3156,8 @@ private Collection initiateCacheChanges( Collection sndReqs = new ArrayList<>(reqs.size()); for (DynamicCacheChangeRequest req : reqs) { + authorizeCacheChange(req); + DynamicCacheStartFuture fut = new DynamicCacheStartFuture(req.requestId()); try { @@ -3215,6 +3222,31 @@ private Collection initiateCacheChanges( return res; } + /** + * Authorize dynamic cache management. + */ + private void authorizeCacheChange(DynamicCacheChangeRequest req) { + if (req.cacheType() == null || req.cacheType() == CacheType.USER) { + if (req.stop()) + ctx.security().authorize(req.cacheName(), SecurityPermission.CACHE_DESTROY, null); + else + authorizeCacheCreate(req.cacheName(), req.startCacheConfiguration()); + } + } + + /** + * Authorize start/create cache operation. + */ + private void authorizeCacheCreate(String cacheName, CacheConfiguration cacheCfg) { + ctx.security().authorize(cacheName, SecurityPermission.CACHE_CREATE, null); + + if (cacheCfg != null && cacheCfg.isOnheapCacheEnabled() && + System.getProperty(IgniteSystemProperties.IGNITE_DISABLE_ONHEAP_CACHE, "false") + .toUpperCase().equals("TRUE") + ) + throw new SecurityException("Authorization failed for enabling on-heap cache."); + } + /** * @return Non null exception if node is stopping or disconnected. */ 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 d672420fafdc8..e244c75ad4a0e 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 @@ -98,6 +98,8 @@ import org.apache.ignite.lang.IgniteReducer; 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; @@ -1290,6 +1292,9 @@ else if (e instanceof SchemaOperationException) if (e.getCause() instanceof NullPointerException) return (NullPointerException)e.getCause(); + if (e.getCause() instanceof SecurityException) + return (SecurityException)e.getCause(); + C1 converter = U.getExceptionConverter(e.getClass()); return converter != null ? new CacheException(converter.apply(e)) : new CacheException(e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java index 7ab2d33e5dbc0..061aab32c0459 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java @@ -20,16 +20,24 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; +import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.processors.authentication.AuthorizationContext; +import org.apache.ignite.internal.processors.authentication.IgniteAccessControlException; import org.apache.ignite.internal.processors.odbc.ClientListenerConnectionContext; 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 java.util.concurrent.atomic.AtomicLong; +import org.apache.ignite.internal.processors.security.SecurityContext; +import org.apache.ignite.plugin.security.AuthenticationContext; +import org.apache.ignite.plugin.security.SecurityCredentials; + +import static org.apache.ignite.plugin.security.SecuritySubjectType.REMOTE_CLIENT; /** * Thin Client connection context. @@ -62,6 +70,9 @@ public class ClientConnectionContext implements ClientListenerConnectionContext /** Cursor counter. */ private final AtomicLong curCnt = new AtomicLong(); + /** Security context or {@code null} if security is disabled. */ + private SecurityContext secCtx = null; + /** * Ctor. * @@ -129,7 +140,9 @@ public GridKernalContext kernalContext() { } } - if (kernalCtx.authentication().enabled()) { + if (kernalCtx.security().enabled()) + authCtx = thirdPartyAuthentication(user, pwd).authorizationContext(); + else if (kernalCtx.authentication().enabled()) { if (user == null || user.length() == 0) throw new IgniteCheckedException("Unauthenticated sessions are prohibited."); @@ -179,4 +192,34 @@ public void incrementCursors() { public void decrementCursors() { curCnt.decrementAndGet(); } + + /** + * @return Security context or {@code null} if security is disabled. + */ + public SecurityContext securityContext() { + return secCtx; + } + + /** + * Do 3-rd party authentication. + */ + private AuthenticationContext thirdPartyAuthentication(String user, String pwd) throws IgniteCheckedException { + SecurityCredentials cred = new SecurityCredentials(user, pwd); + + AuthenticationContext authCtx = new AuthenticationContext(); + + authCtx.subjectType(REMOTE_CLIENT); + authCtx.subjectId(UUID.randomUUID()); + authCtx.nodeAttributes(Collections.emptyMap()); + authCtx.credentials(cred); + + secCtx = kernalCtx.security().authenticate(authCtx); + + if (secCtx == null) + throw new IgniteAccessControlException( + String.format("The user name or password is incorrect [userName=%s]", user) + ); + + return authCtx; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequest.java index 76823b592ce32..799b3e733f410 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequest.java @@ -19,6 +19,9 @@ import org.apache.ignite.binary.BinaryRawReader; import org.apache.ignite.internal.processors.odbc.ClientListenerRequest; +import org.apache.ignite.internal.processors.security.SecurityContext; +import org.apache.ignite.plugin.security.SecurityException; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Thin client request. @@ -58,4 +61,30 @@ public ClientRequest(long reqId) { public ClientResponse process(ClientConnectionContext ctx) { return new ClientResponse(reqId); } + + /** + * Run the code with converting {@link SecurityException} to {@link IgniteClientException}. + */ + protected static void runWithSecurityExceptionHandler(Runnable runnable) { + try { + runnable.run(); + } + catch (SecurityException ex) { + throw new IgniteClientException( + ClientStatus.SECURITY_VIOLATION, + "Client is not authorized to perform this operation", + ex + ); + } + } + + /** + * Authorize for specified permission. + */ + protected void authorize(ClientConnectionContext ctx, SecurityPermission perm) { + SecurityContext secCtx = ctx.securityContext(); + + if (secCtx != null) + runWithSecurityExceptionHandler(() -> ctx.kernalContext().security().authorize(null, perm, secCtx)); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java index e0049b472a18f..b8dfb1fa64b9e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java @@ -48,4 +48,7 @@ private ClientStatus (){ /** Resource does not exist. */ public static final int RESOURCE_DOES_NOT_EXIST = 1011; + + /** Resource does not exist. */ + public static final int SECURITY_VIOLATION = 1012; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearKeyRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearKeyRequest.java index 6bcbbe89b2636..5f8e952234bf8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearKeyRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearKeyRequest.java @@ -20,6 +20,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx; import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Clear key request. @@ -37,6 +38,8 @@ public ClientCacheClearKeyRequest(BinaryRawReaderEx reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_REMOVE); + cache(ctx).clear(key()); return super.process(ctx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearKeysRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearKeysRequest.java index 04eb7f60c9502..d803f697420f3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearKeysRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearKeysRequest.java @@ -20,6 +20,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx; import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Clear keys request. @@ -37,6 +38,8 @@ public ClientCacheClearKeysRequest(BinaryRawReaderEx reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_REMOVE); + cache(ctx).clearAll(keys()); return super.process(ctx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearRequest.java index 0e5f20de1eb1b..7b84522921c7c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearRequest.java @@ -20,6 +20,7 @@ import org.apache.ignite.binary.BinaryRawReader; import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Cache clear request. @@ -37,6 +38,8 @@ public ClientCacheClearRequest(BinaryRawReader reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_REMOVE); + cache(ctx).clear(); return super.process(ctx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheContainsKeyRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheContainsKeyRequest.java index 8470828e424a1..386f448bb4b53 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheContainsKeyRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheContainsKeyRequest.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientBooleanResponse; import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * ContainsKey request. @@ -38,6 +39,8 @@ public ClientCacheContainsKeyRequest(BinaryRawReaderEx reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_READ); + boolean val = cache(ctx).containsKey(key()); return new ClientBooleanResponse(requestId(), val); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheContainsKeysRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheContainsKeysRequest.java index 41e13068db1f5..b5184bfc1afc9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheContainsKeysRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheContainsKeysRequest.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientBooleanResponse; import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * ContainsKeys request. @@ -38,6 +39,8 @@ public ClientCacheContainsKeysRequest(BinaryRawReaderEx reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_READ); + boolean val = cache(ctx).containsKeys(keys()); return new ClientBooleanResponse(requestId(), val); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithConfigurationRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithConfigurationRequest.java index 4b4dcece7c2a6..65f97841b9ad3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithConfigurationRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithConfigurationRequest.java @@ -25,6 +25,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientResponse; import org.apache.ignite.internal.processors.platform.client.ClientStatus; import org.apache.ignite.internal.processors.platform.client.IgniteClientException; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Cache create with configuration request. @@ -47,8 +48,11 @@ public ClientCacheCreateWithConfigurationRequest(BinaryRawReader reader) { /** {@inheritDoc} */ @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_CREATE); + try { - ctx.kernalContext().grid().createCache(cacheCfg); + // Use security exception handler since the code authorizes "enable on-heap cache" permission + runWithSecurityExceptionHandler(() -> ctx.kernalContext().grid().createCache(cacheCfg)); } catch (CacheExistsException e) { throw new IgniteClientException(ClientStatus.CACHE_EXISTS, e.getMessage()); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithNameRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithNameRequest.java index 9155d76bfee00..cacf099b4f81f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithNameRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithNameRequest.java @@ -24,6 +24,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientResponse; import org.apache.ignite.internal.processors.platform.client.ClientStatus; import org.apache.ignite.internal.processors.platform.client.IgniteClientException; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Cache create with name request. @@ -45,6 +46,8 @@ public ClientCacheCreateWithNameRequest(BinaryRawReader reader) { /** {@inheritDoc} */ @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_CREATE); + try { ctx.kernalContext().grid().createCache(cacheName); } catch (CacheExistsException e) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheDestroyRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheDestroyRequest.java index 6645a03a06b4f..b6f85eec3d9fc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheDestroyRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheDestroyRequest.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientRequest; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Cache destroy request. @@ -42,6 +43,8 @@ public ClientCacheDestroyRequest(BinaryRawReader reader) { /** {@inheritDoc} */ @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_DESTROY); + String cacheName = ClientCacheRequest.cacheDescriptor(ctx, cacheId).cacheName(); ctx.kernalContext().grid().destroyCache(cacheName); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAllRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAllRequest.java index 2b33af1cb69f8..a07305c4ce14b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAllRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAllRequest.java @@ -22,6 +22,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientResponse; import java.util.Map; +import org.apache.ignite.plugin.security.SecurityPermission; /** * GetAll request. @@ -39,6 +40,8 @@ public ClientCacheGetAllRequest(BinaryRawReaderEx reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_READ); + Map val = cache(ctx).getAll(keys()); return new ClientCacheGetAllResponse(requestId(), val); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndPutIfAbsentRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndPutIfAbsentRequest.java index 836021313c5fc..8713a211bb4eb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndPutIfAbsentRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndPutIfAbsentRequest.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientObjectResponse; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Cache get and put if absent request. @@ -38,6 +39,8 @@ public ClientCacheGetAndPutIfAbsentRequest(BinaryRawReaderEx reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_READ, SecurityPermission.CACHE_PUT); + Object res = cache(ctx).getAndPutIfAbsent(key(), val()); return new ClientObjectResponse(requestId(), res); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndPutRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndPutRequest.java index 7a540e8473ac9..dde5181303cef 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndPutRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndPutRequest.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientObjectResponse; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Cache get and put request. @@ -38,6 +39,8 @@ public ClientCacheGetAndPutRequest(BinaryRawReaderEx reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_READ, SecurityPermission.CACHE_PUT); + Object res = cache(ctx).getAndPut(key(), val()); return new ClientObjectResponse(requestId(), res); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndRemoveRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndRemoveRequest.java index e4fd735b186ac..3b9dd4bab88c3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndRemoveRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndRemoveRequest.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientObjectResponse; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Cache get and remove request. @@ -38,6 +39,8 @@ public ClientCacheGetAndRemoveRequest(BinaryRawReaderEx reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_READ, SecurityPermission.CACHE_REMOVE); + Object val = cache(ctx).getAndRemove(key()); return new ClientObjectResponse(requestId(), val); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndReplaceRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndReplaceRequest.java index dba8639e4c07a..8ba157a762c9c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndReplaceRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndReplaceRequest.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientObjectResponse; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Cache get and replace request. @@ -38,6 +39,8 @@ public ClientCacheGetAndReplaceRequest(BinaryRawReaderEx reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_READ, SecurityPermission.CACHE_PUT); + Object res = cache(ctx).getAndReplace(key(), val()); return new ClientObjectResponse(requestId(), res); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithConfigurationRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithConfigurationRequest.java index 267318a3e3349..48569b447e156 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithConfigurationRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithConfigurationRequest.java @@ -25,6 +25,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientResponse; import org.apache.ignite.internal.processors.platform.client.ClientStatus; import org.apache.ignite.internal.processors.platform.client.IgniteClientException; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Cache get or create with configuration request. @@ -47,8 +48,11 @@ public ClientCacheGetOrCreateWithConfigurationRequest(BinaryRawReader reader) { /** {@inheritDoc} */ @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_CREATE); + try { - ctx.kernalContext().grid().getOrCreateCache(cacheCfg); + // Use security exception handler since the code authorizes "enable on-heap cache" permission + runWithSecurityExceptionHandler(() -> ctx.kernalContext().grid().getOrCreateCache(cacheCfg)); } catch (CacheExistsException e) { throw new IgniteClientException(ClientStatus.CACHE_EXISTS, e.getMessage()); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithNameRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithNameRequest.java index 94dd115d6075f..3c4ce7b06a694 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithNameRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithNameRequest.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientRequest; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Cache create with name request. @@ -42,6 +43,8 @@ public ClientCacheGetOrCreateWithNameRequest(BinaryRawReader reader) { /** {@inheritDoc} */ @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_CREATE); + ctx.kernalContext().grid().getOrCreateCache(cacheName); return super.process(ctx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetRequest.java index 41558c2863d03..dc17cbfbce548 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetRequest.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientObjectResponse; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Cache get request. @@ -38,6 +39,8 @@ public ClientCacheGetRequest(BinaryRawReaderEx reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_READ); + Object val = cache(ctx).get(key()); return new ClientObjectResponse(requestId(), val); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetSizeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetSizeRequest.java index ba185bf7415d8..474c206b8cae3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetSizeRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetSizeRequest.java @@ -22,6 +22,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientLongResponse; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Cache size request. @@ -50,6 +51,8 @@ public ClientCacheGetSizeRequest(BinaryRawReader reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_READ); + long res = cache(ctx).sizeLong(modes); return new ClientLongResponse(requestId(), res); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutAllRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutAllRequest.java index 28a7fa57e3ee5..57e31443b474a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutAllRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutAllRequest.java @@ -23,6 +23,7 @@ import java.util.LinkedHashMap; import java.util.Map; +import org.apache.ignite.plugin.security.SecurityPermission; /** * PutAll request. @@ -50,6 +51,8 @@ public ClientCachePutAllRequest(BinaryRawReaderEx reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_PUT); + cache(ctx).putAll(map); return super.process(ctx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutIfAbsentRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutIfAbsentRequest.java index 4dd2cde58ce06..ec81bc0c0fe48 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutIfAbsentRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutIfAbsentRequest.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientBooleanResponse; import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Cache put if absent request. @@ -38,6 +39,8 @@ public ClientCachePutIfAbsentRequest(BinaryRawReaderEx reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_READ, SecurityPermission.CACHE_PUT); + boolean res = cache(ctx).putIfAbsent(key(), val()); return new ClientBooleanResponse(requestId(), res); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutRequest.java index 2c396b7ede87a..116460eece965 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutRequest.java @@ -20,6 +20,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx; import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Cache put request. @@ -37,6 +38,8 @@ public ClientCachePutRequest(BinaryRawReaderEx reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_PUT); + cache(ctx).put(key(), val()); return super.process(ctx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveAllRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveAllRequest.java index f5adc6378912e..d90d873968105 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveAllRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveAllRequest.java @@ -20,6 +20,7 @@ import org.apache.ignite.binary.BinaryRawReader; import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Cache removeAll request. @@ -37,6 +38,8 @@ public ClientCacheRemoveAllRequest(BinaryRawReader reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_REMOVE); + cache(ctx).removeAll(); return super.process(ctx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveIfEqualsRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveIfEqualsRequest.java index b86f2f8895d64..26c191f5b5553 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveIfEqualsRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveIfEqualsRequest.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientBooleanResponse; import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Cache remove request with value. @@ -38,6 +39,8 @@ public ClientCacheRemoveIfEqualsRequest(BinaryRawReaderEx reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_READ, SecurityPermission.CACHE_REMOVE); + boolean res = cache(ctx).remove(key(), val()); return new ClientBooleanResponse(requestId(), res); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveKeyRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveKeyRequest.java index a68c32730f4fe..5af9743b3cac3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveKeyRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveKeyRequest.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientBooleanResponse; import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Remove request. @@ -38,6 +39,8 @@ public ClientCacheRemoveKeyRequest(BinaryRawReaderEx reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_REMOVE); + boolean val = cache(ctx).remove(key()); return new ClientBooleanResponse(requestId(), val); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveKeysRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveKeysRequest.java index 043b5688a3f43..62dea00201af8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveKeysRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveKeysRequest.java @@ -20,6 +20,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx; import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Remove keys request. @@ -37,6 +38,8 @@ public ClientCacheRemoveKeysRequest(BinaryRawReaderEx reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_REMOVE); + cache(ctx).removeAll(keys()); return super.process(ctx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheReplaceIfEqualsRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheReplaceIfEqualsRequest.java index 8645fbb817322..056367d71d2a9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheReplaceIfEqualsRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheReplaceIfEqualsRequest.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientBooleanResponse; import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Cache replace request. @@ -43,6 +44,8 @@ public ClientCacheReplaceIfEqualsRequest(BinaryRawReaderEx reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_READ, SecurityPermission.CACHE_PUT); + boolean res = cache(ctx).replace(key(), val(), newVal); return new ClientBooleanResponse(requestId(), res); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheReplaceRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheReplaceRequest.java index bd7a642bb39e0..ea04593e7cf15 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheReplaceRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheReplaceRequest.java @@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientBooleanResponse; import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Cache replace request. @@ -38,6 +39,8 @@ public ClientCacheReplaceRequest(BinaryRawReaderEx reader) { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_READ, SecurityPermission.CACHE_PUT); + boolean res = cache(ctx).replace(key(), val()); return new ClientBooleanResponse(requestId(), res); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRequest.java index 52b799f345120..9e2d1f1f29298 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRequest.java @@ -24,6 +24,8 @@ import org.apache.ignite.internal.processors.platform.client.ClientRequest; import org.apache.ignite.internal.processors.platform.client.ClientStatus; import org.apache.ignite.internal.processors.platform.client.IgniteClientException; +import org.apache.ignite.internal.processors.security.SecurityContext; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Cache get request. @@ -119,4 +121,34 @@ public static DynamicCacheDescriptor cacheDescriptor(ClientConnectionContext ctx protected int cacheId() { return cacheId; } + + /** {@inheritDoc} */ + protected void authorize(ClientConnectionContext ctx, SecurityPermission perm) { + SecurityContext secCtx = ctx.securityContext(); + + if (secCtx != null) { + DynamicCacheDescriptor cacheDesc = cacheDescriptor(ctx, cacheId); + + runWithSecurityExceptionHandler(() -> { + ctx.kernalContext().security().authorize(cacheDesc.cacheName(), perm, secCtx); + }); + } + } + + /** + * Authorize for multiple permissions. + */ + protected void authorize(ClientConnectionContext ctx, SecurityPermission... perm) + throws IgniteClientException { + SecurityContext secCtx = ctx.securityContext(); + + if (secCtx != null) { + DynamicCacheDescriptor cacheDesc = cacheDescriptor(ctx, cacheId); + + runWithSecurityExceptionHandler(() -> { + for (SecurityPermission p : perm) + ctx.kernalContext().security().authorize(cacheDesc.cacheName(), p, secCtx); + }); + } + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheScanQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheScanQueryRequest.java index 26ab236e8be1e..70b6966e999c4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheScanQueryRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheScanQueryRequest.java @@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientResponse; import org.apache.ignite.internal.processors.platform.utils.PlatformUtils; import org.apache.ignite.lang.IgniteBiPredicate; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Scan query request. @@ -80,6 +81,8 @@ public ClientCacheScanQueryRequest(BinaryRawReaderEx reader) { /** {@inheritDoc} */ @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_READ); + IgniteCache cache = filterPlatform == FILTER_PLATFORM_JAVA && !isKeepBinary() ? rawCache(ctx) : cache(ctx); ScanQuery qry = new ScanQuery() diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlFieldsQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlFieldsQueryRequest.java index cfd4498cb4852..3aa95bf2a1a37 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlFieldsQueryRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlFieldsQueryRequest.java @@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientResponse; import org.apache.ignite.internal.processors.query.QueryUtils; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Sql query request. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlQueryRequest.java index 8c21be14a800a..40693e74bd5d6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlQueryRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlQueryRequest.java @@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientResponse; import java.util.concurrent.TimeUnit; +import org.apache.ignite.plugin.security.SecurityPermission; /** * Sql query request. diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/AuthenticationContext.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/AuthenticationContext.java index 91f337929af50..9f2cfe2ddd4ad 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/security/AuthenticationContext.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/AuthenticationContext.java @@ -21,6 +21,7 @@ import java.util.Collections; import java.util.Map; import java.util.UUID; +import org.apache.ignite.internal.processors.authentication.AuthorizationContext; /** * Authentication context. @@ -41,6 +42,12 @@ public class AuthenticationContext { /** */ private Map nodeAttrs; + /** Authorization context. */ + private AuthorizationContext athrCtx; + + /** True if this is a client node context. */ + private boolean client; + /** * Gets subject type. * @@ -130,4 +137,37 @@ public Map nodeAttributes() { public void nodeAttributes(Map nodeAttrs) { this.nodeAttrs = nodeAttrs; } + + /** + * @return Native Apache Ignite authorization context acquired after authentication or {@code null} if native + * Ignite authentication is not used. + */ + public AuthorizationContext authorizationContext(){ + return athrCtx; + } + + /** + * Set authorization context acquired after native Apache Ignite authentication. + */ + public AuthenticationContext authorizationContext(AuthorizationContext newVal) { + athrCtx = newVal; + + return this; + } + + /** + * @return {@code true} if this is a client node context. + */ + public boolean isClient() { + return client; + } + + /** + * Sets flag indicating if this is client node context. + */ + public AuthenticationContext setClient(boolean newVal) { + client = newVal; + + return this; + } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java index 54361614e62a4..bca667ddb3d7a 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java @@ -64,7 +64,16 @@ public enum SecurityPermission { SERVICE_CANCEL, /** Service invoke permission. */ - SERVICE_INVOKE; + SERVICE_INVOKE, + + /** Cache create permission. */ + CACHE_CREATE, + + /** Cache create permission. */ + CACHE_DESTROY, + + /** Join as server node permission. */ + JOIN_AS_SERVER; /** Enumerated values. */ private static final SecurityPermission[] VALS = values(); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 7bf37e1b0ac43..6d3864e704215 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -99,6 +99,7 @@ import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.security.SecurityCredentials; +import org.apache.ignite.plugin.security.SecurityPermission; import org.apache.ignite.plugin.security.SecurityPermissionSet; import org.apache.ignite.spi.IgniteNodeValidationResult; import org.apache.ignite.spi.IgniteSpiContext; @@ -3559,6 +3560,8 @@ else if (log.isDebugEnabled()) return; } else { + String authFailedMsg = null; + if (!(subj instanceof Serializable)) { // Node has not pass authentication. LT.warn(log, "Authentication subject is not Serializable [nodeId=" + node.id() + @@ -3567,9 +3570,16 @@ else if (log.isDebugEnabled()) ", addrs=" + U.addressesAsString(node) + ']'); + authFailedMsg = "Authentication subject is not serializable"; + } + else if (!node.isClient() && + !subj.systemOperationAllowed(SecurityPermission.JOIN_AS_SERVER)) + authFailedMsg = "Node is not authorised to join as a server node"; + + if (authFailedMsg != null) { // Always output in debug. if (log.isDebugEnabled()) - log.debug("Authentication subject is not serializable [nodeId=" + node.id() + + log.debug(authFailedMsg + " [nodeId=" + node.id() + ", addrs=" + U.addressesAsString(node)); try { From b6ad3705c1e68683b72d2237037af66fea23a7ae Mon Sep 17 00:00:00 2001 From: devozerov Date: Wed, 11 Apr 2018 16:44:33 +0300 Subject: [PATCH 0027/1463] IGNITE-8148: JDBC thin: semicolon as delimiter for properties. This closes #3794. --- .../jdbc/thin/JdbcThinConnectionSelfTest.java | 233 ++++++++++++++---- .../jdbc/thin/ConnectionPropertiesImpl.java | 161 ++++++++---- 2 files changed, 300 insertions(+), 94 deletions(-) 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 14b91b260c9fe..ed0b32403db3a 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 @@ -184,6 +184,38 @@ public void testSocketBuffers() throws Exception { } } + /** + * Test invalid socket buffer sizes with semicolon. + * + * @throws Exception If failed. + */ + public void testSocketBuffersSemicolon() throws Exception { + final int dfltDufSize = 64 * 1024; + + assertInvalid("jdbc:ignite:thin://127.0.0.1;socketSendBuffer=-1", + "Property cannot be lower than 0 [name=socketSendBuffer, value=-1]"); + + assertInvalid("jdbc:ignite:thin://127.0.0.1;socketReceiveBuffer=-1", + "Property cannot be lower than 0 [name=socketReceiveBuffer, value=-1]"); + + // Note that SO_* options are hints, so we check that value is equals to either what we set or to default. + try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;socketSendBuffer=1024")) { + assertEquals(1024, io(conn).connectionProperties().getSocketSendBuffer()); + assertEquals(dfltDufSize, io(conn).connectionProperties().getSocketReceiveBuffer()); + } + + try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;socketReceiveBuffer=1024")) { + assertEquals(dfltDufSize, io(conn).connectionProperties().getSocketSendBuffer()); + assertEquals(1024, io(conn).connectionProperties().getSocketReceiveBuffer()); + } + + try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;" + + "socketSendBuffer=1024;socketReceiveBuffer=2048")) { + assertEquals(1024, io(conn).connectionProperties().getSocketSendBuffer()); + assertEquals(2048, io(conn).connectionProperties().getSocketReceiveBuffer()); + } + } + /** * Test SQL hints. * @@ -191,79 +223,97 @@ public void testSocketBuffers() throws Exception { */ public void testSqlHints() throws Exception { try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1")) { - assertFalse(io(conn).connectionProperties().isDistributedJoins()); - assertFalse(io(conn).connectionProperties().isEnforceJoinOrder()); - assertFalse(io(conn).connectionProperties().isCollocated()); - assertFalse(io(conn).connectionProperties().isReplicatedOnly()); - assertFalse(io(conn).connectionProperties().isLazy()); - assertFalse(io(conn).connectionProperties().isSkipReducerOnUpdate()); + assertHints(conn, false, false, false, false, false, false); } try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?distributedJoins=true")) { - assertTrue(io(conn).connectionProperties().isDistributedJoins()); - assertFalse(io(conn).connectionProperties().isEnforceJoinOrder()); - assertFalse(io(conn).connectionProperties().isCollocated()); - assertFalse(io(conn).connectionProperties().isReplicatedOnly()); - assertFalse(io(conn).connectionProperties().isLazy()); - assertFalse(io(conn).connectionProperties().isSkipReducerOnUpdate()); + assertHints(conn, true, false, false, false, false, false); } try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?enforceJoinOrder=true")) { - assertFalse(io(conn).connectionProperties().isDistributedJoins()); - assertTrue(io(conn).connectionProperties().isEnforceJoinOrder()); - assertFalse(io(conn).connectionProperties().isCollocated()); - assertFalse(io(conn).connectionProperties().isReplicatedOnly()); - assertFalse(io(conn).connectionProperties().isLazy()); - assertFalse(io(conn).connectionProperties().isSkipReducerOnUpdate()); + assertHints(conn, false, true, false, false, false, false); } try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?collocated=true")) { - assertFalse(io(conn).connectionProperties().isDistributedJoins()); - assertFalse(io(conn).connectionProperties().isEnforceJoinOrder()); - assertTrue(io(conn).connectionProperties().isCollocated()); - assertFalse(io(conn).connectionProperties().isReplicatedOnly()); - assertFalse(io(conn).connectionProperties().isLazy()); - assertFalse(io(conn).connectionProperties().isSkipReducerOnUpdate()); + assertHints(conn, false, false, true, false, false, false); } try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?replicatedOnly=true")) { - assertFalse(io(conn).connectionProperties().isDistributedJoins()); - assertFalse(io(conn).connectionProperties().isEnforceJoinOrder()); - assertFalse(io(conn).connectionProperties().isCollocated()); - assertTrue(io(conn).connectionProperties().isReplicatedOnly()); - assertFalse(io(conn).connectionProperties().isLazy()); - assertFalse(io(conn).connectionProperties().isSkipReducerOnUpdate()); + assertHints(conn, false, false, false, true, false, false); } try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?lazy=true")) { - assertFalse(io(conn).connectionProperties().isDistributedJoins()); - assertFalse(io(conn).connectionProperties().isEnforceJoinOrder()); - assertFalse(io(conn).connectionProperties().isCollocated()); - assertFalse(io(conn).connectionProperties().isReplicatedOnly()); - assertTrue(io(conn).connectionProperties().isLazy()); - assertFalse(io(conn).connectionProperties().isSkipReducerOnUpdate()); + assertHints(conn, false, false, false, false, true, false); } try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?skipReducerOnUpdate=true")) { - assertFalse(io(conn).connectionProperties().isDistributedJoins()); - assertFalse(io(conn).connectionProperties().isEnforceJoinOrder()); - assertFalse(io(conn).connectionProperties().isCollocated()); - assertFalse(io(conn).connectionProperties().isReplicatedOnly()); - assertFalse(io(conn).connectionProperties().isLazy()); - assertTrue(io(conn).connectionProperties().isSkipReducerOnUpdate()); + assertHints(conn, false, false, false, false, false, true); } try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?distributedJoins=true&" + "enforceJoinOrder=true&collocated=true&replicatedOnly=true&lazy=true&skipReducerOnUpdate=true")) { - assertTrue(io(conn).connectionProperties().isDistributedJoins()); - assertTrue(io(conn).connectionProperties().isEnforceJoinOrder()); - assertTrue(io(conn).connectionProperties().isCollocated()); - assertTrue(io(conn).connectionProperties().isReplicatedOnly()); - assertTrue(io(conn).connectionProperties().isLazy()); - assertTrue(io(conn).connectionProperties().isSkipReducerOnUpdate()); + assertHints(conn, true, true, true, true, true, true); } } + /** + * Test SQL hints with semicolon. + * + * @throws Exception If failed. + */ + public void testSqlHintsSemicolon() throws Exception { + try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;distributedJoins=true")) { + assertHints(conn, true, false, false, false, false, false); + } + + try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;enforceJoinOrder=true")) { + assertHints(conn, false, true, false, false, false, false); + } + + try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;collocated=true")) { + assertHints(conn, false, false, true, false, false, false); + } + + try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;replicatedOnly=true")) { + assertHints(conn, false, false, false, true, false, false); + } + + try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;lazy=true")) { + assertHints(conn, false, false, false, false, true, false); + } + + try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;skipReducerOnUpdate=true")) { + assertHints(conn, false, false, false, false, false, true); + } + + try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;distributedJoins=true;" + + "enforceJoinOrder=true;collocated=true;replicatedOnly=true;lazy=true;skipReducerOnUpdate=true")) { + assertHints(conn, true, true, true, true, true, true); + } + } + + /** + * Assert hints. + * + * @param conn Connection. + * @param distributedJoins Distributed joins. + * @param enforceJoinOrder Enforce join order. + * @param collocated Co-located. + * @param replicatedOnly Replicated only. + * @param lazy Lazy. + * @param skipReducerOnUpdate Skip reducer on update. + * @throws Exception If failed. + */ + private void assertHints(Connection conn, boolean distributedJoins, boolean enforceJoinOrder, boolean collocated, + boolean replicatedOnly, boolean lazy, boolean skipReducerOnUpdate)throws Exception { + assertEquals(distributedJoins, io(conn).connectionProperties().isDistributedJoins()); + assertEquals(enforceJoinOrder, io(conn).connectionProperties().isEnforceJoinOrder()); + assertEquals(collocated, io(conn).connectionProperties().isCollocated()); + assertEquals(replicatedOnly, io(conn).connectionProperties().isReplicatedOnly()); + assertEquals(lazy, io(conn).connectionProperties().isLazy()); + assertEquals(skipReducerOnUpdate, io(conn).connectionProperties().isSkipReducerOnUpdate()); + } + /** * Test TCP no delay property handling. * @@ -303,6 +353,41 @@ public void testTcpNoDelay() throws Exception { } } + /** + * Test TCP no delay property handling with semicolon. + * + * @throws Exception If failed. + */ + public void testTcpNoDelaySemicolon() throws Exception { + assertInvalid("jdbc:ignite:thin://127.0.0.1;tcpNoDelay=0", + "Invalid property value. [name=tcpNoDelay, val=0, choices=[true, false]]"); + + assertInvalid("jdbc:ignite:thin://127.0.0.1;tcpNoDelay=1", + "Invalid property value. [name=tcpNoDelay, val=1, choices=[true, false]]"); + + assertInvalid("jdbc:ignite:thin://127.0.0.1;tcpNoDelay=false1", + "Invalid property value. [name=tcpNoDelay, val=false1, choices=[true, false]]"); + + assertInvalid("jdbc:ignite:thin://127.0.0.1;tcpNoDelay=true1", + "Invalid property value. [name=tcpNoDelay, val=true1, choices=[true, false]]"); + + try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;tcpNoDelay=true")) { + assertTrue(io(conn).connectionProperties().isTcpNoDelay()); + } + + try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;tcpNoDelay=True")) { + assertTrue(io(conn).connectionProperties().isTcpNoDelay()); + } + + try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;tcpNoDelay=false")) { + assertFalse(io(conn).connectionProperties().isTcpNoDelay()); + } + + try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;tcpNoDelay=False")) { + assertFalse(io(conn).connectionProperties().isTcpNoDelay()); + } + } + /** * Test autoCloseServerCursor property handling. * @@ -339,6 +424,38 @@ public void testAutoCloseServerCursorProperty() throws Exception { } } + /** + * Test autoCloseServerCursor property handling with semicolon. + * + * @throws Exception If failed. + */ + public void testAutoCloseServerCursorPropertySemicolon() throws Exception { + String url = "jdbc:ignite:thin://127.0.0.1;autoCloseServerCursor"; + + String err = "Invalid property value. [name=autoCloseServerCursor"; + + assertInvalid(url + "=0", err); + assertInvalid(url + "=1", err); + assertInvalid(url + "=false1", err); + assertInvalid(url + "=true1", err); + + try (Connection conn = DriverManager.getConnection(url + "=true")) { + assertTrue(io(conn).connectionProperties().isAutoCloseServerCursor()); + } + + try (Connection conn = DriverManager.getConnection(url + "=True")) { + assertTrue(io(conn).connectionProperties().isAutoCloseServerCursor()); + } + + try (Connection conn = DriverManager.getConnection(url + "=false")) { + assertFalse(io(conn).connectionProperties().isAutoCloseServerCursor()); + } + + try (Connection conn = DriverManager.getConnection(url + "=False")) { + assertFalse(io(conn).connectionProperties().isAutoCloseServerCursor()); + } + } + /** * Test schema property in URL. * @@ -361,6 +478,25 @@ public void testSchema() throws Exception { } } + /** + * Test schema property in URL with semicolon. + * + * @throws Exception If failed. + */ + public void testSchemaSemicolon() throws Exception { + try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;schema=public")) { + assertEquals("Invalid schema", "PUBLIC", conn.getSchema()); + } + + try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;schema=\"" + DEFAULT_CACHE_NAME + '"')) { + assertEquals("Invalid schema", DEFAULT_CACHE_NAME, conn.getSchema()); + } + + try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;schema=_not_exist_schema_")) { + assertEquals("Invalid schema", "_NOT_EXIST_SCHEMA_", conn.getSchema()); + } + } + /** * Get client socket for connection. * @@ -1010,6 +1146,7 @@ public void testGetSetTransactionIsolation() throws Exception { // Invalid parameter value GridTestUtils.assertThrows(log, new Callable() { + @SuppressWarnings("MagicConstant") @Override public Object call() throws Exception { conn.setTransactionIsolation(-1); 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 5d770054f46d4..86dc2980ecd9a 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 @@ -23,8 +23,6 @@ import java.util.Arrays; import java.util.Properties; import java.util.StringTokenizer; -import javax.naming.RefAddr; -import javax.naming.Reference; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.configuration.ClientConnectorConfiguration; import org.apache.ignite.internal.processors.odbc.SqlStateCode; @@ -44,6 +42,9 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa /** Default socket buffer size. */ private static final int DFLT_SOCK_BUFFER_SIZE = 64 * 1024; + /** Property: schema. */ + private static final String PROP_SCHEMA = "schema"; + /** Connection URL. */ private String url; @@ -51,7 +52,7 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa private HostAndPortRange [] addrs; /** Schema name. Hidden property. Is used to set default schema name part of the URL. */ - private StringProperty schema = new StringProperty("schema", + private StringProperty schema = new StringProperty(PROP_SCHEMA, "Schema name of the connection", "PUBLIC", null, false, null); /** Distributed joins property. */ @@ -487,21 +488,113 @@ private void parseUrl(String url, Properties props) throws SQLException { String nakedUrl = url.substring(JdbcThinUtils.URL_PREFIX.length()).trim(); - int pathPartEndPos = nakedUrl.indexOf('?'); + parseUrl0(nakedUrl, props); + } + + /** + * Parse naked URL (i.e. without {@link JdbcThinUtils#URL_PREFIX}). + * + * @param url Naked URL. + * @param props Properties. + * @throws SQLException If failed. + */ + private void parseUrl0(String url, Properties props) throws SQLException { + // Determine mode - semicolon or ampersand. + int semicolonPos = url.indexOf(";"); + int slashPos = url.indexOf("/"); + int queryPos = url.indexOf("?"); + + boolean semicolonMode; + + if (semicolonPos == -1 && slashPos == -1 && queryPos == -1) + // No special char -> any mode could be used, choose semicolon for simplicity. + semicolonMode = true; + else { + if (semicolonPos != -1) { + // Use semicolon mode if it appears earlier than slash or query. + semicolonMode = + (slashPos == -1 || semicolonPos < slashPos) && (queryPos == -1 || semicolonPos < queryPos); + } + else + // Semicolon is not found. + semicolonMode = false; + } + + if (semicolonMode) + parseUrlWithSemicolon(url, props); + else + parseUrlWithQuery(url, props); + } + + /** + * Parse URL in semicolon mode. + * + * @param url Naked URL + * @param props Properties. + * @throws SQLException If failed. + */ + private void parseUrlWithSemicolon(String url, Properties props) throws SQLException { + int pathPartEndPos = url.indexOf(';'); if (pathPartEndPos == -1) - pathPartEndPos = nakedUrl.length(); + pathPartEndPos = url.length(); - String pathPart = nakedUrl.substring(0, pathPartEndPos); + String pathPart = url.substring(0, pathPartEndPos); String paramPart = null; - if (pathPartEndPos > 0 && pathPartEndPos < nakedUrl.length()) - paramPart = nakedUrl.substring(pathPartEndPos + 1, nakedUrl.length()); + if (pathPartEndPos > 0 && pathPartEndPos < url.length()) + paramPart = url.substring(pathPartEndPos + 1, url.length()); + + parseEndpoints(pathPart); + + if (!F.isEmpty(paramPart)) + parseParameters(paramPart, props, ";"); + } + + /** + * Parse URL in query mode. + * + * @param url Naked URL + * @param props Properties. + * @throws SQLException If failed. + */ + private void parseUrlWithQuery(String url, Properties props) throws SQLException { + int pathPartEndPos = url.indexOf('?'); + + if (pathPartEndPos == -1) + pathPartEndPos = url.length(); + + String pathPart = url.substring(0, pathPartEndPos); + + String paramPart = null; + + if (pathPartEndPos > 0 && pathPartEndPos < url.length()) + paramPart = url.substring(pathPartEndPos + 1, url.length()); String[] pathParts = pathPart.split("/"); - String [] endpoints = pathParts[0].split(","); + parseEndpoints(pathParts[0]); + + if (pathParts.length > 2) { + throw new SQLException("Invalid URL format (only schema name is allowed in URL path parameter " + + "'host:port[/schemaName]'): " + this.url, SqlStateCode.CLIENT_CONNECTION_FAILED); + } + + setSchema(pathParts.length == 2 ? pathParts[1] : null); + + if (!F.isEmpty(paramPart)) + parseParameters(paramPart, props, "&"); + } + + /** + * Parse endpoints. + * + * @param endpointStr Endpoint string. + * @throws SQLException If failed. + */ + private void parseEndpoints(String endpointStr) throws SQLException { + String [] endpoints = endpointStr.split(","); if (endpoints.length > 0) addrs = new HostAndPortRange[endpoints.length]; @@ -519,16 +612,6 @@ private void parseUrl(String url, Properties props) throws SQLException { if (F.isEmpty(addrs) || F.isEmpty(addrs[0].host())) throw new SQLException("Host name is empty", SqlStateCode.CLIENT_CONNECTION_FAILED); - - if (pathParts.length > 2) { - throw new SQLException("Invalid URL format (only schema name is allowed in URL path parameter " + - "'host:port[/schemaName]'): " + url, SqlStateCode.CLIENT_CONNECTION_FAILED); - } - - setSchema(pathParts.length == 2 ? pathParts[1] : null); - - if (!F.isEmpty(paramPart)) - parseParameters(paramPart, props); } /** @@ -536,10 +619,11 @@ private void parseUrl(String url, Properties props) throws SQLException { * * @param paramStr Parameters string. * @param props Properties. + * @param delimChar Delimiter character. * @throws SQLException If failed. */ - private void parseParameters(String paramStr, Properties props) throws SQLException { - StringTokenizer st = new StringTokenizer(paramStr, "&"); + private void parseParameters(String paramStr, Properties props, String delimChar) throws SQLException { + StringTokenizer st = new StringTokenizer(paramStr, delimChar); boolean insideBrace = false; @@ -553,8 +637,8 @@ private void parseParameters(String paramStr, Properties props) throws SQLExcept int eqSymPos = token.indexOf('='); if (eqSymPos < 0) { - throw new SQLException("Invalid parameter format " + - "(URL properties format: key0=value0&key1=value1&... etc. pair: " + token); + throw new SQLException("Invalid parameter format (should be \"key1=val1" + delimChar + + "key2=val2" + delimChar + "...\"): " + token); } if (eqSymPos == token.length()) @@ -570,7 +654,7 @@ private void parseParameters(String paramStr, Properties props) throws SQLExcept } } else - val += "&" + token; + val += delimChar + token; if (val.endsWith("}")) { insideBrace = false; @@ -587,22 +671,24 @@ private void parseParameters(String paramStr, Properties props) throws SQLExcept if (key.isEmpty() || val.isEmpty()) throw new SQLException("Invalid parameter format (key and value cannot be empty): " + token); - props.setProperty(PROP_PREFIX + key, val); + if (PROP_SCHEMA.equalsIgnoreCase(key)) + setSchema(val); + else + props.setProperty(PROP_PREFIX + key, val); } } } - /** * @return Driver's properties info array. */ public DriverPropertyInfo[] getDriverPropertyInfo() { - DriverPropertyInfo[] dpis = new DriverPropertyInfo[propsArray.length]; + DriverPropertyInfo[] infos = new DriverPropertyInfo[propsArray.length]; for (int i = 0; i < propsArray.length; ++i) - dpis[i] = propsArray[i].getDriverPropertyInfo(); + infos[i] = propsArray[i].getDriverPropertyInfo(); - return dpis; + return infos; } /** @@ -740,23 +826,6 @@ protected void checkChoices(String strVal) throws SQLException { } } - /** - * @param ref Reference object. - * @throws SQLException On error. - */ - void init(Reference ref) throws SQLException { - RefAddr refAddr = ref.get(name); - - if (refAddr != null) { - String str = (String) refAddr.getContent(); - - if (validator != null) - validator.validate(str); - - init(str); - } - } - /** * @param str String representation of the * @throws SQLException on error. From e6c30e17c6f0f1852fa781078ee54ccf8c654846 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 11 Apr 2018 14:12:50 +0300 Subject: [PATCH 0028/1463] IGNITE-7871 Check local join future on error. - Fixes #3793. Signed-off-by: dpavlov --- .../distributed/dht/preloader/latch/ExchangeLatchManager.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java index c205cb14e40d9..404f88f9f0695 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java @@ -104,7 +104,8 @@ public ExchangeLatchManager(GridKernalContext ctx) { // First coordinator initialization. ctx.discovery().localJoinFuture().listen(f -> { - this.coordinator = getLatchCoordinator(AffinityTopologyVersion.NONE); + if (f.error() == null) + this.coordinator = getLatchCoordinator(AffinityTopologyVersion.NONE); }); ctx.event().addDiscoveryEventListener((e, cache) -> { From 2769981a5df64f3cd0c38b7599c49580c66192fa Mon Sep 17 00:00:00 2001 From: Aleksey Plekhanov Date: Wed, 11 Apr 2018 18:24:51 +0300 Subject: [PATCH 0029/1463] IGNITE-6892 OOM should be covered by failure handling Signed-off-by: Andrey Gura --- .../apache/ignite/IgniteSystemProperties.java | 15 ++ .../apache/ignite/internal/IgnitionEx.java | 50 +++- .../discovery/GridDiscoveryManager.java | 3 + .../processors/cache/WalStateManager.java | 8 +- .../continuous/GridContinuousProcessor.java | 3 + .../datastreamer/DataStreamProcessor.java | 3 + .../processors/failure/FailureProcessor.java | 11 + .../processors/job/GridJobWorker.java | 8 +- .../service/GridServiceProcessor.java | 15 +- .../IgniteStripedThreadPoolExecutor.java | 8 +- .../ignite/thread/IgniteThreadFactory.java | 30 ++- .../thread/IgniteThreadPoolExecutor.java | 12 +- .../ignite/thread/OomExceptionHandler.java | 44 +++ .../ignite/failure/OomFailureHandlerTest.java | 255 ++++++++++++++++++ .../testsuites/IgniteBasicTestSuite.java | 2 + 15 files changed, 437 insertions(+), 30 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/thread/OomExceptionHandler.java create mode 100644 modules/core/src/test/java/org/apache/ignite/failure/OomFailureHandlerTest.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 ac7947b7e45aa..437f49f6a0a9c 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -862,6 +862,21 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_BPLUS_TREE_LOCK_RETRIES = "IGNITE_BPLUS_TREE_LOCK_RETRIES"; + /** + * Amount of memory reserved in the heap at node start, which can be dropped to increase the chances of success when + * handling OutOfMemoryError. + * + * Default is {@code 64kb}. + */ + public static final String IGNITE_FAILURE_HANDLER_RESERVE_BUFFER_SIZE = "IGNITE_FAILURE_HANDLER_RESERVE_BUFFER_SIZE"; + + /** + * The threshold of uneven distribution above which partition distribution will be logged. + * + * The default is '50', that means: warn about nodes with 50+% difference. + */ + public static final String IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD = "IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD"; + /** * 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 c0de08050fe60..e140609aac497 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 @@ -20,6 +20,7 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; +import java.lang.Thread.UncaughtExceptionHandler; import java.lang.management.ManagementFactory; import java.lang.reflect.Constructor; import java.net.MalformedURLException; @@ -88,6 +89,7 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; @@ -1764,6 +1766,13 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { validateThreadPoolSize(cfg.getPublicThreadPoolSize(), "public"); + UncaughtExceptionHandler oomeHnd = new UncaughtExceptionHandler() { + @Override public void uncaughtException(Thread t, Throwable e) { + if (grid != null && X.hasCause(e, OutOfMemoryError.class)) + grid.context().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e)); + } + }; + execSvc = new IgniteThreadPoolExecutor( "pub", cfg.getIgniteInstanceName(), @@ -1771,7 +1780,8 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { cfg.getPublicThreadPoolSize(), DFLT_THREAD_KEEP_ALIVE_TIME, new LinkedBlockingQueue(), - GridIoPolicy.PUBLIC_POOL); + GridIoPolicy.PUBLIC_POOL, + oomeHnd); execSvc.allowCoreThreadTimeOut(true); @@ -1784,7 +1794,8 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { cfg.getServiceThreadPoolSize(), DFLT_THREAD_KEEP_ALIVE_TIME, new LinkedBlockingQueue(), - GridIoPolicy.SERVICE_POOL); + GridIoPolicy.SERVICE_POOL, + oomeHnd); svcExecSvc.allowCoreThreadTimeOut(true); @@ -1797,7 +1808,8 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { cfg.getSystemThreadPoolSize(), DFLT_THREAD_KEEP_ALIVE_TIME, new LinkedBlockingQueue(), - GridIoPolicy.SYSTEM_POOL); + GridIoPolicy.SYSTEM_POOL, + oomeHnd); sysExecSvc.allowCoreThreadTimeOut(true); @@ -1828,7 +1840,8 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { cfg.getManagementThreadPoolSize(), DFLT_THREAD_KEEP_ALIVE_TIME, new LinkedBlockingQueue(), - GridIoPolicy.MANAGEMENT_POOL); + GridIoPolicy.MANAGEMENT_POOL, + oomeHnd); mgmtExecSvc.allowCoreThreadTimeOut(true); @@ -1844,7 +1857,8 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { cfg.getPeerClassLoadingThreadPoolSize(), DFLT_THREAD_KEEP_ALIVE_TIME, new LinkedBlockingQueue(), - GridIoPolicy.P2P_POOL); + GridIoPolicy.P2P_POOL, + oomeHnd); p2pExecSvc.allowCoreThreadTimeOut(true); @@ -1879,7 +1893,8 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { callbackExecSvc = new IgniteStripedThreadPoolExecutor( cfg.getAsyncCallbackPoolSize(), cfg.getIgniteInstanceName(), - "callback"); + "callback", + oomeHnd); if (myCfg.getConnectorConfiguration() != null) { validateThreadPoolSize(myCfg.getConnectorConfiguration().getThreadPoolSize(), "connector"); @@ -1890,7 +1905,9 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { myCfg.getConnectorConfiguration().getThreadPoolSize(), myCfg.getConnectorConfiguration().getThreadPoolSize(), DFLT_THREAD_KEEP_ALIVE_TIME, - new LinkedBlockingQueue() + new LinkedBlockingQueue(), + GridIoPolicy.UNDEFINED, + oomeHnd ); restExecSvc.allowCoreThreadTimeOut(true); @@ -1905,7 +1922,8 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { myCfg.getUtilityCacheThreadPoolSize(), myCfg.getUtilityCacheKeepAliveTime(), new LinkedBlockingQueue(), - GridIoPolicy.UTILITY_CACHE_POOL); + GridIoPolicy.UTILITY_CACHE_POOL, + oomeHnd); utilityCacheExecSvc.allowCoreThreadTimeOut(true); @@ -1916,7 +1934,8 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { 1, DFLT_THREAD_KEEP_ALIVE_TIME, new LinkedBlockingQueue(), - GridIoPolicy.AFFINITY_POOL); + GridIoPolicy.AFFINITY_POOL, + oomeHnd); affExecSvc.allowCoreThreadTimeOut(true); @@ -1930,7 +1949,8 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { cpus * 2, 3000L, new LinkedBlockingQueue(1000), - GridIoPolicy.IDX_POOL + GridIoPolicy.IDX_POOL, + oomeHnd ); } @@ -1943,7 +1963,8 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { cfg.getQueryThreadPoolSize(), DFLT_THREAD_KEEP_ALIVE_TIME, new LinkedBlockingQueue(), - GridIoPolicy.QUERY_POOL); + GridIoPolicy.QUERY_POOL, + oomeHnd); qryExecSvc.allowCoreThreadTimeOut(true); @@ -1954,7 +1975,8 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { 2, DFLT_THREAD_KEEP_ALIVE_TIME, new LinkedBlockingQueue(), - GridIoPolicy.SCHEMA_POOL); + GridIoPolicy.SCHEMA_POOL, + oomeHnd); schemaExecSvc.allowCoreThreadTimeOut(true); @@ -1970,7 +1992,9 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { execCfg.getSize(), execCfg.getSize(), DFLT_THREAD_KEEP_ALIVE_TIME, - new LinkedBlockingQueue()); + new LinkedBlockingQueue(), + GridIoPolicy.UNDEFINED, + oomeHnd); customExecSvcs.put(execCfg.getName(), exec); } 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 400bb5fd28742..77c96573d0a63 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 @@ -130,6 +130,7 @@ import org.apache.ignite.spi.discovery.DiscoverySpiOrderSupport; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.thread.IgniteThread; +import org.apache.ignite.thread.OomExceptionHandler; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -924,6 +925,8 @@ else if (type == EVT_CLIENT_NODE_RECONNECTED) { segChkThread = new IgniteThread(segChkWrk); + segChkThread.setUncaughtExceptionHandler(new OomExceptionHandler(ctx)); + segChkThread.start(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java index 0ac699f5a8fee..64a6819826357 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java @@ -38,6 +38,7 @@ import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.thread.OomExceptionHandler; import org.apache.ignite.thread.IgniteThread; import org.jetbrains.annotations.Nullable; @@ -473,7 +474,12 @@ public void onProposeExchange(WalStateProposeMessage msg) { // not-yet-flushed dirty pages have been logged. WalStateChangeWorker worker = new WalStateChangeWorker(msg, cpFut); - new IgniteThread(worker).start(); + IgniteThread thread = new IgniteThread(worker); + + thread.setUncaughtExceptionHandler(new OomExceptionHandler( + cctx.kernalContext())); + + thread.start(); } else { // Disable: not-yet-flushed operations are not logged, so wait for them diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java index cebe4b177e300..2d48b7d9b16fb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java @@ -88,6 +88,7 @@ import org.apache.ignite.spi.discovery.DiscoveryDataBag.GridDiscoveryData; import org.apache.ignite.spi.discovery.DiscoveryDataBag.JoiningNodeDiscoveryData; import org.apache.ignite.thread.IgniteThread; +import org.apache.ignite.thread.OomExceptionHandler; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; @@ -1727,6 +1728,8 @@ private boolean registerHandler(final UUID nodeId, } }); + checker.setUncaughtExceptionHandler(new OomExceptionHandler(ctx)); + bufCheckThreads.put(routineId, checker); checker.start(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java index 8b984c05c1988..e63d7d4d28bc1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java @@ -44,6 +44,7 @@ import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.stream.StreamReceiver; import org.apache.ignite.thread.IgniteThread; +import org.apache.ignite.thread.OomExceptionHandler; import org.jetbrains.annotations.Nullable; import java.util.Collection; @@ -125,6 +126,8 @@ public DataStreamProcessor(GridKernalContext ctx) { } }); + flusher.setUncaughtExceptionHandler(new OomExceptionHandler(ctx)); + flusher.start(); if (log.isDebugEnabled()) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java index 615fb9f55456f..0234e84345580 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java @@ -19,12 +19,14 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.failure.FailureContext; import org.apache.ignite.failure.FailureHandler; import org.apache.ignite.failure.StopNodeOrHaltFailureHandler; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.GridProcessorAdapter; +import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; /** @@ -40,6 +42,9 @@ public class FailureProcessor extends GridProcessorAdapter { /** Failure context. */ private volatile FailureContext failureCtx; + /** Reserve buffer, which can be dropped to handle OOME. */ + private volatile byte[] reserveBuf; + /** * @param ctx Context. */ @@ -56,6 +61,9 @@ public FailureProcessor(GridKernalContext ctx) { if (hnd == null) hnd = getDefaultFailureHandler(); + reserveBuf = new byte[IgniteSystemProperties.getInteger( + IgniteSystemProperties.IGNITE_FAILURE_HANDLER_RESERVE_BUFFER_SIZE, 64 * 1024)]; + assert hnd != null; this.hnd = hnd; @@ -102,6 +110,9 @@ public synchronized void process(FailureContext failureCtx, FailureHandler hnd) U.error(ignite.log(), "Critical failure. Will be handled accordingly to configured handler [hnd=" + hnd.getClass() + ", failureCtx=" + failureCtx + ']', failureCtx.error()); + if (reserveBuf != null && X.hasCause(failureCtx.error(), OutOfMemoryError.class)) + reserveBuf = null; + boolean invalidated = hnd.onFailure(ignite, failureCtx); if (invalidated) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java index 6d2e621c3b9af..f7c07f516f413 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java @@ -36,6 +36,8 @@ import org.apache.ignite.compute.ComputeJobMasterLeaveAware; import org.apache.ignite.compute.ComputeUserUndeclaredException; import org.apache.ignite.events.JobEvent; +import org.apache.ignite.failure.FailureContext; +import org.apache.ignite.failure.FailureType; import org.apache.ignite.igfs.IgfsOutOfSpaceException; import org.apache.ignite.internal.GridInternalException; import org.apache.ignite.internal.GridJobContextImpl; @@ -603,9 +605,13 @@ else if (X.hasCause(e, GridServiceNotFoundException.class) || X.hasCause(e, ClusterTopologyCheckedException.class)) // Should be throttled, because GridServiceProxy continuously retry getting service. LT.error(log, e, "Failed to execute job [jobId=" + ses.getJobId() + ", ses=" + ses + ']'); - else + else { U.error(log, "Failed to execute job [jobId=" + ses.getJobId() + ", ses=" + ses + ']', e); + if (X.hasCause(e, OutOfMemoryError.class)) + ctx.failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e)); + } + ex = e; } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index ff68e7290b99b..63f50273c88ae 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 @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.service; +import java.lang.Thread.UncaughtExceptionHandler; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -103,6 +104,7 @@ import org.apache.ignite.services.ServiceDeploymentException; import org.apache.ignite.services.ServiceDescriptor; import org.apache.ignite.thread.IgniteThreadFactory; +import org.apache.ignite.thread.OomExceptionHandler; import org.apache.ignite.transactions.Transaction; import org.jetbrains.annotations.Nullable; import java.util.concurrent.ConcurrentHashMap; @@ -112,7 +114,6 @@ import static org.apache.ignite.configuration.DeploymentMode.ISOLATED; import static org.apache.ignite.configuration.DeploymentMode.PRIVATE; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SERVICES_COMPATIBILITY_MODE; -import static org.apache.ignite.internal.processors.cache.GridCacheUtils.UTILITY_CACHE_NAME; import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED; import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; @@ -154,8 +155,12 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite /** Busy lock. */ private volatile GridSpinBusyLock busyLock = new GridSpinBusyLock(); + /** Uncaught exception handler for thread pools. */ + private final UncaughtExceptionHandler oomeHnd = new OomExceptionHandler(ctx); + /** Thread factory. */ - private ThreadFactory threadFactory = new IgniteThreadFactory(ctx.igniteInstanceName(), "service"); + private ThreadFactory threadFactory = new IgniteThreadFactory(ctx.igniteInstanceName(), "service", + oomeHnd); /** Thread local for service name. */ private ThreadLocal svcName = new ThreadLocal<>(); @@ -175,7 +180,8 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite public GridServiceProcessor(GridKernalContext ctx) { super(ctx); - depExe = Executors.newSingleThreadExecutor(new IgniteThreadFactory(ctx.igniteInstanceName(), "srvc-deploy")); + depExe = Executors.newSingleThreadExecutor(new IgniteThreadFactory(ctx.igniteInstanceName(), + "srvc-deploy", oomeHnd)); String servicesCompatibilityMode = getString(IGNITE_SERVICES_COMPATIBILITY_MODE); @@ -373,7 +379,8 @@ private IgniteInternalCache serviceCache() { busyLock = new GridSpinBusyLock(); - depExe = Executors.newSingleThreadExecutor(new IgniteThreadFactory(ctx.igniteInstanceName(), "srvc-deploy")); + depExe = Executors.newSingleThreadExecutor(new IgniteThreadFactory(ctx.igniteInstanceName(), + "srvc-deploy", oomeHnd)); start(); diff --git a/modules/core/src/main/java/org/apache/ignite/thread/IgniteStripedThreadPoolExecutor.java b/modules/core/src/main/java/org/apache/ignite/thread/IgniteStripedThreadPoolExecutor.java index 3cd7484567b8e..418812f8a20c0 100644 --- a/modules/core/src/main/java/org/apache/ignite/thread/IgniteStripedThreadPoolExecutor.java +++ b/modules/core/src/main/java/org/apache/ignite/thread/IgniteStripedThreadPoolExecutor.java @@ -17,6 +17,7 @@ package org.apache.ignite.thread; +import java.lang.Thread.UncaughtExceptionHandler; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -45,10 +46,11 @@ public class IgniteStripedThreadPoolExecutor implements ExecutorService { * @param igniteInstanceName Node name. * @param threadNamePrefix Thread name prefix. */ - public IgniteStripedThreadPoolExecutor(int concurrentLvl, String igniteInstanceName, String threadNamePrefix) { + public IgniteStripedThreadPoolExecutor(int concurrentLvl, String igniteInstanceName, String threadNamePrefix, + UncaughtExceptionHandler eHnd) { execs = new ExecutorService[concurrentLvl]; - ThreadFactory factory = new IgniteThreadFactory(igniteInstanceName, threadNamePrefix); + ThreadFactory factory = new IgniteThreadFactory(igniteInstanceName, threadNamePrefix, eHnd); for (int i = 0; i < concurrentLvl; i++) execs[i] = Executors.newSingleThreadExecutor(factory); @@ -173,4 +175,4 @@ public int threadId(int idx) { @Override public String toString() { return S.toString(IgniteStripedThreadPoolExecutor.class, this); } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java index 062c973ba1220..23bf14df1b685 100644 --- a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java @@ -17,9 +17,9 @@ package org.apache.ignite.thread; +import java.lang.Thread.UncaughtExceptionHandler; import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.ignite.internal.managers.communication.GridIoPolicy; import org.apache.ignite.internal.util.typedef.internal.S; import org.jetbrains.annotations.NotNull; @@ -41,6 +41,9 @@ public class IgniteThreadFactory implements ThreadFactory { /** */ private final byte plc; + /** Exception handler. */ + private final UncaughtExceptionHandler eHnd; + /** * Constructs new thread factory for given grid. All threads will belong * to the same default thread group. @@ -49,7 +52,19 @@ public class IgniteThreadFactory implements ThreadFactory { * @param threadName Thread name. */ public IgniteThreadFactory(String igniteInstanceName, String threadName) { - this(igniteInstanceName, threadName, GridIoPolicy.UNDEFINED); + this(igniteInstanceName, threadName, null); + } + + /** + * Constructs new thread factory for given grid. All threads will belong + * to the same default thread group. + * + * @param igniteInstanceName Ignite instance name. + * @param threadName Thread name. + * @param eHnd Uncaught exception handler. + */ + public IgniteThreadFactory(String igniteInstanceName, String threadName, UncaughtExceptionHandler eHnd) { + this(igniteInstanceName, threadName, GridIoPolicy.UNDEFINED, eHnd); } /** @@ -59,16 +74,23 @@ public IgniteThreadFactory(String igniteInstanceName, String threadName) { * @param igniteInstanceName Ignite instance name. * @param threadName Thread name. * @param plc {@link GridIoPolicy} for thread pool. + * @param eHnd Uncaught exception handler. */ - public IgniteThreadFactory(String igniteInstanceName, String threadName, byte plc) { + public IgniteThreadFactory(String igniteInstanceName, String threadName, byte plc, UncaughtExceptionHandler eHnd) { this.igniteInstanceName = igniteInstanceName; this.threadName = threadName; this.plc = plc; + this.eHnd = eHnd; } /** {@inheritDoc} */ @Override public Thread newThread(@NotNull Runnable r) { - return new IgniteThread(igniteInstanceName, threadName, r, idxGen.incrementAndGet(), -1, plc); + Thread thread = new IgniteThread(igniteInstanceName, threadName, r, idxGen.incrementAndGet(), -1, plc); + + if (eHnd != null) + thread.setUncaughtExceptionHandler(eHnd); + + return thread; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadPoolExecutor.java b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadPoolExecutor.java index 83c64c3363000..fed77adb055b5 100644 --- a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadPoolExecutor.java +++ b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadPoolExecutor.java @@ -17,6 +17,7 @@ package org.apache.ignite.thread; +import java.lang.Thread.UncaughtExceptionHandler; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutorService; import java.util.concurrent.ThreadFactory; @@ -53,7 +54,8 @@ public IgniteThreadPoolExecutor( maxPoolSize, keepAliveTime, workQ, - GridIoPolicy.UNDEFINED); + GridIoPolicy.UNDEFINED, + null); } /** @@ -68,6 +70,7 @@ public IgniteThreadPoolExecutor( * @param workQ The queue to use for holding tasks before they are executed. This queue will hold only * runnable tasks submitted by the {@link #execute(Runnable)} method. * @param plc {@link GridIoPolicy} for thread pool. + * @param eHnd Uncaught exception handler for thread pool. */ public IgniteThreadPoolExecutor( String threadNamePrefix, @@ -76,14 +79,15 @@ public IgniteThreadPoolExecutor( int maxPoolSize, long keepAliveTime, BlockingQueue workQ, - byte plc) { + byte plc, + UncaughtExceptionHandler eHnd) { super( corePoolSize, maxPoolSize, keepAliveTime, TimeUnit.MILLISECONDS, workQ, - new IgniteThreadFactory(igniteInstanceName, threadNamePrefix, plc) + new IgniteThreadFactory(igniteInstanceName, threadNamePrefix, plc, eHnd) ); } @@ -114,4 +118,4 @@ public IgniteThreadPoolExecutor( new AbortPolicy() ); } -} \ No newline at end of file +} diff --git a/modules/core/src/main/java/org/apache/ignite/thread/OomExceptionHandler.java b/modules/core/src/main/java/org/apache/ignite/thread/OomExceptionHandler.java new file mode 100644 index 0000000000000..3a62ad87e5de9 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/thread/OomExceptionHandler.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.thread; + +import org.apache.ignite.failure.FailureContext; +import org.apache.ignite.failure.FailureType; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.util.typedef.X; + +/** + * OOM exception handler for system threads. + */ +public class OomExceptionHandler implements Thread.UncaughtExceptionHandler { + /** Context. */ + private final GridKernalContext ctx; + + /** + * @param ctx Context. + */ + public OomExceptionHandler(GridKernalContext ctx) { + this.ctx = ctx; + } + + /** {@inheritDoc} */ + @Override public void uncaughtException(Thread t, Throwable e) { + if (X.hasCause(e, OutOfMemoryError.class)) + ctx.failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e)); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/failure/OomFailureHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/failure/OomFailureHandlerTest.java new file mode 100644 index 0000000000000..2af94b88ee3e4 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/failure/OomFailureHandlerTest.java @@ -0,0 +1,255 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.failure; + +import javax.cache.processor.EntryProcessor; +import javax.cache.processor.EntryProcessorException; +import javax.cache.processor.MutableEntry; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.events.Event; +import org.apache.ignite.events.EventType; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.services.Service; +import org.apache.ignite.services.ServiceContext; +import org.apache.ignite.testframework.GridTestUtils; + +/** + * Out of memory error failure handler test. + */ +public class OomFailureHandlerTest extends AbstractFailureHandlerTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setCacheConfiguration(new CacheConfiguration() + .setName(DEFAULT_CACHE_NAME) + .setCacheMode(CacheMode.PARTITIONED) + .setBackups(0) + ); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + } + + /** + * Test OOME in IgniteCompute. + */ + public void testComputeOomError() throws Exception { + IgniteEx ignite0 = startGrid(0); + IgniteEx ignite1 = startGrid(1); + + try { + IgniteFuture res = ignite0.compute(ignite0.cluster().forNodeId(ignite1.cluster().localNode().id())) + .callAsync(new IgniteCallable() { + @Override public Boolean call() throws Exception { + throw new OutOfMemoryError(); + } + }); + + res.get(); + } + catch (Throwable ignore) { + // Expected. + } + + assertFailureState(ignite0, ignite1); + } + + /** + * Test OOME in EntryProcessor. + */ + public void testEntryProcessorOomError() throws Exception { + IgniteEx ignite0 = startGrid(0); + IgniteEx ignite1 = startGrid(1); + + IgniteCache cache0 = ignite0.getOrCreateCache(DEFAULT_CACHE_NAME); + IgniteCache cache1 = ignite1.getOrCreateCache(DEFAULT_CACHE_NAME); + + awaitPartitionMapExchange(); + + Integer key = primaryKey(cache1); + + cache1.put(key, key); + + try { + IgniteFuture fut = cache0.invokeAsync(key, new EntryProcessor() { + @Override public Object process(MutableEntry entry, + Object... arguments) throws EntryProcessorException { + throw new OutOfMemoryError(); + } + }); + + fut.get(); + } + catch (Throwable ignore) { + // Expected. + } + + assertFailureState(ignite0, ignite1); + } + + /** + * Test OOME in service method invocation. + */ + public void testServiceInvokeOomError() throws Exception { + IgniteEx ignite0 = startGrid(0); + IgniteEx ignite1 = startGrid(1); + + IgniteCache cache1 = ignite1.getOrCreateCache(DEFAULT_CACHE_NAME); + + awaitPartitionMapExchange(); + + Integer key = primaryKey(cache1); + + ignite0.services().deployKeyAffinitySingleton("fail-invoke-service", new FailServiceImpl(false), + DEFAULT_CACHE_NAME, key); + + FailService svc = ignite0.services().serviceProxy("fail-invoke-service", FailService.class, false); + + try { + svc.fail(); + } + catch (Throwable ignore) { + // Expected. + } + + assertFailureState(ignite0, ignite1); + } + + /** + * Test OOME in service execute. + */ + public void testServiceExecuteOomError() throws Exception { + IgniteEx ignite0 = startGrid(0); + IgniteEx ignite1 = startGrid(1); + + IgniteCache cache1 = ignite1.getOrCreateCache(DEFAULT_CACHE_NAME); + + awaitPartitionMapExchange(); + + Integer key = primaryKey(cache1); + + ignite0.services().deployKeyAffinitySingleton("fail-execute-service", new FailServiceImpl(true), + DEFAULT_CACHE_NAME, key); + + assertFailureState(ignite0, ignite1); + } + + /** + * Test OOME in event listener. + */ + public void testEventListenerOomError() throws Exception { + IgniteEx ignite0 = startGrid(0); + IgniteEx ignite1 = startGrid(1); + + IgniteCache cache0 = ignite0.getOrCreateCache(DEFAULT_CACHE_NAME); + IgniteCache cache1 = ignite1.getOrCreateCache(DEFAULT_CACHE_NAME); + + awaitPartitionMapExchange(); + + ignite1.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event evt) { + throw new OutOfMemoryError(); + } + }, EventType.EVT_CACHE_OBJECT_PUT); + + Integer key = primaryKey(cache1); + + try { + cache0.put(key, key); + } + catch (Throwable ignore) { + // Expected. + } + + assertFailureState(ignite0, ignite1); + } + + /** + * @param igniteWork Working ignite instance. + * @param igniteFail Failed ignite instance. + */ + private static void assertFailureState(Ignite igniteWork, Ignite igniteFail) throws IgniteInterruptedCheckedException { + assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return dummyFailureHandler(igniteFail).failure(); + } + }, 5000L)); + + assertFalse(dummyFailureHandler(igniteWork).failure()); + } + + /** + * + */ + private interface FailService extends Service { + /** + * Fail. + */ + void fail(); + } + + /** + * + */ + private static class FailServiceImpl implements FailService { + /** Fail on execute. */ + private final boolean failOnExec; + + /** + * @param failOnExec Fail on execute. + */ + private FailServiceImpl(boolean failOnExec) { + this.failOnExec = failOnExec; + } + + /** {@inheritDoc} */ + @Override public void fail() { + throw new OutOfMemoryError(); + } + + /** {@inheritDoc} */ + @Override public void cancel(ServiceContext ctx) { + } + + /** {@inheritDoc} */ + @Override public void init(ServiceContext ctx) throws Exception { + } + + /** {@inheritDoc} */ + @Override public void execute(ServiceContext ctx) throws Exception { + if (failOnExec) + throw new OutOfMemoryError(); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java index c4b7d9227f4ee..c388f1dd4d5c7 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java @@ -22,6 +22,7 @@ import org.apache.ignite.GridSuppressedExceptionSelfTest; import org.apache.ignite.failure.FailureHandlerTriggeredTest; import org.apache.ignite.failure.IoomFailureHandlerTest; +import org.apache.ignite.failure.OomFailureHandlerTest; import org.apache.ignite.failure.StopNodeFailureHandlerTest; import org.apache.ignite.failure.StopNodeOrHaltFailureHandlerTest; import org.apache.ignite.internal.ClassSetTest; @@ -199,6 +200,7 @@ public static TestSuite suite(@Nullable final Set ignoredTests) throws Ex suite.addTestSuite(StopNodeFailureHandlerTest.class); suite.addTestSuite(StopNodeOrHaltFailureHandlerTest.class); suite.addTestSuite(IoomFailureHandlerTest.class); + suite.addTestSuite(OomFailureHandlerTest.class); return suite; } From 687194461f445be9902752f38f873d321cde1d85 Mon Sep 17 00:00:00 2001 From: Ilya Borisov Date: Fri, 6 Apr 2018 11:17:05 +0700 Subject: [PATCH 0030/1463] IGNITE-7996 Move configuration form templates. (cherry picked from commit c2c03a9) --- .../cache-edit-form/template.tpl.pug | 22 +++---- .../cache-edit-form/templates}/affinity.pug | 0 .../templates}/concurrency.pug | 0 .../cache-edit-form/templates}/general.pug | 0 .../cache-edit-form/templates}/memory.pug | 0 .../templates}/near-cache-client.pug | 0 .../templates}/near-cache-server.pug | 0 .../templates}/node-filter.pug | 0 .../cache-edit-form/templates}/query.pug | 0 .../cache-edit-form/templates}/rebalance.pug | 0 .../cache-edit-form/templates}/statistics.pug | 0 .../cache-edit-form/templates}/store.pug | 0 .../cluster-edit-form/template.tpl.pug | 62 +++++++++---------- .../cluster-edit-form/templates}/atomic.pug | 0 .../templates}/attributes.pug | 0 .../cluster-edit-form/templates}/binary.pug | 0 .../templates}/cache-key-cfg.pug | 0 .../templates}/checkpoint.pug | 0 .../templates}/checkpoint/fs.pug | 0 .../templates}/checkpoint/jdbc.pug | 0 .../templates}/checkpoint/s3.pug | 0 .../templates}/client-connector.pug | 0 .../templates}/collision.pug | 0 .../templates}/collision/custom.pug | 0 .../templates}/collision/fifo-queue.pug | 0 .../templates}/collision/job-stealing.pug | 0 .../templates}/collision/priority-queue.pug | 0 .../templates}/communication.pug | 0 .../templates}/connector.pug | 0 .../templates}/data-storage.pug | 0 .../templates}/deployment.pug | 0 .../templates}/discovery.pug | 0 .../cluster-edit-form/templates}/events.pug | 0 .../cluster-edit-form/templates}/failover.pug | 0 .../cluster-edit-form/templates}/general.pug | 0 .../templates}/general/discovery/cloud.pug | 0 .../templates}/general/discovery/google.pug | 0 .../templates}/general/discovery/jdbc.pug | 0 .../general/discovery/kubernetes.pug | 0 .../general/discovery/multicast.pug | 0 .../templates}/general/discovery/s3.pug | 0 .../templates}/general/discovery/shared.pug | 0 .../templates}/general/discovery/vm.pug | 0 .../general/discovery/zookeeper.pug | 0 .../bounded-exponential-backoff.pug | 0 .../zookeeper/retrypolicy/custom.pug | 0 .../retrypolicy/exponential-backoff.pug | 0 .../zookeeper/retrypolicy/forever.pug | 0 .../zookeeper/retrypolicy/n-times.pug | 0 .../zookeeper/retrypolicy/one-time.pug | 0 .../zookeeper/retrypolicy/until-elapsed.pug | 0 .../cluster-edit-form/templates}/hadoop.pug | 0 .../cluster-edit-form/templates}/igfs.pug | 0 .../templates}/load-balancing.pug | 0 .../cluster-edit-form/templates}/logger.pug | 0 .../templates}/logger/custom.pug | 0 .../templates}/logger/log4j.pug | 0 .../templates}/logger/log4j2.pug | 0 .../templates}/marshaller.pug | 0 .../cluster-edit-form/templates}/memory.pug | 0 .../cluster-edit-form/templates}/metrics.pug | 0 .../cluster-edit-form/templates}/misc.pug | 0 .../cluster-edit-form/templates}/odbc.pug | 0 .../templates}/persistence.pug | 0 .../cluster-edit-form/templates}/service.pug | 0 .../templates}/sql-connector.pug | 0 .../cluster-edit-form/templates}/ssl.pug | 0 .../cluster-edit-form/templates}/swap.pug | 0 .../cluster-edit-form/templates}/thread.pug | 0 .../cluster-edit-form/templates}/time.pug | 0 .../templates}/transactions.pug | 0 .../igfs-edit-form/template.tpl.pug | 12 ++-- .../igfs-edit-form/templates}/dual.pug | 0 .../templates}/fragmentizer.pug | 0 .../igfs-edit-form/templates}/general.pug | 0 .../igfs-edit-form/templates}/ipc.pug | 0 .../igfs-edit-form/templates}/misc.pug | 0 .../igfs-edit-form/templates}/secondary.pug | 0 .../model-edit-form/template.tpl.pug | 6 +- .../model-edit-form/templates}/general.pug | 0 .../model-edit-form/templates}/query.pug | 0 .../model-edit-form/templates}/store.pug | 0 .../page-configure-basic/template.pug | 18 +++--- 83 files changed, 60 insertions(+), 60 deletions(-) rename modules/web-console/frontend/app/{modules/states/configuration/caches => components/page-configure-advanced/components/cache-edit-form/templates}/affinity.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/caches => components/page-configure-advanced/components/cache-edit-form/templates}/concurrency.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/caches => components/page-configure-advanced/components/cache-edit-form/templates}/general.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/caches => components/page-configure-advanced/components/cache-edit-form/templates}/memory.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/caches => components/page-configure-advanced/components/cache-edit-form/templates}/near-cache-client.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/caches => components/page-configure-advanced/components/cache-edit-form/templates}/near-cache-server.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/caches => components/page-configure-advanced/components/cache-edit-form/templates}/node-filter.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/caches => components/page-configure-advanced/components/cache-edit-form/templates}/query.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/caches => components/page-configure-advanced/components/cache-edit-form/templates}/rebalance.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/caches => components/page-configure-advanced/components/cache-edit-form/templates}/statistics.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/caches => components/page-configure-advanced/components/cache-edit-form/templates}/store.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/atomic.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/attributes.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/binary.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/cache-key-cfg.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/checkpoint.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/checkpoint/fs.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/checkpoint/jdbc.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/checkpoint/s3.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/client-connector.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/collision.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/collision/custom.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/collision/fifo-queue.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/collision/job-stealing.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/collision/priority-queue.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/communication.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/connector.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/data-storage.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/deployment.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/discovery.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/events.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/failover.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/general.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/general/discovery/cloud.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/general/discovery/google.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/general/discovery/jdbc.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/general/discovery/kubernetes.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/general/discovery/multicast.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/general/discovery/s3.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/general/discovery/shared.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/general/discovery/vm.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/general/discovery/zookeeper.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/general/discovery/zookeeper/retrypolicy/custom.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/general/discovery/zookeeper/retrypolicy/exponential-backoff.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/general/discovery/zookeeper/retrypolicy/forever.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/general/discovery/zookeeper/retrypolicy/n-times.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/general/discovery/zookeeper/retrypolicy/one-time.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/general/discovery/zookeeper/retrypolicy/until-elapsed.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/hadoop.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/igfs.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/load-balancing.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/logger.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/logger/custom.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/logger/log4j.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/logger/log4j2.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/marshaller.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/memory.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/metrics.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/misc.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/odbc.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/persistence.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/service.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/sql-connector.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/ssl.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/swap.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/thread.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/time.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/clusters => components/page-configure-advanced/components/cluster-edit-form/templates}/transactions.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/igfs => components/page-configure-advanced/components/igfs-edit-form/templates}/dual.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/igfs => components/page-configure-advanced/components/igfs-edit-form/templates}/fragmentizer.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/igfs => components/page-configure-advanced/components/igfs-edit-form/templates}/general.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/igfs => components/page-configure-advanced/components/igfs-edit-form/templates}/ipc.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/igfs => components/page-configure-advanced/components/igfs-edit-form/templates}/misc.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/igfs => components/page-configure-advanced/components/igfs-edit-form/templates}/secondary.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/domains => components/page-configure-advanced/components/model-edit-form/templates}/general.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/domains => components/page-configure-advanced/components/model-edit-form/templates}/query.pug (100%) rename modules/web-console/frontend/app/{modules/states/configuration/domains => components/page-configure-advanced/components/model-edit-form/templates}/store.pug (100%) diff --git a/modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/template.tpl.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/template.tpl.pug index 70cb445812596..a8ae2f2a93e2e 100644 --- a/modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/template.tpl.pug +++ b/modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/template.tpl.pug @@ -20,18 +20,18 @@ form( novalidate ng-submit='$ctrl.save($ctrl.clonedCache)' ) - include /app/modules/states/configuration/caches/general - include /app/modules/states/configuration/caches/memory - include /app/modules/states/configuration/caches/query - include /app/modules/states/configuration/caches/store + include ./templates/general + include ./templates/memory + include ./templates/query + include ./templates/store - include /app/modules/states/configuration/caches/affinity - include /app/modules/states/configuration/caches/concurrency - include /app/modules/states/configuration/caches/near-cache-client - include /app/modules/states/configuration/caches/near-cache-server - include /app/modules/states/configuration/caches/node-filter - include /app/modules/states/configuration/caches/rebalance - include /app/modules/states/configuration/caches/statistics + include ./templates/affinity + include ./templates/concurrency + include ./templates/near-cache-client + include ./templates/near-cache-server + include ./templates/node-filter + include ./templates/rebalance + include ./templates/statistics .pc-form-actions-panel .pc-form-actions-panel__right-after diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/affinity.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/affinity.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/caches/affinity.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/affinity.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/concurrency.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/concurrency.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/caches/concurrency.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/concurrency.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/general.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/general.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/caches/general.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/general.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/memory.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/memory.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/caches/memory.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/memory.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-client.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/near-cache-client.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-client.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/near-cache-client.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-server.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/near-cache-server.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-server.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/near-cache-server.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/node-filter.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/node-filter.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/query.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/query.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/caches/query.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/query.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/rebalance.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/rebalance.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/caches/rebalance.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/rebalance.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/statistics.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/statistics.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/caches/statistics.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/statistics.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/store.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/store.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/caches/store.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cache-edit-form/templates/store.pug diff --git a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/template.tpl.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/template.tpl.pug index 4dd0e17159867..c2bfd68a57668 100644 --- a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/template.tpl.pug +++ b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/template.tpl.pug @@ -18,57 +18,57 @@ include /app/helpers/jade/mixins form(id='cluster' name='ui.inputForm' novalidate ng-submit='$ctrl.save()') .panel-group - include /app/modules/states/configuration/clusters/general + include ./templates/general - include /app/modules/states/configuration/clusters/atomic - include /app/modules/states/configuration/clusters/binary - include /app/modules/states/configuration/clusters/cache-key-cfg - include /app/modules/states/configuration/clusters/checkpoint + include ./templates/atomic + include ./templates/binary + include ./templates/cache-key-cfg + include ./templates/checkpoint //- Since ignite 2.3 - include /app/modules/states/configuration/clusters/client-connector + include ./templates/client-connector - include /app/modules/states/configuration/clusters/collision - include /app/modules/states/configuration/clusters/communication - include /app/modules/states/configuration/clusters/connector - include /app/modules/states/configuration/clusters/deployment + include ./templates/collision + include ./templates/communication + include ./templates/connector + include ./templates/deployment //- Since ignite 2.3 - include /app/modules/states/configuration/clusters/data-storage + include ./templates/data-storage - include /app/modules/states/configuration/clusters/discovery - include /app/modules/states/configuration/clusters/events - include /app/modules/states/configuration/clusters/failover - include /app/modules/states/configuration/clusters/hadoop - include /app/modules/states/configuration/clusters/load-balancing - include /app/modules/states/configuration/clusters/logger - include /app/modules/states/configuration/clusters/marshaller + include ./templates/discovery + include ./templates/events + include ./templates/failover + include ./templates/hadoop + include ./templates/load-balancing + include ./templates/logger + include ./templates/marshaller //- Since ignite 2.0, deprecated in ignite 2.3 - include /app/modules/states/configuration/clusters/memory + include ./templates/memory - include /app/modules/states/configuration/clusters/misc - include /app/modules/states/configuration/clusters/metrics + include ./templates/misc + include ./templates/metrics //- Deprecated in ignite 2.1 - include /app/modules/states/configuration/clusters/odbc + include ./templates/odbc //- Since ignite 2.1, deprecated in ignite 2.3 - include /app/modules/states/configuration/clusters/persistence + include ./templates/persistence //- Deprecated in ignite 2.3 - include /app/modules/states/configuration/clusters/sql-connector + include ./templates/sql-connector - include /app/modules/states/configuration/clusters/service - include /app/modules/states/configuration/clusters/ssl + include ./templates/service + include ./templates/ssl //- Removed in ignite 2.0 - include /app/modules/states/configuration/clusters/swap + include ./templates/swap - include /app/modules/states/configuration/clusters/thread - include /app/modules/states/configuration/clusters/time - include /app/modules/states/configuration/clusters/transactions - include /app/modules/states/configuration/clusters/attributes + include ./templates/thread + include ./templates/time + include ./templates/transactions + include ./templates/attributes .pc-form-actions-panel(n_g-show='$ctrl.$scope.selectedItem') button-preview-project(cluster='$ctrl.cluster' ng-hide='$ctrl.isNew') diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/atomic.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/atomic.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/atomic.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/atomic.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/attributes.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/attributes.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/attributes.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/attributes.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/binary.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/binary.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/binary.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/binary.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/cache-key-cfg.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/cache-key-cfg.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/checkpoint.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/checkpoint.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/checkpoint/fs.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/checkpoint/fs.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/checkpoint/jdbc.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/checkpoint/jdbc.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/checkpoint/s3.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/checkpoint/s3.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/client-connector.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/client-connector.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/client-connector.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/client-connector.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/collision.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/collision.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/collision.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/collision/custom.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/collision/custom.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/fifo-queue.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/collision/fifo-queue.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/collision/fifo-queue.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/collision/fifo-queue.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/collision/job-stealing.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/collision/job-stealing.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/priority-queue.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/collision/priority-queue.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/collision/priority-queue.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/collision/priority-queue.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/communication.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/communication.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/communication.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/connector.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/connector.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/connector.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/connector.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/data-storage.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/data-storage.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/data-storage.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/data-storage.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/deployment.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/deployment.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/discovery.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/discovery.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/discovery.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/discovery.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/events.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/events.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/events.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/events.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/failover.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/failover.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/failover.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/general.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/cloud.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/cloud.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/cloud.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/cloud.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/google.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/google.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/google.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/google.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/jdbc.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/jdbc.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/jdbc.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/jdbc.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/kubernetes.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/kubernetes.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/kubernetes.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/kubernetes.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/multicast.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/multicast.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/multicast.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/multicast.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/s3.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/s3.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/s3.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/s3.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/shared.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/shared.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/shared.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/shared.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/vm.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/vm.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/vm.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/vm.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/zookeeper.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/zookeeper.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/zookeeper/retrypolicy/custom.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/zookeeper/retrypolicy/custom.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/zookeeper/retrypolicy/exponential-backoff.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/zookeeper/retrypolicy/exponential-backoff.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/zookeeper/retrypolicy/forever.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/zookeeper/retrypolicy/forever.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/zookeeper/retrypolicy/n-times.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/zookeeper/retrypolicy/n-times.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/zookeeper/retrypolicy/one-time.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/zookeeper/retrypolicy/one-time.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/zookeeper/retrypolicy/until-elapsed.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/general/discovery/zookeeper/retrypolicy/until-elapsed.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/hadoop.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/hadoop.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/hadoop.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/hadoop.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/igfs.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/igfs.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/igfs.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/igfs.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/load-balancing.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/load-balancing.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/logger.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/logger.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/logger.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/logger/custom.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/logger/custom.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/logger/log4j.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/logger/log4j.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j2.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/logger/log4j2.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j2.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/logger/log4j2.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/marshaller.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/marshaller.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/marshaller.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/marshaller.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/memory.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/memory.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/memory.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/memory.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/metrics.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/metrics.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/metrics.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/metrics.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/misc.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/misc.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/misc.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/misc.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/odbc.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/odbc.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/odbc.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/odbc.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/persistence.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/persistence.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/persistence.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/persistence.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/service.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/service.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/service.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/service.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/sql-connector.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/sql-connector.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/sql-connector.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/sql-connector.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/ssl.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/ssl.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/swap.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/swap.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/swap.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/swap.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/thread.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/thread.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/thread.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/thread.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/time.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/time.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/time.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/time.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/transactions.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/transactions.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/clusters/transactions.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/transactions.pug diff --git a/modules/web-console/frontend/app/components/page-configure-advanced/components/igfs-edit-form/template.tpl.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/igfs-edit-form/template.tpl.pug index fe8b2182f53b1..f505e589562df 100644 --- a/modules/web-console/frontend/app/components/page-configure-advanced/components/igfs-edit-form/template.tpl.pug +++ b/modules/web-console/frontend/app/components/page-configure-advanced/components/igfs-edit-form/template.tpl.pug @@ -15,15 +15,15 @@ limitations under the License. form(id='igfs' name='ui.inputForm' novalidate ng-submit='$ctrl.save()') - include /app/modules/states/configuration/igfs/general + include ./templates/general - include /app/modules/states/configuration/igfs/secondary - include /app/modules/states/configuration/igfs/ipc - include /app/modules/states/configuration/igfs/fragmentizer + include ./templates/secondary + include ./templates/ipc + include ./templates/fragmentizer //- Removed in ignite 2.0 - include /app/modules/states/configuration/igfs/dual - include /app/modules/states/configuration/igfs/misc + include ./templates/dual + include ./templates/misc .pc-form-actions-panel .pc-form-actions-panel__right-after diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/dual.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/igfs-edit-form/templates/dual.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/igfs/dual.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/igfs-edit-form/templates/dual.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/fragmentizer.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/igfs-edit-form/templates/fragmentizer.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/igfs/fragmentizer.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/igfs-edit-form/templates/fragmentizer.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/general.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/igfs-edit-form/templates/general.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/igfs/general.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/igfs-edit-form/templates/general.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/ipc.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/igfs-edit-form/templates/ipc.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/igfs/ipc.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/igfs-edit-form/templates/ipc.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/misc.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/igfs-edit-form/templates/misc.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/igfs/misc.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/igfs-edit-form/templates/misc.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/secondary.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/igfs-edit-form/templates/secondary.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/igfs/secondary.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/igfs-edit-form/templates/secondary.pug diff --git a/modules/web-console/frontend/app/components/page-configure-advanced/components/model-edit-form/template.tpl.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/model-edit-form/template.tpl.pug index 8ebd11cdc7466..78ae7693dcadb 100644 --- a/modules/web-console/frontend/app/components/page-configure-advanced/components/model-edit-form/template.tpl.pug +++ b/modules/web-console/frontend/app/components/page-configure-advanced/components/model-edit-form/template.tpl.pug @@ -15,9 +15,9 @@ limitations under the License. form(id='model' name='ui.inputForm' novalidate ng-submit='$ctrl.save()') - include /app/modules/states/configuration/domains/general - include /app/modules/states/configuration/domains/query - include /app/modules/states/configuration/domains/store + include ./templates/general + include ./templates/query + include ./templates/store .pc-form-actions-panel .pc-form-actions-panel__right-after diff --git a/modules/web-console/frontend/app/modules/states/configuration/domains/general.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/model-edit-form/templates/general.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/domains/general.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/model-edit-form/templates/general.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/domains/query.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/model-edit-form/templates/query.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/domains/query.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/model-edit-form/templates/query.pug diff --git a/modules/web-console/frontend/app/modules/states/configuration/domains/store.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/model-edit-form/templates/store.pug similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/domains/store.pug rename to modules/web-console/frontend/app/components/page-configure-advanced/components/model-edit-form/templates/store.pug diff --git a/modules/web-console/frontend/app/components/page-configure-basic/template.pug b/modules/web-console/frontend/app/components/page-configure-basic/template.pug index 7714c8144c12d..3bcb216dd2861 100644 --- a/modules/web-console/frontend/app/components/page-configure-basic/template.pug +++ b/modules/web-console/frontend/app/components/page-configure-basic/template.pug @@ -15,15 +15,15 @@ limitations under the License. include /app/helpers/jade/mixins -include /app/modules/states/configuration/clusters/general/discovery/cloud -include /app/modules/states/configuration/clusters/general/discovery/google -include /app/modules/states/configuration/clusters/general/discovery/jdbc -include /app/modules/states/configuration/clusters/general/discovery/multicast -include /app/modules/states/configuration/clusters/general/discovery/s3 -include /app/modules/states/configuration/clusters/general/discovery/shared -include /app/modules/states/configuration/clusters/general/discovery/vm -include /app/modules/states/configuration/clusters/general/discovery/zookeeper -include /app/modules/states/configuration/clusters/general/discovery/kubernetes +include ./../page-configure-advanced/components/cluster-edit-form/templates/general/discovery/cloud +include ./../page-configure-advanced/components/cluster-edit-form/templates/general/discovery/google +include ./../page-configure-advanced/components/cluster-edit-form/templates/general/discovery/jdbc +include ./../page-configure-advanced/components/cluster-edit-form/templates/general/discovery/multicast +include ./../page-configure-advanced/components/cluster-edit-form/templates/general/discovery/s3 +include ./../page-configure-advanced/components/cluster-edit-form/templates/general/discovery/shared +include ./../page-configure-advanced/components/cluster-edit-form/templates/general/discovery/vm +include ./../page-configure-advanced/components/cluster-edit-form/templates/general/discovery/zookeeper +include ./../page-configure-advanced/components/cluster-edit-form/templates/general/discovery/kubernetes - const model = '$ctrl.clonedCluster' - const modelDiscoveryKind = `${model}.discovery.kind` From 9955728a72e0f9c11faa313a521d4566b5a93dc1 Mon Sep 17 00:00:00 2001 From: Ilya Borisov Date: Fri, 6 Apr 2018 11:19:07 +0700 Subject: [PATCH 0031/1463] IGNITE-7996 Move config state module index. (cherry picked from commit d5e0be0) --- .../states => components/page-configure}/configuration.state.js | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename modules/web-console/frontend/app/{modules/states => components/page-configure}/configuration.state.js (100%) diff --git a/modules/web-console/frontend/app/modules/states/configuration.state.js b/modules/web-console/frontend/app/components/page-configure/configuration.state.js similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration.state.js rename to modules/web-console/frontend/app/components/page-configure/configuration.state.js From 536d8b2080b93505e67206a240385baced150674 Mon Sep 17 00:00:00 2001 From: Ilya Borisov Date: Fri, 6 Apr 2018 11:20:13 +0700 Subject: [PATCH 0032/1463] IGNITE-7996 Use configuration.state for state registration only. (cherry picked from commit 2800ef0) --- .../page-configure/configuration.state.js | 490 +++++++++--------- 1 file changed, 233 insertions(+), 257 deletions(-) diff --git a/modules/web-console/frontend/app/components/page-configure/configuration.state.js b/modules/web-console/frontend/app/components/page-configure/configuration.state.js index 1a0a598905888..f8bb4dc2457f1 100644 --- a/modules/web-console/frontend/app/components/page-configure/configuration.state.js +++ b/modules/web-console/frontend/app/components/page-configure/configuration.state.js @@ -17,20 +17,11 @@ import angular from 'angular'; -import {default as ActivitiesData} from 'app/core/activities/Activities.data'; - -// Common directives. -import previewPanel from './configuration/preview-panel.directive.js'; - -// Summary screen. -import ConfigurationResource from './configuration/Configuration.resource'; -import IgniteSummaryZipper from './configuration/summary/summary-zipper.service'; - import base2 from 'views/base2.pug'; -import pageConfigureAdvancedClusterComponent from 'app/components/page-configure-advanced/components/page-configure-advanced-cluster/component'; -import pageConfigureAdvancedModelsComponent from 'app/components/page-configure-advanced/components/page-configure-advanced-models/component'; -import pageConfigureAdvancedCachesComponent from 'app/components/page-configure-advanced/components/page-configure-advanced-caches/component'; -import pageConfigureAdvancedIGFSComponent from 'app/components/page-configure-advanced/components/page-configure-advanced-igfs/component'; +import pageConfigureAdvancedClusterComponent from '../page-configure-advanced/components/page-configure-advanced-cluster/component'; +import pageConfigureAdvancedModelsComponent from '../page-configure-advanced/components/page-configure-advanced-models/component'; +import pageConfigureAdvancedCachesComponent from '../page-configure-advanced/components/page-configure-advanced-caches/component'; +import pageConfigureAdvancedIGFSComponent from '../page-configure-advanced/components/page-configure-advanced-igfs/component'; import get from 'lodash/get'; import {Observable} from 'rxjs/Observable'; @@ -47,251 +38,236 @@ const shortCachesResolve = ['ConfigSelectors', 'ConfigureState', 'ConfigEffects' .toPromise(); }]; -/** - * @param {ActivitiesData} ActivitiesData - * @param {uirouter.UIRouter} $uiRouter - */ -function initConfiguration(ActivitiesData, $uiRouter) { - $uiRouter.transitionService.onSuccess({to: 'base.configuration.**'}, (transition) => { - ActivitiesData.post({group: 'configuration', action: transition.targetState().name()}); +function registerStates($stateProvider) { + // Setup the states. + $stateProvider + .state('base.configuration', { + abstract: true, + permission: 'configuration', + url: '/configuration', + onEnter: ['ConfigureState', (ConfigureState) => ConfigureState.dispatchAction({type: 'PRELOAD_STATE', state: {}})], + views: { + '@': { + template: base2 + } + }, + resolve: { + _shortClusters: ['ConfigEffects', ({etp}) => { + return etp('LOAD_USER_CLUSTERS'); + }] + }, + resolvePolicy: { + async: 'NOWAIT' + } + }) + .state('base.configuration.overview', { + url: '/overview', + component: 'pageConfigureOverview', + permission: 'configuration', + tfMetaTags: { + title: 'Configuration' + } + }) + .state('base.configuration.edit', { + url: `/{clusterID:${idRegex}}`, + permission: 'configuration', + component: 'pageConfigure', + resolve: { + _cluster: ['ConfigEffects', '$transition$', ({etp}, $transition$) => { + return $transition$.injector().getAsync('_shortClusters').then(() => { + return etp('LOAD_AND_EDIT_CLUSTER', {clusterID: $transition$.params().clusterID}); + }); + }] + }, + data: { + errorState: 'base.configuration.overview' + }, + redirectTo: ($transition$) => { + const [ConfigureState, ConfigSelectors] = ['ConfigureState', 'ConfigSelectors'].map((t) => $transition$.injector().get(t)); + const waitFor = ['_cluster', '_shortClusters'].map((t) => $transition$.injector().getAsync(t)); + return Observable.fromPromise(Promise.all(waitFor)).switchMap(() => { + return Observable.combineLatest( + ConfigureState.state$.let(ConfigSelectors.selectCluster($transition$.params().clusterID)).take(1), + ConfigureState.state$.let(ConfigSelectors.selectShortClusters()).take(1) + ); + }) + .map(([cluster = {caches: []}, clusters]) => { + return (clusters.value.size > 10 || cluster.caches.length > 5) + ? 'base.configuration.edit.advanced' + : 'base.configuration.edit.basic'; + }) + .toPromise(); + }, + failState: 'signin', + tfMetaTags: { + title: 'Configuration' + } + }) + .state('base.configuration.edit.basic', { + url: '/basic', + component: 'pageConfigureBasic', + permission: 'configuration', + resolve: { + _shortCaches: shortCachesResolve + }, + resolvePolicy: { + async: 'NOWAIT' + }, + tfMetaTags: { + title: 'Basic Configuration' + } + }) + .state('base.configuration.edit.advanced', { + url: '/advanced', + component: 'pageConfigureAdvanced', + permission: 'configuration', + redirectTo: 'base.configuration.edit.advanced.cluster' + }) + .state('base.configuration.edit.advanced.cluster', { + url: '/cluster', + component: pageConfigureAdvancedClusterComponent.name, + permission: 'configuration', + resolve: { + _shortCaches: shortCachesResolve + }, + resolvePolicy: { + async: 'NOWAIT' + }, + tfMetaTags: { + title: 'Configure Cluster' + } + }) + .state('base.configuration.edit.advanced.caches', { + url: '/caches', + permission: 'configuration', + component: pageConfigureAdvancedCachesComponent.name, + resolve: { + _shortCachesAndModels: ['ConfigSelectors', 'ConfigureState', 'ConfigEffects', '$transition$', (ConfigSelectors, ConfigureState, {etp}, $transition$) => { + if ($transition$.params().clusterID === 'new') return Promise.resolve(); + return Observable.fromPromise($transition$.injector().getAsync('_cluster')) + .switchMap(() => ConfigureState.state$.let(ConfigSelectors.selectCluster($transition$.params().clusterID)).take(1)) + .map((cluster) => { + return Promise.all([ + etp('LOAD_SHORT_CACHES', {ids: cluster.caches, clusterID: cluster._id}), + etp('LOAD_SHORT_MODELS', {ids: cluster.models, clusterID: cluster._id}), + etp('LOAD_SHORT_IGFSS', {ids: cluster.igfss, clusterID: cluster._id}) + ]); + }) + .toPromise(); + }] + }, + resolvePolicy: { + async: 'NOWAIT' + }, + tfMetaTags: { + title: 'Configure Caches' + } + }) + .state('base.configuration.edit.advanced.caches.cache', { + url: `/{cacheID:${idRegex}}`, + permission: 'configuration', + resolve: { + _cache: ['ConfigEffects', '$transition$', ({etp}, $transition$) => { + const {clusterID, cacheID} = $transition$.params(); + if (cacheID === 'new') return Promise.resolve(); + return etp('LOAD_CACHE', {cacheID}); + }] + }, + data: { + errorState: 'base.configuration.edit.advanced.caches' + }, + resolvePolicy: { + async: 'NOWAIT' + }, + tfMetaTags: { + title: 'Configure Caches' + } + }) + .state('base.configuration.edit.advanced.models', { + url: '/models', + component: pageConfigureAdvancedModelsComponent.name, + permission: 'configuration', + resolve: { + _shortCachesAndModels: ['ConfigSelectors', 'ConfigureState', 'ConfigEffects', '$transition$', (ConfigSelectors, ConfigureState, {etp}, $transition$) => { + if ($transition$.params().clusterID === 'new') return Promise.resolve(); + return Observable.fromPromise($transition$.injector().getAsync('_cluster')) + .switchMap(() => ConfigureState.state$.let(ConfigSelectors.selectCluster($transition$.params().clusterID)).take(1)) + .map((cluster) => { + return Promise.all([ + etp('LOAD_SHORT_CACHES', {ids: cluster.caches, clusterID: cluster._id}), + etp('LOAD_SHORT_MODELS', {ids: cluster.models, clusterID: cluster._id}) + ]); + }) + .toPromise(); + }] + }, + resolvePolicy: { + async: 'NOWAIT' + }, + tfMetaTags: { + title: 'Configure SQL Schemes' + } + }) + .state('base.configuration.edit.advanced.models.model', { + url: `/{modelID:${idRegex}}`, + resolve: { + _cache: ['ConfigEffects', '$transition$', ({etp}, $transition$) => { + const {clusterID, modelID} = $transition$.params(); + if (modelID === 'new') return Promise.resolve(); + return etp('LOAD_MODEL', {modelID}); + }] + }, + data: { + errorState: 'base.configuration.edit.advanced.models' + }, + permission: 'configuration', + resolvePolicy: { + async: 'NOWAIT' + } + }) + .state('base.configuration.edit.advanced.igfs', { + url: '/igfs', + component: pageConfigureAdvancedIGFSComponent.name, + permission: 'configuration', + resolve: { + _shortIGFSs: ['ConfigSelectors', 'ConfigureState', 'ConfigEffects', '$transition$', (ConfigSelectors, ConfigureState, {etp}, $transition$) => { + if ($transition$.params().clusterID === 'new') return Promise.resolve(); + return Observable.fromPromise($transition$.injector().getAsync('_cluster')) + .switchMap(() => ConfigureState.state$.let(ConfigSelectors.selectCluster($transition$.params().clusterID)).take(1)) + .map((cluster) => { + return Promise.all([ + etp('LOAD_SHORT_IGFSS', {ids: cluster.igfss, clusterID: cluster._id}) + ]); + }) + .toPromise(); + }] + }, + resolvePolicy: { + async: 'NOWAIT' + }, + tfMetaTags: { + title: 'Configure IGFS' + } + }) + .state('base.configuration.edit.advanced.igfs.igfs', { + url: `/{igfsID:${idRegex}}`, + permission: 'configuration', + resolve: { + _igfs: ['ConfigEffects', '$transition$', ({etp}, $transition$) => { + const {clusterID, igfsID} = $transition$.params(); + if (igfsID === 'new') return Promise.resolve(); + return etp('LOAD_IGFS', {igfsID}); + }] + }, + data: { + errorState: 'base.configuration.edit.advanced.igfs' + }, + resolvePolicy: { + async: 'NOWAIT' + } }); } -initConfiguration.$inject = ['IgniteActivitiesData', '$uiRouter']; +registerStates.$inject = ['$stateProvider']; -angular.module('ignite-console.states.configuration', ['ui.router']) - .directive(...previewPanel) - // Services. - .service('IgniteSummaryZipper', IgniteSummaryZipper) - .service('IgniteConfigurationResource', ConfigurationResource) - .run(initConfiguration) - // Configure state provider. - .config(['$stateProvider', ($stateProvider) => { - // Setup the states. - $stateProvider - .state('base.configuration', { - abstract: true, - permission: 'configuration', - url: '/configuration', - onEnter: ['ConfigureState', (ConfigureState) => ConfigureState.dispatchAction({type: 'PRELOAD_STATE', state: {}})], - views: { - '@': { - template: base2 - } - }, - resolve: { - _shortClusters: ['ConfigEffects', ({etp}) => { - return etp('LOAD_USER_CLUSTERS'); - }] - }, - resolvePolicy: { - async: 'NOWAIT' - } - }) - .state('base.configuration.overview', { - url: '/overview', - component: 'pageConfigureOverview', - permission: 'configuration', - tfMetaTags: { - title: 'Configuration' - } - }) - .state('base.configuration.edit', { - url: `/{clusterID:${idRegex}}`, - permission: 'configuration', - component: 'pageConfigure', - resolve: { - _cluster: ['ConfigEffects', '$transition$', ({etp}, $transition$) => { - return $transition$.injector().getAsync('_shortClusters').then(() => { - return etp('LOAD_AND_EDIT_CLUSTER', {clusterID: $transition$.params().clusterID}); - }); - }] - }, - data: { - errorState: 'base.configuration.overview' - }, - redirectTo: ($transition$) => { - const [ConfigureState, ConfigSelectors] = ['ConfigureState', 'ConfigSelectors'].map((t) => $transition$.injector().get(t)); - const waitFor = ['_cluster', '_shortClusters'].map((t) => $transition$.injector().getAsync(t)); - return Observable.fromPromise(Promise.all(waitFor)).switchMap(() => { - return Observable.combineLatest( - ConfigureState.state$.let(ConfigSelectors.selectCluster($transition$.params().clusterID)).take(1), - ConfigureState.state$.let(ConfigSelectors.selectShortClusters()).take(1) - ); - }) - .map(([cluster = {caches: []}, clusters]) => { - return (clusters.value.size > 10 || cluster.caches.length > 5) - ? 'base.configuration.edit.advanced' - : 'base.configuration.edit.basic'; - }) - .toPromise(); - }, - failState: 'signin', - tfMetaTags: { - title: 'Configuration' - } - }) - .state('base.configuration.edit.basic', { - url: '/basic', - component: 'pageConfigureBasic', - permission: 'configuration', - resolve: { - _shortCaches: shortCachesResolve - }, - resolvePolicy: { - async: 'NOWAIT' - }, - tfMetaTags: { - title: 'Basic Configuration' - } - }) - .state('base.configuration.edit.advanced', { - url: '/advanced', - component: 'pageConfigureAdvanced', - permission: 'configuration', - redirectTo: 'base.configuration.edit.advanced.cluster' - }) - .state('base.configuration.edit.advanced.cluster', { - url: '/cluster', - component: pageConfigureAdvancedClusterComponent.name, - permission: 'configuration', - resolve: { - _shortCaches: shortCachesResolve - }, - resolvePolicy: { - async: 'NOWAIT' - }, - tfMetaTags: { - title: 'Configure Cluster' - } - }) - .state('base.configuration.edit.advanced.caches', { - url: '/caches', - permission: 'configuration', - component: pageConfigureAdvancedCachesComponent.name, - resolve: { - _shortCachesAndModels: ['ConfigSelectors', 'ConfigureState', 'ConfigEffects', '$transition$', (ConfigSelectors, ConfigureState, {etp}, $transition$) => { - if ($transition$.params().clusterID === 'new') return Promise.resolve(); - return Observable.fromPromise($transition$.injector().getAsync('_cluster')) - .switchMap(() => ConfigureState.state$.let(ConfigSelectors.selectCluster($transition$.params().clusterID)).take(1)) - .map((cluster) => { - return Promise.all([ - etp('LOAD_SHORT_CACHES', {ids: cluster.caches, clusterID: cluster._id}), - etp('LOAD_SHORT_MODELS', {ids: cluster.models, clusterID: cluster._id}), - etp('LOAD_SHORT_IGFSS', {ids: cluster.igfss, clusterID: cluster._id}) - ]); - }) - .toPromise(); - }] - }, - resolvePolicy: { - async: 'NOWAIT' - }, - tfMetaTags: { - title: 'Configure Caches' - } - }) - .state('base.configuration.edit.advanced.caches.cache', { - url: `/{cacheID:${idRegex}}`, - permission: 'configuration', - resolve: { - _cache: ['ConfigEffects', '$transition$', ({etp}, $transition$) => { - const {clusterID, cacheID} = $transition$.params(); - if (cacheID === 'new') return Promise.resolve(); - return etp('LOAD_CACHE', {cacheID}); - }] - }, - data: { - errorState: 'base.configuration.edit.advanced.caches' - }, - resolvePolicy: { - async: 'NOWAIT' - }, - tfMetaTags: { - title: 'Configure Caches' - } - }) - .state('base.configuration.edit.advanced.models', { - url: '/models', - component: pageConfigureAdvancedModelsComponent.name, - permission: 'configuration', - resolve: { - _shortCachesAndModels: ['ConfigSelectors', 'ConfigureState', 'ConfigEffects', '$transition$', (ConfigSelectors, ConfigureState, {etp}, $transition$) => { - if ($transition$.params().clusterID === 'new') return Promise.resolve(); - return Observable.fromPromise($transition$.injector().getAsync('_cluster')) - .switchMap(() => ConfigureState.state$.let(ConfigSelectors.selectCluster($transition$.params().clusterID)).take(1)) - .map((cluster) => { - return Promise.all([ - etp('LOAD_SHORT_CACHES', {ids: cluster.caches, clusterID: cluster._id}), - etp('LOAD_SHORT_MODELS', {ids: cluster.models, clusterID: cluster._id}) - ]); - }) - .toPromise(); - }] - }, - resolvePolicy: { - async: 'NOWAIT' - }, - tfMetaTags: { - title: 'Configure SQL Schemes' - } - }) - .state('base.configuration.edit.advanced.models.model', { - url: `/{modelID:${idRegex}}`, - resolve: { - _cache: ['ConfigEffects', '$transition$', ({etp}, $transition$) => { - const {clusterID, modelID} = $transition$.params(); - if (modelID === 'new') return Promise.resolve(); - return etp('LOAD_MODEL', {modelID}); - }] - }, - data: { - errorState: 'base.configuration.edit.advanced.models' - }, - permission: 'configuration', - resolvePolicy: { - async: 'NOWAIT' - } - }) - .state('base.configuration.edit.advanced.igfs', { - url: '/igfs', - component: pageConfigureAdvancedIGFSComponent.name, - permission: 'configuration', - resolve: { - _shortIGFSs: ['ConfigSelectors', 'ConfigureState', 'ConfigEffects', '$transition$', (ConfigSelectors, ConfigureState, {etp}, $transition$) => { - if ($transition$.params().clusterID === 'new') return Promise.resolve(); - return Observable.fromPromise($transition$.injector().getAsync('_cluster')) - .switchMap(() => ConfigureState.state$.let(ConfigSelectors.selectCluster($transition$.params().clusterID)).take(1)) - .map((cluster) => { - return Promise.all([ - etp('LOAD_SHORT_IGFSS', {ids: cluster.igfss, clusterID: cluster._id}) - ]); - }) - .toPromise(); - }] - }, - resolvePolicy: { - async: 'NOWAIT' - }, - tfMetaTags: { - title: 'Configure IGFS' - } - }) - .state('base.configuration.edit.advanced.igfs.igfs', { - url: `/{igfsID:${idRegex}}`, - permission: 'configuration', - resolve: { - _igfs: ['ConfigEffects', '$transition$', ({etp}, $transition$) => { - const {clusterID, igfsID} = $transition$.params(); - if (igfsID === 'new') return Promise.resolve(); - return etp('LOAD_IGFS', {igfsID}); - }] - }, - data: { - errorState: 'base.configuration.edit.advanced.igfs' - }, - resolvePolicy: { - async: 'NOWAIT' - } - }); - }]); +export {registerStates}; From a2d9f97c0ae77e81344849bc2c3cba2d3964cf01 Mon Sep 17 00:00:00 2001 From: Ilya Borisov Date: Fri, 6 Apr 2018 11:20:39 +0700 Subject: [PATCH 0033/1463] IGNITE-7996 Rename configuration.state to states. (cherry picked from commit 14dd2df) --- .../page-configure/{configuration.state.js => states.js} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename modules/web-console/frontend/app/components/page-configure/{configuration.state.js => states.js} (100%) diff --git a/modules/web-console/frontend/app/components/page-configure/configuration.state.js b/modules/web-console/frontend/app/components/page-configure/states.js similarity index 100% rename from modules/web-console/frontend/app/components/page-configure/configuration.state.js rename to modules/web-console/frontend/app/components/page-configure/states.js From 1d5f45b4eb900a3c4cc0bedb8919b35f2a0032c8 Mon Sep 17 00:00:00 2001 From: Ilya Borisov Date: Fri, 6 Apr 2018 11:22:20 +0700 Subject: [PATCH 0034/1463] IGNITE-7996 Move configuration assets into page-configure module. (cherry picked from commit d02e87b) --- modules/web-console/frontend/app/app.js | 2 -- .../components/preview-panel/directive.js} | 11 ++++++-- .../components/preview-panel/index.js | 23 +++++++++++++++++ .../app/components/page-configure/index.js | 25 ++++++++++++++++++- .../services/ConfigurationResource.js} | 11 ++++++-- .../page-configure/services/SummaryZipper.js} | 9 +++++-- .../services}/summary.worker.js | 0 7 files changed, 72 insertions(+), 9 deletions(-) rename modules/web-console/frontend/app/{modules/states/configuration/preview-panel.directive.js => components/page-configure/components/preview-panel/directive.js} (96%) create mode 100644 modules/web-console/frontend/app/components/page-configure/components/preview-panel/index.js rename modules/web-console/frontend/app/{modules/states/configuration/Configuration.resource.js => components/page-configure/services/ConfigurationResource.js} (90%) rename modules/web-console/frontend/app/{modules/states/configuration/summary/summary-zipper.service.js => components/page-configure/services/SummaryZipper.js} (90%) rename modules/web-console/frontend/app/{modules/states/configuration/summary => components/page-configure/services}/summary.worker.js (100%) diff --git a/modules/web-console/frontend/app/app.js b/modules/web-console/frontend/app/app.js index 757be229e55bc..692acc54f3d64 100644 --- a/modules/web-console/frontend/app/app.js +++ b/modules/web-console/frontend/app/app.js @@ -27,7 +27,6 @@ import './modules/nodes/nodes.module'; import './modules/demo/Demo.module'; import './modules/states/logout.state'; -import './modules/states/configuration.state'; import './modules/states/admin.state'; import './modules/states/errors.state'; @@ -192,7 +191,6 @@ angular.module('ignite-console', [ 'ignite-console.demo', // States. 'ignite-console.states.logout', - 'ignite-console.states.configuration', 'ignite-console.states.admin', 'ignite-console.states.errors', // Common modules. diff --git a/modules/web-console/frontend/app/modules/states/configuration/preview-panel.directive.js b/modules/web-console/frontend/app/components/page-configure/components/preview-panel/directive.js similarity index 96% rename from modules/web-console/frontend/app/modules/states/configuration/preview-panel.directive.js rename to modules/web-console/frontend/app/components/page-configure/components/preview-panel/directive.js index be7bf1ed52ec4..b7519ce1804cb 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/preview-panel.directive.js +++ b/modules/web-console/frontend/app/components/page-configure/components/preview-panel/directive.js @@ -16,8 +16,13 @@ */ import ace from 'brace'; +import _ from 'lodash'; -export default ['previewPanel', ['$interval', '$timeout', ($interval, $timeout) => { +/** + * @param {ng.IIntervalService} $interval + * @param {ng.ITimeoutService} $timeout + */ +export default function previewPanelDirective($interval, $timeout) { let animation = {editor: null, stage: 0, start: 0, stop: 0}; let prevContent = []; @@ -236,4 +241,6 @@ export default ['previewPanel', ['$interval', '$timeout', ($interval, $timeout) link, require: ['?igniteUiAceTabs', '?^igniteUiAceTabs'] }; -}]]; +} + +previewPanelDirective.$inject = ['$interval', '$timeout']; diff --git a/modules/web-console/frontend/app/components/page-configure/components/preview-panel/index.js b/modules/web-console/frontend/app/components/page-configure/components/preview-panel/index.js new file mode 100644 index 0000000000000..ff1367b2dbe17 --- /dev/null +++ b/modules/web-console/frontend/app/components/page-configure/components/preview-panel/index.js @@ -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. + */ + +import angular from 'angular'; +import directive from './directive'; + +export default angular + .module('ignite-console.page-configure.preview-panel', []) + .directive('previewPanel', directive); diff --git a/modules/web-console/frontend/app/components/page-configure/index.js b/modules/web-console/frontend/app/components/page-configure/index.js index 5874df51dcd73..3209edeb4094f 100644 --- a/modules/web-console/frontend/app/components/page-configure/index.js +++ b/modules/web-console/frontend/app/components/page-configure/index.js @@ -28,6 +28,8 @@ import PageConfigure from './services/PageConfigure'; import ConfigurationDownload from './services/ConfigurationDownload'; import ConfigChangesGuard from './services/ConfigChangesGuard'; import ConfigSelectionManager from './services/ConfigSelectionManager'; +import SummaryZipper from './services/SummaryZipper'; +import ConfigurationResource from './services/ConfigurationResource'; import selectors from './store/selectors'; import effects from './store/effects'; @@ -43,8 +45,10 @@ import modalImportModels from './components/modal-import-models'; import buttonImportModels from './components/button-import-models'; import buttonDownloadProject from './components/button-download-project'; import buttonPreviewProject from './components/button-preview-project'; +import previewPanel from './components/preview-panel'; import {errorState} from './transitionHooks/errorState'; +import {default as ActivitiesData} from 'app/core/activities/Activities.data'; import 'rxjs/add/operator/withLatestFrom'; import 'rxjs/add/operator/skip'; @@ -73,10 +77,24 @@ import { shortIGFSsActionTypes, refsReducer } from './reducer'; + import {reducer as reduxDevtoolsReducer, devTools} from './reduxDevtoolsIntegration'; +import {registerStates} from './states'; + +/** + * @param {ActivitiesData} ActivitiesData + * @param {uirouter.UIRouter} $uiRouter + */ +function registerActivitiesHook(ActivitiesData, $uiRouter) { + $uiRouter.transitionService.onSuccess({to: 'base.configuration.**'}, (transition) => { + ActivitiesData.post({group: 'configuration', action: transition.targetState().name()}); + }); +} +registerActivitiesHook.$inject = ['IgniteActivitiesData', '$uiRouter']; export default angular .module('ignite-console.page-configure', [ + 'ui.router', 'asyncFilter', uiValidate, pcFormFieldSize.name, @@ -87,11 +105,14 @@ export default angular modalImportModels.name, buttonImportModels.name, buttonDownloadProject.name, - buttonPreviewProject.name + buttonPreviewProject.name, + previewPanel.name ]) + .config(registerStates) .config(['DefaultStateProvider', (DefaultState) => { DefaultState.setRedirectTo(() => 'base.configuration.overview'); }]) + .run(registerActivitiesHook) .run(['ConfigEffects', 'ConfigureState', '$uiRouter', (ConfigEffects, ConfigureState, $uiRouter) => { $uiRouter.plugin(UIRouterRx); // $uiRouter.plugin(Visualizer); @@ -153,6 +174,8 @@ export default angular .directive(fakeUiCanExit.name, fakeUiCanExit) .directive(formUICanExitGuard.name, formUICanExitGuard) .factory('configSelectionManager', ConfigSelectionManager) + .service('IgniteSummaryZipper', SummaryZipper) + .service('IgniteConfigurationResource', ConfigurationResource) .service('ConfigSelectors', selectors) .service('ConfigEffects', effects) .service('ConfigChangesGuard', ConfigChangesGuard) diff --git a/modules/web-console/frontend/app/modules/states/configuration/Configuration.resource.js b/modules/web-console/frontend/app/components/page-configure/services/ConfigurationResource.js similarity index 90% rename from modules/web-console/frontend/app/modules/states/configuration/Configuration.resource.js rename to modules/web-console/frontend/app/components/page-configure/services/ConfigurationResource.js index 0582d5c6e8f76..2dab8a3d2334e 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/Configuration.resource.js +++ b/modules/web-console/frontend/app/components/page-configure/services/ConfigurationResource.js @@ -15,7 +15,12 @@ * limitations under the License. */ -export default ['$http', ($http) => { +import _ from 'lodash'; + +/** + * @param {ng.IHttpService} $http + */ +export default function ConfigurationResourceService($http) { return { read() { return $http.get('/api/v1/configuration/list') @@ -39,4 +44,6 @@ export default ['$http', ($http) => { return Promise.resolve({spaces, clusters, caches, igfss, domains}); } }; -}]; +} + +ConfigurationResourceService.$inject = ['$http']; diff --git a/modules/web-console/frontend/app/modules/states/configuration/summary/summary-zipper.service.js b/modules/web-console/frontend/app/components/page-configure/services/SummaryZipper.js similarity index 90% rename from modules/web-console/frontend/app/modules/states/configuration/summary/summary-zipper.service.js rename to modules/web-console/frontend/app/components/page-configure/services/SummaryZipper.js index 119fb529e8e3f..8bc16b46203ed 100644 --- a/modules/web-console/frontend/app/modules/states/configuration/summary/summary-zipper.service.js +++ b/modules/web-console/frontend/app/components/page-configure/services/SummaryZipper.js @@ -17,7 +17,10 @@ import Worker from './summary.worker'; -export default ['$q', function($q) { +/** + * @param {ng.IQService} $q + */ +export default function SummaryZipperService($q) { return function(message) { const defer = $q.defer(); const worker = new Worker(); @@ -36,4 +39,6 @@ export default ['$q', function($q) { return defer.promise; }; -}]; +} + +SummaryZipperService.$inject = ['$q']; diff --git a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js b/modules/web-console/frontend/app/components/page-configure/services/summary.worker.js similarity index 100% rename from modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js rename to modules/web-console/frontend/app/components/page-configure/services/summary.worker.js From a5a83d211a572b8419866fabbc31975168157729 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Thu, 12 Apr 2018 11:21:23 +0700 Subject: [PATCH 0035/1463] IGNITE-7996 Merge with master. --- .../frontend/app/components/page-configure/states.js | 3 --- 1 file changed, 3 deletions(-) diff --git a/modules/web-console/frontend/app/components/page-configure/states.js b/modules/web-console/frontend/app/components/page-configure/states.js index f8bb4dc2457f1..a75e851174ffe 100644 --- a/modules/web-console/frontend/app/components/page-configure/states.js +++ b/modules/web-console/frontend/app/components/page-configure/states.js @@ -15,15 +15,12 @@ * limitations under the License. */ -import angular from 'angular'; - import base2 from 'views/base2.pug'; import pageConfigureAdvancedClusterComponent from '../page-configure-advanced/components/page-configure-advanced-cluster/component'; import pageConfigureAdvancedModelsComponent from '../page-configure-advanced/components/page-configure-advanced-models/component'; import pageConfigureAdvancedCachesComponent from '../page-configure-advanced/components/page-configure-advanced-caches/component'; import pageConfigureAdvancedIGFSComponent from '../page-configure-advanced/components/page-configure-advanced-igfs/component'; -import get from 'lodash/get'; import {Observable} from 'rxjs/Observable'; const idRegex = `new|[a-z0-9]+`; From dbf2d722b0563a9637cc72fe2fcdf3dbd07291fc Mon Sep 17 00:00:00 2001 From: devozerov Date: Thu, 12 Apr 2018 10:37:36 +0300 Subject: [PATCH 0036/1463] IGNITE-8042: .NET thin client: authentication support. This closes #3790. --- .../client/ClientAuthenticationException.java | 16 +- .../client/thin/TcpClientChannel.java | 9 +- .../odbc/ClientListenerNioListener.java | 19 +++ .../client/ClientConnectionContext.java | 8 +- .../platform/client/ClientStatus.java | 5 +- .../Client/ClientConnectionTest.cs | 161 ++++++++++++++++++ .../Client/ClientStatusCode.cs | 12 +- .../Client/IgniteClientConfiguration.cs | 13 ++ .../IgniteClientConfigurationSection.xsd | 10 ++ .../Impl/Client/ClientProtocolVersion.cs | 22 ++- .../Impl/Client/ClientSocket.cs | 85 ++++++++- 11 files changed, 329 insertions(+), 31 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java b/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java index 0c24db8880a4f..526690a69c8b0 100644 --- a/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java +++ b/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java @@ -24,22 +24,10 @@ public class ClientAuthenticationException extends ClientException { /** Serial version uid. */ private static final long serialVersionUID = 0L; - /** Message. */ - private static final String MSG = "Invalid user name or password"; - /** * Default constructor. */ - public ClientAuthenticationException() { - super(MSG); - } - - /** - * Constructs a new exception with the specified cause. - * - * @param cause the cause. - */ - public ClientAuthenticationException(Throwable cause) { - super(MSG, cause); + public ClientAuthenticationException(String msg) { + super(msg); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java index 8e8294f1ef6f2..10dc8652071f9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java @@ -272,8 +272,13 @@ private void handshakeRes(String user, String pwd) try (BinaryReaderExImpl r = new BinaryReaderExImpl(null, res, null, true)) { String err = r.readString(); - if (err != null && err.toUpperCase().matches(".*USER.*INCORRECT.*")) - throw new ClientAuthenticationException(); + int errCode = ClientStatus.FAILED; + + if (res.remaining() > 0) + errCode = r.readInt(); + + if (errCode == ClientStatus.AUTH_FAILED) + throw new ClientAuthenticationException(err); else if (ver.equals(srvVer)) throw new ClientProtocolError(err); else if (!supportedVers.contains(srvVer) || diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java index 53b14d7cdb268..407c1a02efc34 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java @@ -26,9 +26,11 @@ import org.apache.ignite.internal.binary.streams.BinaryHeapInputStream; import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream; import org.apache.ignite.internal.binary.streams.BinaryInputStream; +import org.apache.ignite.internal.processors.authentication.IgniteAccessControlException; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext; import org.apache.ignite.internal.processors.odbc.odbc.OdbcConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; +import org.apache.ignite.internal.processors.platform.client.ClientStatus; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.nio.GridNioServerListenerAdapter; import org.apache.ignite.internal.util.nio.GridNioSession; @@ -228,6 +230,18 @@ private void onHandshake(GridNioSession ses, byte[] msg) { connCtx.handler().writeHandshake(writer); } + catch (IgniteAccessControlException authEx) { + writer.writeBoolean(false); + + writer.writeShort((short)0); + writer.writeShort((short)0); + writer.writeShort((short)0); + + writer.doWriteString(authEx.getMessage()); + + if (ver.compareTo(ClientConnectionContext.VER_1_1_0) >= 0) + writer.writeInt(ClientStatus.AUTH_FAILED); + } catch (IgniteCheckedException e) { U.warn(log, "Error during handshake [rmtAddr=" + ses.remoteAddress() + ", msg=" + e.getMessage() + ']'); @@ -239,10 +253,15 @@ private void onHandshake(GridNioSession ses, byte[] msg) { currVer = connCtx.currentVersion(); writer.writeBoolean(false); + writer.writeShort(currVer.major()); writer.writeShort(currVer.minor()); writer.writeShort(currVer.maintenance()); + writer.doWriteString(e.getMessage()); + + if (ver.compareTo(ClientConnectionContext.VER_1_1_0) >= 0) + writer.writeInt(ClientStatus.FAILED); } ses.send(writer.array()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java index 061aab32c0459..056ea8306fa53 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java @@ -44,10 +44,10 @@ */ public class ClientConnectionContext implements ClientListenerConnectionContext { /** Version 1.0.0. */ - private static final ClientListenerProtocolVersion VER_1_0_0 = ClientListenerProtocolVersion.create(1, 0, 0); + public static final ClientListenerProtocolVersion VER_1_0_0 = ClientListenerProtocolVersion.create(1, 0, 0); /** Version 1.1.0. */ - private static final ClientListenerProtocolVersion VER_1_1_0 = ClientListenerProtocolVersion.create(1, 1, 0); + public static final ClientListenerProtocolVersion VER_1_1_0 = ClientListenerProtocolVersion.create(1, 1, 0); /** Supported versions. */ private static final Collection SUPPORTED_VERS = Arrays.asList(VER_1_1_0, VER_1_0_0); @@ -144,12 +144,12 @@ public GridKernalContext kernalContext() { authCtx = thirdPartyAuthentication(user, pwd).authorizationContext(); else if (kernalCtx.authentication().enabled()) { if (user == null || user.length() == 0) - throw new IgniteCheckedException("Unauthenticated sessions are prohibited."); + throw new IgniteAccessControlException("Unauthenticated sessions are prohibited."); authCtx = kernalCtx.authentication().authenticate(user, pwd); if (authCtx == null) - throw new IgniteCheckedException("Unknown authentication error."); + throw new IgniteAccessControlException("Unknown authentication error."); } handler = new ClientRequestHandler(this, authCtx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java index b8dfb1fa64b9e..e63812c991b89 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java @@ -49,6 +49,9 @@ private ClientStatus (){ /** Resource does not exist. */ public static final int RESOURCE_DOES_NOT_EXIST = 1011; - /** Resource does not exist. */ + /** Authorization failure. */ public static final int SECURITY_VIOLATION = 1012; + + /** Authentication failed. */ + public static final int AUTH_FAILED = 2000; } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs index 9da9a0310b7c3..2ea17a8739227 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs @@ -19,13 +19,17 @@ namespace Apache.Ignite.Core.Tests.Client { using System; using System.Collections.Generic; + using System.IO; using System.Linq; using System.Net; using System.Net.Sockets; using System.Text.RegularExpressions; using System.Threading; using System.Threading.Tasks; + using Apache.Ignite.Core.Cache.Configuration; + using Apache.Ignite.Core.Cache.Query; using Apache.Ignite.Core.Client; + using Apache.Ignite.Core.Client.Cache; using Apache.Ignite.Core.Configuration; using NUnit.Framework; @@ -34,6 +38,18 @@ namespace Apache.Ignite.Core.Tests.Client /// public class ClientConnectionTest { + /** Temp dir for WAL. */ + private readonly string _tempDir = TestUtils.GetTempDirectoryName(); + + /// + /// Sets up the test. + /// + [SetUp] + public void SetUp() + { + TestUtils.ClearWorkDir(); + } + /// /// Fixture tear down. /// @@ -41,6 +57,13 @@ public class ClientConnectionTest public void TearDown() { Ignition.StopAll(true); + + if (Directory.Exists(_tempDir)) + { + Directory.Delete(_tempDir, true); + } + + TestUtils.ClearWorkDir(); } /// @@ -54,6 +77,107 @@ public void TestNoServerConnectionRefused() Assert.AreEqual(SocketError.ConnectionRefused, socketEx.SocketErrorCode); } + /// + /// Tests that empty username or password are not allowed. + /// + [Test] + public void TestAuthenticationEmptyCredentials() + { + using (Ignition.Start(SecureServerConfig())) + { + var cliCfg = SecureClientConfig(); + + cliCfg.Password = null; + var ex = Assert.Throws(() => { Ignition.StartClient(cliCfg); }); + Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.Password cannot be null")); + + cliCfg.Password = ""; + ex = Assert.Throws(() => { Ignition.StartClient(cliCfg); }); + Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.Password cannot be empty")); + + cliCfg.Password = "ignite"; + + cliCfg.Username = null; + ex = Assert.Throws(() => { Ignition.StartClient(cliCfg); }); + Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.Username cannot be null")); + + cliCfg.Username = ""; + ex = Assert.Throws(() => { Ignition.StartClient(cliCfg); }); + Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.Username cannot be empty")); + } + } + + /// + /// Test invalid username or password. + /// + [Test] + public void TestAuthenticationInvalidCredentials() + { + using (Ignition.Start(SecureServerConfig())) + { + var cliCfg = SecureClientConfig(); + + cliCfg.Username = "invalid"; + + var ex = Assert.Throws(() => { Ignition.StartClient(cliCfg); }); + Assert.True(ex.StatusCode == ClientStatusCode.AuthenticationFailed); + + cliCfg.Username = "ignite"; + cliCfg.Password = "invalid"; + + ex = Assert.Throws(() => { Ignition.StartClient(cliCfg); }); + Assert.True(ex.StatusCode == ClientStatusCode.AuthenticationFailed); + } + } + + /// + /// Test authentication. + /// + [Test] + public void TestAuthentication() + { + using (var srv = Ignition.Start(SecureServerConfig())) + { + srv.GetCluster().SetActive(true); + + using (var cli = Ignition.StartClient(SecureClientConfig())) + { + CacheClientConfiguration ccfg = new CacheClientConfiguration() + { + Name = "TestCache", + QueryEntities = new[] + { + new QueryEntity + { + KeyType = typeof(string), + ValueType = typeof(string), + }, + }, + }; + + ICacheClient cache = cli.GetOrCreateCache(ccfg); + + cache.Put("key1", "val1"); + + cache.Query(new SqlFieldsQuery("CREATE USER \"my_User\" WITH PASSWORD 'my_Password'")).GetAll(); + } + + var cliCfg = SecureClientConfig(); + + cliCfg.Username = "my_User"; + cliCfg.Password = "my_Password"; + + using (var cli = Ignition.StartClient(cliCfg)) + { + ICacheClient cache = cli.GetCache("TestCache"); + + string val = cache.Get("key1"); + + Assert.True(val == "val1"); + } + } + } + /// /// Tests that multiple clients can connect to one server. /// @@ -374,5 +498,42 @@ private static SocketException GetSocketException(Exception ex) throw new Exception("SocketException not found.", origEx); } + + /// + /// Create server configuration with enabled authentication. + /// + /// Server configuration. + private IgniteConfiguration SecureServerConfig() + { + return new IgniteConfiguration(TestUtils.GetTestConfiguration()) + { + AuthenticationEnabled = true, + DataStorageConfiguration = new DataStorageConfiguration() + { + StoragePath = Path.Combine(_tempDir, "Store"), + WalPath = Path.Combine(_tempDir, "WalStore"), + WalArchivePath = Path.Combine(_tempDir, "WalArchive"), + DefaultDataRegionConfiguration = new DataRegionConfiguration() + { + Name = "default", + PersistenceEnabled = true + } + } + }; + } + + /// + /// Create client configuration with enabled authentication. + /// + /// Client configuration. + private static IgniteClientConfiguration SecureClientConfig() + { + return new IgniteClientConfiguration() + { + Host = "localhost", + Username = "ignite", + Password = "ignite" + }; + } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Client/ClientStatusCode.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Client/ClientStatusCode.cs index 3f5ee8eb4bcee..3bdd9e1ac0d6c 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Client/ClientStatusCode.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Client/ClientStatusCode.cs @@ -52,6 +52,16 @@ public enum ClientStatusCode /// /// The too many cursors (see ). /// - TooManyCursors = 1010 + TooManyCursors = 1010, + + /// + /// Authorization failure. + /// + SecurityViolation = 1012, + + /// + /// Authentication failed. + /// + AuthenticationFailed = 2000 } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs index 8730f39340965..32524955ee4c2 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs @@ -90,6 +90,9 @@ public IgniteClientConfiguration(IgniteClientConfiguration cfg) : this() BinaryProcessor = cfg.BinaryProcessor; SslStreamFactory = cfg.SslStreamFactory; + + Username = cfg.Username; + Password = cfg.Password; } /// @@ -145,6 +148,16 @@ public IgniteClientConfiguration(IgniteClientConfiguration cfg) : this() /// public ISslStreamFactory SslStreamFactory { get; set; } + /// + /// Username to be used to connect to secured cluster. + /// + public string Username { get; set; } + + /// + /// Password to be used to connect to secured cluster. + /// + public string Password { get; set; } + /// /// Gets or sets custom binary processor. Internal property for tests. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd index 569ee6fd2d5bd..7e6caff6c4d19 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd @@ -237,6 +237,16 @@ Socket operation timeout. Zero or negative for infinite timeout. + + + Username to be used to connect to secured cluster. + + + + + Password to be used to connect to secured cluster. + + \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientProtocolVersion.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientProtocolVersion.cs index bfdf5a309a876..4fe5c712a3da6 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientProtocolVersion.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientProtocolVersion.cs @@ -22,7 +22,7 @@ namespace Apache.Ignite.Core.Impl.Client /// /// Client protocol version. /// - internal struct ClientProtocolVersion : IEquatable + internal struct ClientProtocolVersion : IEquatable, IComparable { /** */ private readonly short _major; @@ -67,6 +67,26 @@ public short Maintenance get { return _maintenance; } } + /// + /// Compare this version to other version. + /// + /// + /// + public int CompareTo(ClientProtocolVersion other) + { + int res = Major - other.Major; + + if (res == 0) + { + res = Minor - other.Minor; + + if (res == 0) + res = Maintenance - other.Maintenance; + } + + return res; + } + /// /// Returns a value indicating whether specified instance equals to current. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs index fca5dab55062b..27d8f0bea092a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs @@ -28,6 +28,7 @@ namespace Apache.Ignite.Core.Impl.Client using System.Net.Sockets; using System.Threading; using System.Threading.Tasks; + using System.Xml.Schema; using Apache.Ignite.Core.Client; using Apache.Ignite.Core.Common; using Apache.Ignite.Core.Impl.Binary; @@ -38,8 +39,14 @@ namespace Apache.Ignite.Core.Impl.Client /// internal sealed class ClientSocket : IDisposable { + /** Version 1.0.0. */ + private static readonly ClientProtocolVersion Ver100 = new ClientProtocolVersion(1, 0, 0); + + /** Version 1.1.0. */ + private static readonly ClientProtocolVersion Ver110 = new ClientProtocolVersion(1, 1, 0); + /** Current version. */ - private static readonly ClientProtocolVersion CurrentProtocolVersion = new ClientProtocolVersion(1, 0, 0); + private static readonly ClientProtocolVersion CurrentProtocolVersion = Ver110; /** Handshake opcode. */ private const byte OpHandshake = 1; @@ -98,7 +105,9 @@ public ClientSocket(IgniteClientConfiguration clientConfiguration, ClientProtoco _socket = Connect(clientConfiguration); _stream = GetSocketStream(_socket, clientConfiguration); - Handshake(version ?? CurrentProtocolVersion); + Validate(clientConfiguration); + + Handshake(clientConfiguration, version ?? CurrentProtocolVersion); // Check periodically if any request has timed out. if (_timeout > TimeSpan.Zero) @@ -111,6 +120,31 @@ public ClientSocket(IgniteClientConfiguration clientConfiguration, ClientProtoco Task.Factory.StartNew(WaitForMessages); } + /// + /// Validate configuration. + /// + /// Configuration. + private void Validate(IgniteClientConfiguration cfg) + { + if (cfg.Username != null) + { + if (cfg.Username.Length == 0) + throw new IgniteClientException("IgniteClientConfiguration.Username cannot be empty."); + + if (cfg.Password == null) + throw new IgniteClientException("IgniteClientConfiguration.Password cannot be null when Username is set."); + } + + if (cfg.Password != null) + { + if (cfg.Password.Length == 0) + throw new IgniteClientException("IgniteClientConfiguration.Password cannot be empty."); + + if (cfg.Username == null) + throw new IgniteClientException("IgniteClientConfiguration.Username cannot be null when Password is set."); + } + } + /// /// Performs a send-receive operation. /// @@ -226,8 +260,10 @@ private static T DecodeResponse(BinaryHeapStream stream, Func /// Performs client protocol handshake. /// - private void Handshake(ClientProtocolVersion version) + private void Handshake(IgniteClientConfiguration clientConfiguration, ClientProtocolVersion version) { + bool auth = version.CompareTo(Ver110) >= 0 && clientConfiguration.Username != null; + // Send request. int messageLen; var buf = WriteMessage(stream => @@ -242,10 +278,19 @@ private void Handshake(ClientProtocolVersion version) // Client type: platform. stream.WriteByte(ClientType); - }, 12, out messageLen); - Debug.Assert(messageLen == 12); + // Authentication data. + if (auth) + { + var writer = BinaryUtils.Marshaller.StartMarshal(stream); + + writer.WriteString(clientConfiguration.Username); + writer.WriteString(clientConfiguration.Password); + BinaryUtils.Marshaller.FinishMarshal(writer); + } + }, 12, out messageLen); + _stream.Write(buf, 0, messageLen); // Decode response. @@ -253,6 +298,7 @@ private void Handshake(ClientProtocolVersion version) using (var stream = new BinaryHeapStream(res)) { + // Read input. var success = stream.ReadBool(); if (success) @@ -265,9 +311,32 @@ private void Handshake(ClientProtocolVersion version) var errMsg = BinaryUtils.Marshaller.Unmarshal(stream); - throw new IgniteClientException(string.Format( - "Client handshake failed: '{0}'. Client version: {1}. Server version: {2}", - errMsg, version, serverVersion)); + ClientStatusCode errCode = ClientStatusCode.Fail; + + if (stream.Remaining > 0) + { + errCode = (ClientStatusCode) stream.ReadInt(); + } + + // Authentication error is handled immediately. + if (errCode == ClientStatusCode.AuthenticationFailed) + { + throw new IgniteClientException(errMsg, null, ClientStatusCode.AuthenticationFailed); + } + + // Re-try if possible. + bool retry = serverVersion.CompareTo(version) < 0 && serverVersion.Equals(Ver100); + + if (retry) + { + Handshake(clientConfiguration, serverVersion); + } + else + { + throw new IgniteClientException(string.Format( + "Client handshake failed: '{0}'. Client version: {1}. Server version: {2}", + errMsg, version, serverVersion), null, errCode); + } } } From 72259b01e0c6d72794eca4c28c9d9d848b0ff97f Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Thu, 12 Apr 2018 11:16:22 +0300 Subject: [PATCH 0037/1463] IGNITE-8176: Integrate gradient descent linear regression with partition based dataset this closes #3787 (cherry picked from commit df6356d) --- .../ml/knn/KNNClassificationExample.java | 11 +- .../examples/ml/nn/MLPTrainerExample.java | 4 +- .../preprocessing/NormalizationExample.java | 17 +-- ...ithLSQRTrainerAndNormalizationExample.java | 23 ++-- ...inearRegressionWithLSQRTrainerExample.java | 14 +-- ...dLinearRegressionWithQRTrainerExample.java | 9 +- ...LinearRegressionWithSGDTrainerExample.java | 78 +++++++++--- .../SVMBinaryClassificationExample.java | 11 +- .../SVMMultiClassClassificationExample.java | 24 ++-- ...isionTreeClassificationTrainerExample.java | 7 +- .../DecisionTreeRegressionTrainerExample.java | 4 +- .../org/apache/ignite/ml/nn/Activators.java | 20 +++ .../org/apache/ignite/ml/nn/MLPTrainer.java | 46 +++++-- .../preprocessing/PreprocessingTrainer.java | 41 +++++- .../normalization/NormalizationTrainer.java | 35 ++++-- .../linear/FeatureExtractorWrapper.java | 55 ++++++++ .../linear/LinearRegressionLSQRTrainer.java | 38 +----- .../linear/LinearRegressionSGDTrainer.java | 118 ++++++++++++------ .../ignite/ml/trainers/DatasetTrainer.java | 46 +++++++ .../ignite/ml/knn/KNNClassificationTest.java | 20 +-- .../ml/nn/MLPTrainerIntegrationTest.java | 14 +-- .../apache/ignite/ml/nn/MLPTrainerTest.java | 22 ++-- .../MLPTrainerMnistIntegrationTest.java | 7 +- .../nn/performance/MLPTrainerMnistTest.java | 11 +- .../NormalizationTrainerTest.java | 10 +- .../ml/regressions/RegressionsTestSuite.java | 15 +-- ...ributedLinearRegressionSGDTrainerTest.java | 35 ------ ...ributedLinearRegressionSGDTrainerTest.java | 35 ------ ...reAbstractLinearRegressionTrainerTest.java | 3 + .../LinearRegressionLSQRTrainerTest.java | 14 ++- .../LinearRegressionSGDTrainerTest.java | 94 ++++++++++++++ .../LocalLinearRegressionSGDTrainerTest.java | 35 ------ .../ignite/ml/svm/SVMBinaryTrainerTest.java | 11 +- .../ml/svm/SVMMultiClassTrainerTest.java | 11 +- ...eClassificationTrainerIntegrationTest.java | 9 +- ...DecisionTreeClassificationTrainerTest.java | 12 +- ...nTreeRegressionTrainerIntegrationTest.java | 9 +- .../DecisionTreeRegressionTrainerTest.java | 12 +- .../DecisionTreeMNISTIntegrationTest.java | 7 +- .../performance/DecisionTreeMNISTTest.java | 11 +- 40 files changed, 612 insertions(+), 386 deletions(-) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/FeatureExtractorWrapper.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/BlockDistributedLinearRegressionSGDTrainerTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/DistributedLinearRegressionSGDTrainerTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainerTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LocalLinearRegressionSGDTrainerTest.java 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 f3cdbbefc2a2a..39a8431a18761 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 @@ -17,9 +17,6 @@ package org.apache.ignite.examples.ml.knn; -import java.util.Arrays; -import java.util.UUID; -import javax.cache.Cache; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.Ignition; @@ -27,7 +24,6 @@ import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; import org.apache.ignite.ml.knn.classification.KNNClassificationModel; import org.apache.ignite.ml.knn.classification.KNNClassificationTrainer; import org.apache.ignite.ml.knn.classification.KNNStrategy; @@ -35,6 +31,10 @@ import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; import org.apache.ignite.thread.IgniteThread; +import javax.cache.Cache; +import java.util.Arrays; +import java.util.UUID; + /** * Run kNN multi-class classification trainer over distributed dataset. * @@ -56,7 +56,8 @@ public static void main(String[] args) throws InterruptedException { KNNClassificationTrainer trainer = new KNNClassificationTrainer(); KNNClassificationModel knnMdl = trainer.fit( - new CacheBasedDatasetBuilder<>(ignite, dataCache), + ignite, + dataCache, (k, v) -> Arrays.copyOfRange(v, 1, v.length), (k, v) -> v[0] ).withK(3) 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 efa1ba73d0401..ce44cc64b935e 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 @@ -23,7 +23,6 @@ import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.examples.ExampleNodeStartup; -import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; import org.apache.ignite.ml.math.Matrix; import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; import org.apache.ignite.ml.nn.Activators; @@ -99,7 +98,8 @@ public static void main(String[] args) throws InterruptedException { // Train neural network and get multilayer perceptron model. MultilayerPerceptron mlp = trainer.fit( - new CacheBasedDatasetBuilder<>(ignite, trainingSet), + ignite, + trainingSet, (k, v) -> new double[] {v.x, v.y}, (k, v) -> new double[] {v.lb} ); diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/NormalizationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/NormalizationExample.java index e0bcd089b674b..b2c4e128da332 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/NormalizationExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/NormalizationExample.java @@ -17,21 +17,19 @@ package org.apache.ignite.examples.ml.preprocessing; -import java.util.Arrays; 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.ml.dataset.DatasetBuilder; import org.apache.ignite.ml.dataset.DatasetFactory; -import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; import org.apache.ignite.ml.dataset.primitive.SimpleDataset; import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.preprocessing.normalization.NormalizationPreprocessor; import org.apache.ignite.ml.preprocessing.normalization.NormalizationTrainer; +import java.util.Arrays; + /** * Example that shows how to use normalization preprocessor to normalize data. * @@ -47,8 +45,6 @@ public static void main(String[] args) throws Exception { IgniteCache persons = createCache(ignite); - DatasetBuilder builder = new CacheBasedDatasetBuilder<>(ignite, persons); - // Defines first preprocessor that extracts features from an upstream data. IgniteBiFunction featureExtractor = (k, v) -> new double[] { v.getAge(), @@ -56,14 +52,11 @@ public static void main(String[] args) throws Exception { }; // Defines second preprocessor that normalizes features. - NormalizationPreprocessor preprocessor = new NormalizationTrainer() - .fit(builder, featureExtractor, 2); + IgniteBiFunction preprocessor = new NormalizationTrainer() + .fit(ignite, persons, featureExtractor); // Creates a cache based simple dataset containing features and providing standard dataset API. - try (SimpleDataset dataset = DatasetFactory.createSimpleDataset( - builder, - preprocessor - )) { + try (SimpleDataset dataset = DatasetFactory.createSimpleDataset(ignite, persons, preprocessor)) { // Calculation of the mean value. This calculation will be performed in map-reduce manner. double[] mean = dataset.mean(); System.out.println("Mean \n\t" + Arrays.toString(mean)); diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample.java index 567a59975a385..99e657781b131 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample.java @@ -17,9 +17,6 @@ package org.apache.ignite.examples.ml.regression.linear; -import java.util.Arrays; -import java.util.UUID; -import javax.cache.Cache; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.Ignition; @@ -28,7 +25,7 @@ import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.examples.ml.math.matrix.SparseDistributedMatrixExample; -import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; import org.apache.ignite.ml.preprocessing.normalization.NormalizationPreprocessor; import org.apache.ignite.ml.preprocessing.normalization.NormalizationTrainer; @@ -36,6 +33,10 @@ import org.apache.ignite.ml.regressions.linear.LinearRegressionModel; import org.apache.ignite.thread.IgniteThread; +import javax.cache.Cache; +import java.util.Arrays; +import java.util.UUID; + /** * Run linear regression model over distributed matrix. * @@ -119,21 +120,17 @@ public static void main(String[] args) throws InterruptedException { NormalizationTrainer normalizationTrainer = new NormalizationTrainer<>(); System.out.println(">>> Perform the training to get the normalization preprocessor."); - NormalizationPreprocessor preprocessor = normalizationTrainer.fit( - new CacheBasedDatasetBuilder<>(ignite, dataCache), - (k, v) -> Arrays.copyOfRange(v, 1, v.length), - 4 + IgniteBiFunction preprocessor = normalizationTrainer.fit( + ignite, + dataCache, + (k, v) -> Arrays.copyOfRange(v, 1, v.length) ); System.out.println(">>> Create new linear regression trainer object."); LinearRegressionLSQRTrainer trainer = new LinearRegressionLSQRTrainer(); System.out.println(">>> Perform the training to get the model."); - LinearRegressionModel mdl = trainer.fit( - new CacheBasedDatasetBuilder<>(ignite, dataCache), - preprocessor, - (k, v) -> v[0] - ); + LinearRegressionModel mdl = trainer.fit(ignite, dataCache, preprocessor, (k, v) -> v[0]); System.out.println(">>> Linear regression model: " + mdl); diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java index a853092f92a08..25aec0cbba400 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java @@ -17,9 +17,6 @@ package org.apache.ignite.examples.ml.regression.linear; -import java.util.Arrays; -import java.util.UUID; -import javax.cache.Cache; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.Ignition; @@ -27,13 +24,15 @@ import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.examples.ml.math.matrix.SparseDistributedMatrixExample; -import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; import org.apache.ignite.ml.regressions.linear.LinearRegressionLSQRTrainer; import org.apache.ignite.ml.regressions.linear.LinearRegressionModel; import org.apache.ignite.thread.IgniteThread; +import javax.cache.Cache; +import java.util.Arrays; +import java.util.UUID; + /** * Run linear regression model over distributed matrix. * @@ -108,7 +107,7 @@ public static void main(String[] args) throws InterruptedException { // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread // because we create ignite cache internally. IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), - SparseDistributedMatrixExample.class.getSimpleName(), () -> { + DistributedLinearRegressionWithLSQRTrainerExample.class.getSimpleName(), () -> { IgniteCache dataCache = getTestCache(ignite); System.out.println(">>> Create new linear regression trainer object."); @@ -116,7 +115,8 @@ public static void main(String[] args) throws InterruptedException { System.out.println(">>> Perform the training to get the model."); LinearRegressionModel mdl = trainer.fit( - new CacheBasedDatasetBuilder<>(ignite, dataCache), + ignite, + dataCache, (k, v) -> Arrays.copyOfRange(v, 1, v.length), (k, v) -> v[0] ); diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java index 2b45aa26b87bb..98d5e4e84ced1 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java @@ -17,7 +17,6 @@ package org.apache.ignite.examples.ml.regression.linear; -import java.util.Arrays; import org.apache.ignite.Ignite; import org.apache.ignite.Ignition; import org.apache.ignite.examples.ml.math.matrix.SparseDistributedMatrixExample; @@ -30,6 +29,8 @@ import org.apache.ignite.ml.regressions.linear.LinearRegressionQRTrainer; import org.apache.ignite.thread.IgniteThread; +import java.util.Arrays; + /** * Run linear regression model over distributed matrix. * @@ -113,15 +114,15 @@ public static void main(String[] args) throws InterruptedException { Trainer trainer = new LinearRegressionQRTrainer(); System.out.println(">>> Perform the training to get the model."); - LinearRegressionModel model = trainer.train(distributedMatrix); - System.out.println(">>> Linear regression model: " + model); + LinearRegressionModel mdl = trainer.train(distributedMatrix); + System.out.println(">>> Linear regression model: " + mdl); System.out.println(">>> ---------------------------------"); System.out.println(">>> | Prediction\t| Ground Truth\t|"); System.out.println(">>> ---------------------------------"); for (double[] observation : data) { Vector inputs = new SparseDistributedVector(Arrays.copyOfRange(observation, 1, observation.length)); - double prediction = model.apply(inputs); + double prediction = mdl.apply(inputs); double groundTruth = observation[0]; System.out.printf(">>> | %.4f\t\t| %.4f\t\t|\n", prediction, groundTruth); } diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithSGDTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithSGDTrainerExample.java index f3b2655167dc4..44366e1aafc33 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithSGDTrainerExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithSGDTrainerExample.java @@ -17,20 +17,26 @@ package org.apache.ignite.examples.ml.regression.linear; -import java.util.Arrays; import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; import org.apache.ignite.Ignition; -import org.apache.ignite.examples.ml.math.matrix.SparseDistributedMatrixExample; -import org.apache.ignite.ml.Trainer; -import org.apache.ignite.ml.math.Matrix; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.ml.math.impls.vector.SparseDistributedVector; +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.ml.math.impls.vector.DenseLocalOnHeapVector; +import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate; +import org.apache.ignite.ml.optimization.updatecalculators.RPropUpdateCalculator; import org.apache.ignite.ml.regressions.linear.LinearRegressionModel; import org.apache.ignite.ml.regressions.linear.LinearRegressionQRTrainer; import org.apache.ignite.ml.regressions.linear.LinearRegressionSGDTrainer; +import org.apache.ignite.ml.trainers.group.UpdatesStrategy; import org.apache.ignite.thread.IgniteThread; +import javax.cache.Cache; +import java.util.Arrays; +import java.util.UUID; + /** * Run linear regression model over distributed matrix. * @@ -104,28 +110,43 @@ public static void main(String[] args) throws InterruptedException { // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread // because we create ignite cache internally. IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), - SparseDistributedMatrixExample.class.getSimpleName(), () -> { + DistributedLinearRegressionWithSGDTrainerExample.class.getSimpleName(), () -> { - // Create SparseDistributedMatrix, new cache will be created automagically. - System.out.println(">>> Create new SparseDistributedMatrix inside IgniteThread."); - SparseDistributedMatrix distributedMatrix = new SparseDistributedMatrix(data); + IgniteCache dataCache = getTestCache(ignite); System.out.println(">>> Create new linear regression trainer object."); - Trainer trainer = new LinearRegressionSGDTrainer(100_000, 1e-12); + LinearRegressionSGDTrainer trainer = new LinearRegressionSGDTrainer<>(new UpdatesStrategy<>( + new RPropUpdateCalculator(), + RPropParameterUpdate::sumLocal, + RPropParameterUpdate::avg + ), 100000, 10, 100, 123L); System.out.println(">>> Perform the training to get the model."); - LinearRegressionModel model = trainer.train(distributedMatrix); - System.out.println(">>> Linear regression model: " + model); + LinearRegressionModel mdl = trainer.fit( + ignite, + dataCache, + (k, v) -> Arrays.copyOfRange(v, 1, v.length), + (k, v) -> v[0] + ); + + System.out.println(">>> Linear regression model: " + mdl); System.out.println(">>> ---------------------------------"); System.out.println(">>> | Prediction\t| Ground Truth\t|"); System.out.println(">>> ---------------------------------"); - for (double[] observation : data) { - Vector inputs = new SparseDistributedVector(Arrays.copyOfRange(observation, 1, observation.length)); - double prediction = model.apply(inputs); - double groundTruth = observation[0]; - System.out.printf(">>> | %.4f\t\t| %.4f\t\t|\n", prediction, groundTruth); + + try (QueryCursor> observations = dataCache.query(new ScanQuery<>())) { + for (Cache.Entry observation : observations) { + double[] val = observation.getValue(); + double[] inputs = Arrays.copyOfRange(val, 1, val.length); + double groundTruth = val[0]; + + double prediction = mdl.apply(new DenseLocalOnHeapVector(inputs)); + + System.out.printf(">>> | %.4f\t\t| %.4f\t\t|\n", prediction, groundTruth); + } } + System.out.println(">>> ---------------------------------"); }); @@ -134,4 +155,23 @@ public static void main(String[] args) throws InterruptedException { igniteThread.join(); } } + + /** + * Fills cache with data and returns it. + * + * @param ignite Ignite instance. + * @return Filled Ignite Cache. + */ + private static IgniteCache getTestCache(Ignite ignite) { + CacheConfiguration cacheConfiguration = new CacheConfiguration<>(); + cacheConfiguration.setName("TEST_" + UUID.randomUUID()); + 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; + } } diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/svm/binary/SVMBinaryClassificationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/svm/binary/SVMBinaryClassificationExample.java index f8bf521637010..ce37112978dd5 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/svm/binary/SVMBinaryClassificationExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/svm/binary/SVMBinaryClassificationExample.java @@ -17,9 +17,6 @@ package org.apache.ignite.examples.ml.svm.binary; -import java.util.Arrays; -import java.util.UUID; -import javax.cache.Cache; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.Ignition; @@ -27,12 +24,15 @@ import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; import org.apache.ignite.ml.svm.SVMLinearBinaryClassificationModel; import org.apache.ignite.ml.svm.SVMLinearBinaryClassificationTrainer; import org.apache.ignite.thread.IgniteThread; +import javax.cache.Cache; +import java.util.Arrays; +import java.util.UUID; + /** * Run SVM binary-class classification model over distributed dataset. * @@ -54,7 +54,8 @@ public static void main(String[] args) throws InterruptedException { SVMLinearBinaryClassificationTrainer trainer = new SVMLinearBinaryClassificationTrainer(); SVMLinearBinaryClassificationModel mdl = trainer.fit( - new CacheBasedDatasetBuilder<>(ignite, dataCache), + ignite, + dataCache, (k, v) -> Arrays.copyOfRange(v, 1, v.length), (k, v) -> v[0] ); diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/svm/multiclass/SVMMultiClassClassificationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/svm/multiclass/SVMMultiClassClassificationExample.java index f8281e489f69d..4054201ee6b96 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/svm/multiclass/SVMMultiClassClassificationExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/svm/multiclass/SVMMultiClassClassificationExample.java @@ -17,9 +17,6 @@ package org.apache.ignite.examples.ml.svm.multiclass; -import java.util.Arrays; -import java.util.UUID; -import javax.cache.Cache; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.Ignition; @@ -27,14 +24,17 @@ import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; -import org.apache.ignite.ml.preprocessing.normalization.NormalizationPreprocessor; import org.apache.ignite.ml.preprocessing.normalization.NormalizationTrainer; import org.apache.ignite.ml.svm.SVMLinearMultiClassClassificationModel; import org.apache.ignite.ml.svm.SVMLinearMultiClassClassificationTrainer; import org.apache.ignite.thread.IgniteThread; +import javax.cache.Cache; +import java.util.Arrays; +import java.util.UUID; + /** * Run SVM multi-class classification trainer over distributed dataset to build two models: * one with normalization and one without normalization. @@ -57,7 +57,8 @@ public static void main(String[] args) throws InterruptedException { SVMLinearMultiClassClassificationTrainer trainer = new SVMLinearMultiClassClassificationTrainer(); SVMLinearMultiClassClassificationModel mdl = trainer.fit( - new CacheBasedDatasetBuilder<>(ignite, dataCache), + ignite, + dataCache, (k, v) -> Arrays.copyOfRange(v, 1, v.length), (k, v) -> v[0] ); @@ -67,14 +68,15 @@ public static void main(String[] args) throws InterruptedException { NormalizationTrainer normalizationTrainer = new NormalizationTrainer<>(); - NormalizationPreprocessor preprocessor = normalizationTrainer.fit( - new CacheBasedDatasetBuilder<>(ignite, dataCache), - (k, v) -> Arrays.copyOfRange(v, 1, v.length), - 5 + IgniteBiFunction preprocessor = normalizationTrainer.fit( + ignite, + dataCache, + (k, v) -> Arrays.copyOfRange(v, 1, v.length) ); SVMLinearMultiClassClassificationModel mdlWithNormalization = trainer.fit( - new CacheBasedDatasetBuilder<>(ignite, dataCache), + ignite, + dataCache, preprocessor, (k, v) -> v[0] ); diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeClassificationTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeClassificationTrainerExample.java index cef63683cb4ec..1ecf460148ddb 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeClassificationTrainerExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeClassificationTrainerExample.java @@ -17,17 +17,17 @@ package org.apache.ignite.examples.ml.tree; -import java.util.Random; 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.ml.dataset.impl.cache.CacheBasedDatasetBuilder; import org.apache.ignite.ml.tree.DecisionTreeClassificationTrainer; import org.apache.ignite.ml.tree.DecisionTreeNode; import org.apache.ignite.thread.IgniteThread; +import java.util.Random; + /** * Example of using distributed {@link DecisionTreeClassificationTrainer}. */ @@ -65,7 +65,8 @@ public static void main(String... args) throws InterruptedException { // Train decision tree model. DecisionTreeNode mdl = trainer.fit( - new CacheBasedDatasetBuilder<>(ignite, trainingSet), + ignite, + trainingSet, (k, v) -> new double[]{v.x, v.y}, (k, v) -> v.lb ); diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeRegressionTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeRegressionTrainerExample.java index 61ba5f9dca471..19b15f3bbf3ca 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeRegressionTrainerExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeRegressionTrainerExample.java @@ -22,7 +22,6 @@ import org.apache.ignite.Ignition; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; import org.apache.ignite.ml.tree.DecisionTreeNode; import org.apache.ignite.ml.tree.DecisionTreeRegressionTrainer; import org.apache.ignite.thread.IgniteThread; @@ -61,7 +60,8 @@ public static void main(String... args) throws InterruptedException { // Train decision tree model. DecisionTreeNode mdl = trainer.fit( - new CacheBasedDatasetBuilder<>(ignite, trainingSet), + ignite, + trainingSet, (k, v) -> new double[] {v.x}, (k, v) -> v.y ); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/nn/Activators.java b/modules/ml/src/main/java/org/apache/ignite/ml/nn/Activators.java index f05bde8311440..4c34cd2677247 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/nn/Activators.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/nn/Activators.java @@ -58,4 +58,24 @@ public class Activators { return Math.max(val, 0); } }; + + /** + * Linear unit activation function. + */ + public static IgniteDifferentiableDoubleToDoubleFunction LINEAR = new IgniteDifferentiableDoubleToDoubleFunction() { + /** {@inheritDoc} */ + @Override public double differential(double pnt) { + return 1.0; + } + + /** + * Differential of linear at pnt. + * + * @param pnt Point to differentiate at. + * @return Differential at pnt. + */ + @Override public Double apply(double pnt) { + return pnt; + } + }; } 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 47d20226991d2..fe955cbe696de 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 @@ -17,11 +17,6 @@ package org.apache.ignite.ml.nn; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; -import java.util.Random; -import org.apache.ignite.ml.trainers.MultiLabelDatasetTrainer; import org.apache.ignite.ml.dataset.Dataset; import org.apache.ignite.ml.dataset.DatasetBuilder; import org.apache.ignite.ml.dataset.primitive.builder.context.EmptyContextBuilder; @@ -37,17 +32,23 @@ import org.apache.ignite.ml.nn.architecture.MLPArchitecture; import org.apache.ignite.ml.nn.initializers.RandomInitializer; import org.apache.ignite.ml.optimization.updatecalculators.ParameterUpdateCalculator; +import org.apache.ignite.ml.trainers.MultiLabelDatasetTrainer; import org.apache.ignite.ml.trainers.group.UpdatesStrategy; import org.apache.ignite.ml.util.Utils; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; + /** * Multilayer perceptron trainer based on partition based {@link Dataset}. * * @param

    Type of model update used in this trainer. */ public class MLPTrainer

    implements MultiLabelDatasetTrainer { - /** Multilayer perceptron architecture that defines layers and activators. */ - private final MLPArchitecture arch; + /** Multilayer perceptron architecture supplier that defines layers and activators. */ + private final IgniteFunction, MLPArchitecture> archSupplier; /** Loss function to be minimized during the training. */ private final IgniteFunction loss; @@ -81,7 +82,25 @@ public class MLPTrainer

    implements MultiLabelDatasetTrai public MLPTrainer(MLPArchitecture arch, IgniteFunction loss, UpdatesStrategy updatesStgy, int maxIterations, int batchSize, int locIterations, long seed) { - this.arch = arch; + this(dataset -> arch, loss, updatesStgy, maxIterations, batchSize, locIterations, seed); + } + + /** + * Constructs a new instance of multilayer perceptron trainer. + * + * @param archSupplier Multilayer perceptron architecture supplier that defines layers and activators. + * @param loss Loss function to be minimized during the training. + * @param updatesStgy Update strategy that defines how to update model parameters during the training. + * @param maxIterations Maximal number of iterations before the training will be stopped. + * @param batchSize Batch size (per every partition). + * @param locIterations Maximal number of local iterations before synchronization. + * @param seed Random initializer seed. + */ + public MLPTrainer(IgniteFunction, MLPArchitecture> archSupplier, + IgniteFunction loss, + UpdatesStrategy updatesStgy, int maxIterations, int batchSize, + int locIterations, long seed) { + this.archSupplier = archSupplier; this.loss = loss; this.updatesStgy = updatesStgy; this.maxIterations = maxIterations; @@ -94,13 +113,14 @@ public MLPTrainer(MLPArchitecture arch, IgniteFunction MultilayerPerceptron fit(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { - MultilayerPerceptron mdl = new MultilayerPerceptron(arch, new RandomInitializer(seed)); - ParameterUpdateCalculator updater = updatesStgy.getUpdatesCalculator(); - try (Dataset dataset = datasetBuilder.build( new EmptyContextBuilder<>(), new SimpleLabeledDatasetDataBuilder<>(featureExtractor, lbExtractor) )) { + MLPArchitecture arch = archSupplier.apply(dataset); + MultilayerPerceptron mdl = new MultilayerPerceptron(arch, new RandomInitializer(seed)); + ParameterUpdateCalculator updater = updatesStgy.getUpdatesCalculator(); + for (int i = 0; i < maxIterations; i += locIterations) { MultilayerPerceptron finalMdl = mdl; @@ -163,12 +183,12 @@ else if (b == null) P update = updatesStgy.allUpdatesReducer().apply(totUp); mdl = updater.update(mdl, update); } + + return mdl; } catch (Exception e) { throw new RuntimeException(e); } - - return mdl; } /** 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 f5a6bb0ceb132..1886ee557da75 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,9 +17,15 @@ package org.apache.ignite.ml.preprocessing; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; +import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import java.util.Map; + /** * Trainer for preprocessor. * @@ -34,9 +40,40 @@ public interface PreprocessingTrainer { * * @param datasetBuilder Dataset builder. * @param basePreprocessor Base preprocessor. - * @param cols Number of columns. * @return Preprocessor. */ public IgniteBiFunction fit(DatasetBuilder datasetBuilder, - IgniteBiFunction basePreprocessor, int cols); + IgniteBiFunction basePreprocessor); + + /** + * Fits preprocessor. + * + * @param ignite Ignite instance. + * @param cache Ignite cache. + * @param basePreprocessor Base preprocessor. + * @return Preprocessor. + */ + public default IgniteBiFunction fit(Ignite ignite, IgniteCache cache, + IgniteBiFunction basePreprocessor) { + return fit( + new CacheBasedDatasetBuilder<>(ignite, cache), + basePreprocessor + ); + } + + /** + * Fits preprocessor. + * + * @param data Data. + * @param parts Number of partitions. + * @param basePreprocessor Base preprocessor. + * @return Preprocessor. + */ + public default IgniteBiFunction fit(Map data, int parts, + IgniteBiFunction basePreprocessor) { + return fit( + new LocalDatasetBuilder<>(data, parts), + basePreprocessor + ); + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/normalization/NormalizationTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/normalization/NormalizationTrainer.java index 16623ba5748a7..57acbad54345c 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/normalization/NormalizationTrainer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/normalization/NormalizationTrainer.java @@ -33,33 +33,48 @@ public class NormalizationTrainer implements PreprocessingTrainer { /** {@inheritDoc} */ @Override public NormalizationPreprocessor fit(DatasetBuilder datasetBuilder, - IgniteBiFunction basePreprocessor, int cols) { + IgniteBiFunction basePreprocessor) { try (Dataset dataset = datasetBuilder.build( (upstream, upstreamSize) -> new EmptyContext(), (upstream, upstreamSize, ctx) -> { - double[] min = new double[cols]; - double[] max = new double[cols]; - - for (int i = 0; i < cols; i++) { - min[i] = Double.MAX_VALUE; - max[i] = -Double.MAX_VALUE; - } + double[] min = null; + double[] max = null; while (upstream.hasNext()) { UpstreamEntry entity = upstream.next(); double[] row = basePreprocessor.apply(entity.getKey(), entity.getValue()); - for (int i = 0; i < cols; i++) { + + if (min == null) { + min = new double[row.length]; + for (int i = 0; i < min.length; i++) + min[i] = Double.MAX_VALUE; + } + else + assert min.length == row.length : "Base preprocessor must return exactly " + min.length + + " features"; + + if (max == null) { + max = new double[row.length]; + for (int i = 0; i < max.length; i++) + max[i] = -Double.MAX_VALUE; + } + else + assert max.length == row.length : "Base preprocessor must return exactly " + min.length + + " features"; + + for (int i = 0; i < row.length; i++) { if (row[i] < min[i]) min[i] = row[i]; if (row[i] > max[i]) max[i] = row[i]; } } + return new NormalizationPartitionData(min, max); } )) { double[][] minMax = dataset.compute( - data -> new double[][]{ data.getMin(), data.getMax() }, + data -> data.getMin() != null ? new double[][]{ data.getMin(), data.getMax() } : null, (a, b) -> { if (a == null) return b; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/FeatureExtractorWrapper.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/FeatureExtractorWrapper.java new file mode 100644 index 0000000000000..8e8f467488efc --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/FeatureExtractorWrapper.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.regressions.linear; + +import org.apache.ignite.ml.math.functions.IgniteBiFunction; + +import java.util.Arrays; + +/** + * Feature extractor wrapper that adds additional column filled by 1. + * + * @param Type of a key in {@code upstream} data. + * @param Type of a value in {@code upstream} data. + */ +public class FeatureExtractorWrapper implements IgniteBiFunction { + /** */ + private static final long serialVersionUID = -2686524650955735635L; + + /** Underlying feature extractor. */ + private final IgniteBiFunction featureExtractor; + + /** + * Constructs a new instance of feature extractor wrapper. + * + * @param featureExtractor Underlying feature extractor. + */ + FeatureExtractorWrapper(IgniteBiFunction featureExtractor) { + this.featureExtractor = featureExtractor; + } + + /** {@inheritDoc} */ + @Override public double[] apply(K k, V v) { + double[] featureRow = featureExtractor.apply(k, v); + double[] row = Arrays.copyOf(featureRow, featureRow.length + 1); + + row[featureRow.length] = 1.0; + + return row; + } +} 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 ae15f2f3a888e..9526db1e366ae 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 @@ -17,8 +17,6 @@ package org.apache.ignite.ml.regressions.linear; -import java.util.Arrays; -import org.apache.ignite.ml.trainers.SingleLabelDatasetTrainer; import org.apache.ignite.ml.dataset.DatasetBuilder; import org.apache.ignite.ml.math.Vector; import org.apache.ignite.ml.math.functions.IgniteBiFunction; @@ -27,6 +25,9 @@ import org.apache.ignite.ml.math.isolve.lsqr.AbstractLSQR; import org.apache.ignite.ml.math.isolve.lsqr.LSQROnHeap; import org.apache.ignite.ml.math.isolve.lsqr.LSQRResult; +import org.apache.ignite.ml.trainers.SingleLabelDatasetTrainer; + +import java.util.Arrays; /** * Trainer of the linear regression model based on LSQR algorithm. @@ -55,37 +56,4 @@ public class LinearRegressionLSQRTrainer implements SingleLabelDatasetTrainer

  • Type of a key in {@code upstream} data. - * @param Type of a value in {@code upstream} data. - */ - private static class FeatureExtractorWrapper implements IgniteBiFunction { - /** */ - private static final long serialVersionUID = -2686524650955735635L; - - /** Underlying feature extractor. */ - private final IgniteBiFunction featureExtractor; - - /** - * Constructs a new instance of feature extractor wrapper. - * - * @param featureExtractor Underlying feature extractor. - */ - FeatureExtractorWrapper(IgniteBiFunction featureExtractor) { - this.featureExtractor = featureExtractor; - } - - /** {@inheritDoc} */ - @Override public double[] apply(K k, V v) { - double[] featureRow = featureExtractor.apply(k, v); - double[] row = Arrays.copyOf(featureRow, featureRow.length + 1); - - row[featureRow.length] = 1.0; - - return row; - } - } } 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 aad4c7a73fd16..9be3fdd2a8aa2 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 @@ -17,51 +17,99 @@ package org.apache.ignite.ml.regressions.linear; -import org.apache.ignite.ml.Trainer; -import org.apache.ignite.ml.math.Matrix; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.optimization.BarzilaiBorweinUpdater; -import org.apache.ignite.ml.optimization.GradientDescent; -import org.apache.ignite.ml.optimization.LeastSquaresGradientFunction; -import org.apache.ignite.ml.optimization.SimpleUpdater; +import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; +import org.apache.ignite.ml.dataset.primitive.data.SimpleLabeledDatasetData; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.functions.IgniteFunction; +import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; +import org.apache.ignite.ml.nn.Activators; +import org.apache.ignite.ml.nn.MLPTrainer; +import org.apache.ignite.ml.nn.MultilayerPerceptron; +import org.apache.ignite.ml.nn.architecture.MLPArchitecture; +import org.apache.ignite.ml.optimization.LossFunctions; +import org.apache.ignite.ml.trainers.SingleLabelDatasetTrainer; +import org.apache.ignite.ml.trainers.group.UpdatesStrategy; + +import java.io.Serializable; +import java.util.Arrays; /** - * Linear regression trainer based on least squares loss function and gradient descent optimization algorithm. + * Trainer of the linear regression model based on stochastic gradient descent algorithm. */ -public class LinearRegressionSGDTrainer implements Trainer { - /** - * Gradient descent optimizer. - */ - private final GradientDescent gradientDescent; +public class LinearRegressionSGDTrainer

    implements SingleLabelDatasetTrainer { + /** Update strategy. */ + private final UpdatesStrategy updatesStgy; - /** */ - public LinearRegressionSGDTrainer(GradientDescent gradientDescent) { - this.gradientDescent = gradientDescent; - } + /** Max number of iteration. */ + private final int maxIterations; - /** */ - public LinearRegressionSGDTrainer(int maxIterations, double convergenceTol) { - this.gradientDescent = new GradientDescent(new LeastSquaresGradientFunction(), new BarzilaiBorweinUpdater()) - .withMaxIterations(maxIterations) - .withConvergenceTol(convergenceTol); - } + /** Batch size. */ + private final int batchSize; - /** */ - public LinearRegressionSGDTrainer(int maxIterations, double convergenceTol, double learningRate) { - this.gradientDescent = new GradientDescent(new LeastSquaresGradientFunction(), new SimpleUpdater(learningRate)) - .withMaxIterations(maxIterations) - .withConvergenceTol(convergenceTol); - } + /** Number of local iterations. */ + private final int locIterations; + + /** Seed for random generator. */ + private final long seed; /** - * {@inheritDoc} + * Constructs a new instance of linear regression SGD trainer. + * + * @param updatesStgy Update strategy. + * @param maxIterations Max number of iteration. + * @param batchSize Batch size. + * @param locIterations Number of local iterations. + * @param seed Seed for random generator. */ - @Override public LinearRegressionModel train(Matrix data) { - Vector variables = gradientDescent.optimize(data, data.likeVector(data.columnSize())); - Vector weights = variables.viewPart(1, variables.size() - 1); + public LinearRegressionSGDTrainer(UpdatesStrategy updatesStgy, int maxIterations, + int batchSize, int locIterations, long seed) { + this.updatesStgy = updatesStgy; + this.maxIterations = maxIterations; + this.batchSize = batchSize; + this.locIterations = locIterations; + this.seed = seed; + } + + /** {@inheritDoc} */ + @Override public LinearRegressionModel fit(DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + + IgniteFunction, MLPArchitecture> archSupplier = dataset -> { + + int cols = dataset.compute(data -> { + if (data.getFeatures() == null) + return null; + return data.getFeatures().length / data.getRows(); + }, (a, b) -> a == null ? b : a); + + MLPArchitecture architecture = new MLPArchitecture(cols); + architecture = architecture.withAddedLayer(1, true, Activators.LINEAR); + + return architecture; + }; + + MLPTrainer trainer = new MLPTrainer<>( + archSupplier, + LossFunctions.MSE, + updatesStgy, + maxIterations, + batchSize, + locIterations, + seed + ); + + IgniteBiFunction lbE = new IgniteBiFunction() { + @Override public double[] apply(K k, V v) { + return new double[]{lbExtractor.apply(k, v)}; + } + }; + + MultilayerPerceptron mlp = trainer.fit(datasetBuilder, featureExtractor, lbE); - double intercept = variables.get(0); + double[] p = mlp.parameters().getStorage().data(); - return new LinearRegressionModel(weights, intercept); + return new LinearRegressionModel(new DenseLocalOnHeapVector(Arrays.copyOf(p, p.length - 1)), p[p.length - 1]); } } 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 8119a2996f36e..fcde3f57ca45c 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 @@ -17,10 +17,16 @@ package org.apache.ignite.ml.trainers; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; import org.apache.ignite.ml.Model; import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; +import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import java.util.Map; + /** * Interface for trainers. Trainer is just a function which produces model from the data. * @@ -40,4 +46,44 @@ public interface DatasetTrainer { */ public M fit(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor); + + /** + * Trains model based on the specified data. + * + * @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 Model. + */ + public default M fit(Ignite ignite, IgniteCache cache, IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { + return fit( + new CacheBasedDatasetBuilder<>(ignite, cache), + featureExtractor, + lbExtractor + ); + } + + /** + * Trains model based on the specified data. + * + * @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 Model. + */ + public default M fit(Map data, int parts, IgniteBiFunction featureExtractor, + IgniteBiFunction lbExtractor) { + return fit( + new LocalDatasetBuilder<>(data, parts), + featureExtractor, + lbExtractor + ); + } } 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 b5a4b540a2232..b27fcba20d246 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 @@ -17,11 +17,7 @@ package org.apache.ignite.ml.knn; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; import org.apache.ignite.ml.knn.classification.KNNClassificationModel; import org.apache.ignite.ml.knn.classification.KNNClassificationTrainer; import org.apache.ignite.ml.knn.classification.KNNStrategy; @@ -29,6 +25,10 @@ import org.apache.ignite.ml.math.distances.EuclideanDistance; import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + /** Tests behaviour of KNNClassificationTest. */ public class KNNClassificationTest extends BaseKNNTest { /** */ @@ -46,7 +46,8 @@ public void testBinaryClassificationTest() { KNNClassificationTrainer trainer = new KNNClassificationTrainer(); KNNClassificationModel knnMdl = trainer.fit( - new LocalDatasetBuilder<>(data, 2), + data, + 2, (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1), (k, v) -> v[2] ).withK(3) @@ -74,7 +75,8 @@ public void testBinaryClassificationWithSmallestKTest() { KNNClassificationTrainer trainer = new KNNClassificationTrainer(); KNNClassificationModel knnMdl = trainer.fit( - new LocalDatasetBuilder<>(data, 2), + data, + 2, (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1), (k, v) -> v[2] ).withK(1) @@ -102,7 +104,8 @@ public void testBinaryClassificationFarPointsWithSimpleStrategy() { KNNClassificationTrainer trainer = new KNNClassificationTrainer(); KNNClassificationModel knnMdl = trainer.fit( - new LocalDatasetBuilder<>(data, 2), + data, + 2, (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1), (k, v) -> v[2] ).withK(3) @@ -128,7 +131,8 @@ public void testBinaryClassificationFarPointsWithWeightedStrategy() { KNNClassificationTrainer trainer = new KNNClassificationTrainer(); KNNClassificationModel knnMdl = trainer.fit( - new LocalDatasetBuilder<>(data, 2), + data, + 2, (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1), (k, v) -> v[2] ).withK(3) diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerIntegrationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerIntegrationTest.java index 5ca661fdbc4b2..038b880f5cebc 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerIntegrationTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerIntegrationTest.java @@ -17,7 +17,6 @@ package org.apache.ignite.ml.nn; -import java.io.Serializable; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; @@ -25,22 +24,18 @@ import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.ml.TestUtils; -import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; import org.apache.ignite.ml.math.Matrix; import org.apache.ignite.ml.math.Tracer; import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; import org.apache.ignite.ml.nn.architecture.MLPArchitecture; import org.apache.ignite.ml.optimization.LossFunctions; -import org.apache.ignite.ml.optimization.updatecalculators.NesterovParameterUpdate; -import org.apache.ignite.ml.optimization.updatecalculators.NesterovUpdateCalculator; -import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate; -import org.apache.ignite.ml.optimization.updatecalculators.RPropUpdateCalculator; -import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDParameterUpdate; -import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDUpdateCalculator; +import org.apache.ignite.ml.optimization.updatecalculators.*; import org.apache.ignite.ml.trainers.group.UpdatesStrategy; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import java.io.Serializable; + /** * Tests for {@link MLPTrainer} that require to start the whole Ignite infrastructure. */ @@ -137,7 +132,8 @@ private

    void xorTest(UpdatesStrategy(ignite, xorCache), + ignite, + xorCache, (k, v) -> new double[]{ v.x, v.y }, (k, v) -> new double[]{ v.lb} ); 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 6906424959906..c53f6f104d84e 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 @@ -17,24 +17,13 @@ package org.apache.ignite.ml.nn; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; import org.apache.ignite.ml.TestUtils; -import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; import org.apache.ignite.ml.math.Matrix; import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; import org.apache.ignite.ml.nn.architecture.MLPArchitecture; import org.apache.ignite.ml.optimization.LossFunctions; -import org.apache.ignite.ml.optimization.updatecalculators.NesterovParameterUpdate; -import org.apache.ignite.ml.optimization.updatecalculators.NesterovUpdateCalculator; -import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate; -import org.apache.ignite.ml.optimization.updatecalculators.RPropUpdateCalculator; -import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDParameterUpdate; -import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDUpdateCalculator; +import org.apache.ignite.ml.optimization.updatecalculators.*; import org.apache.ignite.ml.trainers.group.UpdatesStrategy; import org.junit.Before; import org.junit.Test; @@ -42,6 +31,12 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + /** * Tests for {@link MLPTrainer} that don't require to start the whole Ignite infrastructure. */ @@ -140,7 +135,8 @@ private

    void xorTest(UpdatesStrategy(xorData, parts), + xorData, + parts, (k, v) -> v[0], (k, v) -> v[1] ); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistIntegrationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistIntegrationTest.java index c787a47da53c4..a64af9b5cc545 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistIntegrationTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistIntegrationTest.java @@ -17,13 +17,11 @@ package org.apache.ignite.ml.nn.performance; -import java.io.IOException; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; import org.apache.ignite.ml.math.Matrix; import org.apache.ignite.ml.math.VectorUtils; import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; @@ -38,6 +36,8 @@ import org.apache.ignite.ml.util.MnistUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import java.io.IOException; + /** * Tests {@link MLPTrainer} on the MNIST dataset that require to start the whole Ignite infrastructure. */ @@ -104,7 +104,8 @@ public void testMNIST() throws IOException { System.out.println("Start training..."); long start = System.currentTimeMillis(); MultilayerPerceptron mdl = trainer.fit( - new CacheBasedDatasetBuilder<>(ignite, trainingSet), + ignite, + trainingSet, (k, v) -> v.getPixels(), (k, v) -> VectorUtils.num2Vec(v.getLabel(), 10).getStorage().data() ); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistTest.java index 354af2cb68e58..d966484d548c6 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistTest.java @@ -17,10 +17,6 @@ package org.apache.ignite.ml.nn.performance; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; import org.apache.ignite.ml.math.Matrix; import org.apache.ignite.ml.math.VectorUtils; import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; @@ -35,6 +31,10 @@ import org.apache.ignite.ml.util.MnistUtils; import org.junit.Test; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + import static org.junit.Assert.assertTrue; /** @@ -74,7 +74,8 @@ public void testMNIST() throws IOException { System.out.println("Start training..."); long start = System.currentTimeMillis(); MultilayerPerceptron mdl = trainer.fit( - new LocalDatasetBuilder<>(trainingSet, 1), + trainingSet, + 1, (k, v) -> v.getPixels(), (k, v) -> VectorUtils.num2Vec(v.getLabel(), 10).getStorage().data() ); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/normalization/NormalizationTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/normalization/NormalizationTrainerTest.java index 15482539f22e3..e7a0d47b1f861 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/normalization/NormalizationTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/normalization/NormalizationTrainerTest.java @@ -17,15 +17,16 @@ package org.apache.ignite.ml.preprocessing.normalization; -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.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + import static org.junit.Assert.assertArrayEquals; /** @@ -66,8 +67,7 @@ public void testFit() { NormalizationPreprocessor preprocessor = standardizationTrainer.fit( datasetBuilder, - (k, v) -> v, - 3 + (k, v) -> v ); assertArrayEquals(new double[] {0, 4, 1}, preprocessor.getMin(), 1e-8); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java index 82b3a1b24df63..b3c9368194366 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java @@ -17,14 +17,7 @@ package org.apache.ignite.ml.regressions; -import org.apache.ignite.ml.regressions.linear.BlockDistributedLinearRegressionQRTrainerTest; -import org.apache.ignite.ml.regressions.linear.BlockDistributedLinearRegressionSGDTrainerTest; -import org.apache.ignite.ml.regressions.linear.DistributedLinearRegressionQRTrainerTest; -import org.apache.ignite.ml.regressions.linear.DistributedLinearRegressionSGDTrainerTest; -import org.apache.ignite.ml.regressions.linear.LinearRegressionLSQRTrainerTest; -import org.apache.ignite.ml.regressions.linear.LinearRegressionModelTest; -import org.apache.ignite.ml.regressions.linear.LocalLinearRegressionQRTrainerTest; -import org.apache.ignite.ml.regressions.linear.LocalLinearRegressionSGDTrainerTest; +import org.apache.ignite.ml.regressions.linear.*; import org.junit.runner.RunWith; import org.junit.runners.Suite; @@ -35,12 +28,10 @@ @Suite.SuiteClasses({ LinearRegressionModelTest.class, LocalLinearRegressionQRTrainerTest.class, - LocalLinearRegressionSGDTrainerTest.class, DistributedLinearRegressionQRTrainerTest.class, - DistributedLinearRegressionSGDTrainerTest.class, BlockDistributedLinearRegressionQRTrainerTest.class, - BlockDistributedLinearRegressionSGDTrainerTest.class, - LinearRegressionLSQRTrainerTest.class + LinearRegressionLSQRTrainerTest.class, + LinearRegressionSGDTrainerTest.class }) public class RegressionsTestSuite { // No-op. diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/BlockDistributedLinearRegressionSGDTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/BlockDistributedLinearRegressionSGDTrainerTest.java deleted file mode 100644 index 58037e2e8d883..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/BlockDistributedLinearRegressionSGDTrainerTest.java +++ /dev/null @@ -1,35 +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.regressions.linear; - -import org.apache.ignite.ml.math.impls.matrix.SparseBlockDistributedMatrix; -import org.apache.ignite.ml.math.impls.vector.SparseBlockDistributedVector; - -/** - * Tests for {@link LinearRegressionSGDTrainer} on {@link SparseBlockDistributedMatrix}. - */ -public class BlockDistributedLinearRegressionSGDTrainerTest extends GridAwareAbstractLinearRegressionTrainerTest { - /** */ - public BlockDistributedLinearRegressionSGDTrainerTest() { - super( - new LinearRegressionSGDTrainer(100_000, 1e-12), - SparseBlockDistributedMatrix::new, - SparseBlockDistributedVector::new, - 1e-2); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/DistributedLinearRegressionSGDTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/DistributedLinearRegressionSGDTrainerTest.java deleted file mode 100644 index 71d3b3ba6a15d..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/DistributedLinearRegressionSGDTrainerTest.java +++ /dev/null @@ -1,35 +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.regressions.linear; - -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.ml.math.impls.vector.SparseDistributedVector; - -/** - * Tests for {@link LinearRegressionSGDTrainer} on {@link SparseDistributedMatrix}. - */ -public class DistributedLinearRegressionSGDTrainerTest extends GridAwareAbstractLinearRegressionTrainerTest { - /** */ - public DistributedLinearRegressionSGDTrainerTest() { - super( - new LinearRegressionSGDTrainer(100_000, 1e-12), - SparseDistributedMatrix::new, - SparseDistributedVector::new, - 1e-2); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GridAwareAbstractLinearRegressionTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GridAwareAbstractLinearRegressionTrainerTest.java index 1a60b80addf49..9b75bd41392a6 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GridAwareAbstractLinearRegressionTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GridAwareAbstractLinearRegressionTrainerTest.java @@ -26,6 +26,9 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Test; +/** + * Grid aware abstract linear regression trainer test. + */ public abstract class GridAwareAbstractLinearRegressionTrainerTest extends GridCommonAbstractTest { /** Number of nodes in grid */ private static final int NODE_COUNT = 3; 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 e3f60ec9772a3..2414236a33198 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 @@ -17,14 +17,14 @@ package org.apache.ignite.ml.regressions.linear; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.Random; -import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; @@ -72,7 +72,8 @@ public void testSmallDataFit() { LinearRegressionLSQRTrainer trainer = new LinearRegressionLSQRTrainer(); LinearRegressionModel mdl = trainer.fit( - new LocalDatasetBuilder<>(data, parts), + data, + parts, (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1), (k, v) -> v[4] ); @@ -110,7 +111,8 @@ public void testBigDataFit() { LinearRegressionLSQRTrainer trainer = new LinearRegressionLSQRTrainer(); LinearRegressionModel mdl = trainer.fit( - new LocalDatasetBuilder<>(data, parts), + data, + parts, (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1), (k, v) -> v[coef.length] ); 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 new file mode 100644 index 0000000000000..fa8fac408b112 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainerTest.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.ml.regressions.linear; + +import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate; +import org.apache.ignite.ml.optimization.updatecalculators.RPropUpdateCalculator; +import org.apache.ignite.ml.trainers.group.UpdatesStrategy; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link LinearRegressionSGDTrainer}. + */ +@RunWith(Parameterized.class) +public class LinearRegressionSGDTrainerTest { + /** 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} + ); + } + + /** Number of partitions. */ + @Parameterized.Parameter + public int parts; + + /** + * Tests {@code fit()} method on a simple small dataset. + */ + @Test + public void testSmallDataFit() { + 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, 123L); + + LinearRegressionModel mdl = trainer.fit( + data, + parts, + (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1), + (k, v) -> v[4] + ); + + assertArrayEquals( + new double[] {72.26948107, 15.95144674, 24.07403921, 66.73038781}, + mdl.getWeights().getStorage().data(), + 1e-1 + ); + + assertEquals(2.8421709430404007e-14, mdl.getIntercept(), 1e-1); + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LocalLinearRegressionSGDTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LocalLinearRegressionSGDTrainerTest.java deleted file mode 100644 index bea164d603c48..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LocalLinearRegressionSGDTrainerTest.java +++ /dev/null @@ -1,35 +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.regressions.linear; - -import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; - -/** - * Tests for {@link LinearRegressionSGDTrainer} on {@link DenseLocalOnHeapMatrix}. - */ -public class LocalLinearRegressionSGDTrainerTest extends GenericLinearRegressionTrainerTest { - /** */ - public LocalLinearRegressionSGDTrainerTest() { - super( - new LinearRegressionSGDTrainer(100_000, 1e-12), - DenseLocalOnHeapMatrix::new, - DenseLocalOnHeapVector::new, - 1e-2); - } -} \ No newline at end of file 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 26ba2fbba117e..0befd9b503831 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 @@ -17,14 +17,14 @@ package org.apache.ignite.ml.svm; +import org.apache.ignite.ml.TestUtils; +import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; +import org.junit.Test; + import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; -import org.apache.ignite.ml.TestUtils; -import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; -import org.junit.Test; /** * Tests for {@link SVMLinearBinaryClassificationTrainer}. @@ -62,7 +62,8 @@ public void testTrainWithTheLinearlySeparableCase() { SVMLinearBinaryClassificationTrainer trainer = new SVMLinearBinaryClassificationTrainer(); SVMLinearBinaryClassificationModel mdl = trainer.fit( - new LocalDatasetBuilder<>(data, 10), + data, + 10, (k, v) -> Arrays.copyOfRange(v, 1, v.length), (k, v) -> v[0] ); 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 ad95eb4542c4c..31ab4d7d7a368 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 @@ -17,14 +17,14 @@ package org.apache.ignite.ml.svm; +import org.apache.ignite.ml.TestUtils; +import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; +import org.junit.Test; + import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; -import org.apache.ignite.ml.TestUtils; -import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; -import org.junit.Test; /** * Tests for {@link SVMLinearBinaryClassificationTrainer}. @@ -65,7 +65,8 @@ public void testTrainWithTheLinearlySeparableCase() { .withAmountOfIterations(20); SVMLinearMultiClassClassificationModel mdl = trainer.fit( - new LocalDatasetBuilder<>(data, 10), + data, + 10, (k, v) -> Arrays.copyOfRange(v, 1, v.length), (k, v) -> v[0] ); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerIntegrationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerIntegrationTest.java index 94bca3f83e2f6..d5b0b86229e0d 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerIntegrationTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerIntegrationTest.java @@ -17,16 +17,16 @@ package org.apache.ignite.ml.tree; -import java.util.Arrays; -import java.util.Random; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import java.util.Arrays; +import java.util.Random; + /** * Tests for {@link DecisionTreeClassificationTrainer} that require to start the whole Ignite infrastructure. */ @@ -77,7 +77,8 @@ public void testFit() { DecisionTreeClassificationTrainer trainer = new DecisionTreeClassificationTrainer(1, 0); DecisionTreeNode tree = trainer.fit( - new CacheBasedDatasetBuilder<>(ignite, data), + ignite, + data, (k, v) -> Arrays.copyOf(v, v.length - 1), (k, v) -> v[v.length - 1] ); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerTest.java index 2599bfe2b17e1..12ef698c2df06 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerTest.java @@ -17,17 +17,12 @@ package org.apache.ignite.ml.tree; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; -import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import java.util.*; + import static junit.framework.TestCase.assertEquals; import static junit.framework.TestCase.assertTrue; @@ -68,7 +63,8 @@ public void testFit() { DecisionTreeClassificationTrainer trainer = new DecisionTreeClassificationTrainer(1, 0); DecisionTreeNode tree = trainer.fit( - new LocalDatasetBuilder<>(data, parts), + data, + parts, (k, v) -> Arrays.copyOf(v, v.length - 1), (k, v) -> v[v.length - 1] ); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerIntegrationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerIntegrationTest.java index 754ff20f8046e..c2a463844c202 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerIntegrationTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerIntegrationTest.java @@ -17,16 +17,16 @@ package org.apache.ignite.ml.tree; -import java.util.Arrays; -import java.util.Random; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import java.util.Arrays; +import java.util.Random; + /** * Tests for {@link DecisionTreeRegressionTrainer} that require to start the whole Ignite infrastructure. */ @@ -77,7 +77,8 @@ public void testFit() { DecisionTreeRegressionTrainer trainer = new DecisionTreeRegressionTrainer(1, 0); DecisionTreeNode tree = trainer.fit( - new CacheBasedDatasetBuilder<>(ignite, data), + ignite, + data, (k, v) -> Arrays.copyOf(v, v.length - 1), (k, v) -> v[v.length - 1] ); 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 3bdbf60675a3f..bcfb53fc6b320 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 @@ -17,17 +17,12 @@ package org.apache.ignite.ml.tree; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; -import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import java.util.*; + import static junit.framework.TestCase.assertEquals; import static junit.framework.TestCase.assertTrue; @@ -68,7 +63,8 @@ public void testFit() { DecisionTreeRegressionTrainer trainer = new DecisionTreeRegressionTrainer(1, 0); DecisionTreeNode tree = trainer.fit( - new LocalDatasetBuilder<>(data, parts), + data, + parts, (k, v) -> Arrays.copyOf(v, v.length - 1), (k, v) -> v[v.length - 1] ); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTIntegrationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTIntegrationTest.java index b259ec9700de5..35f805e37e4a4 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTIntegrationTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTIntegrationTest.java @@ -17,13 +17,11 @@ package org.apache.ignite.ml.tree.performance; -import java.io.IOException; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; import org.apache.ignite.ml.nn.performance.MnistMLPTestUtil; import org.apache.ignite.ml.tree.DecisionTreeClassificationTrainer; import org.apache.ignite.ml.tree.DecisionTreeNode; @@ -31,6 +29,8 @@ import org.apache.ignite.ml.util.MnistUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import java.io.IOException; + /** * Tests {@link DecisionTreeClassificationTrainer} on the MNIST dataset that require to start the whole Ignite * infrastructure. For manual run. @@ -81,7 +81,8 @@ public void testMNIST() throws IOException { new SimpleStepFunctionCompressor<>()); DecisionTreeNode mdl = trainer.fit( - new CacheBasedDatasetBuilder<>(ignite, trainingSet), + ignite, + trainingSet, (k, v) -> v.getPixels(), (k, v) -> (double) v.getLabel() ); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTTest.java index 6dbd44c5919b3..b40c7ac2bfe6f 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTTest.java @@ -17,10 +17,6 @@ package org.apache.ignite.ml.tree.performance; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; import org.apache.ignite.ml.nn.performance.MnistMLPTestUtil; import org.apache.ignite.ml.tree.DecisionTreeClassificationTrainer; import org.apache.ignite.ml.tree.DecisionTreeNode; @@ -28,6 +24,10 @@ import org.apache.ignite.ml.util.MnistUtils; import org.junit.Test; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + import static junit.framework.TestCase.assertTrue; /** @@ -50,7 +50,8 @@ public void testMNIST() throws IOException { new SimpleStepFunctionCompressor<>()); DecisionTreeNode mdl = trainer.fit( - new LocalDatasetBuilder<>(trainingSet, 10), + trainingSet, + 10, (k, v) -> v.getPixels(), (k, v) -> (double) v.getLabel() ); From f143ad0057ddb326f6d8199bf660b354913e6b61 Mon Sep 17 00:00:00 2001 From: devozerov Date: Thu, 12 Apr 2018 15:02:57 +0300 Subject: [PATCH 0038/1463] IGNITE-8135: SQL: authentication for CREATE TABLE and DROP TABLE commands. This closes #3801. --- .../apache/ignite/client/ClientException.java | 3 +- .../client/thin/ClientQueryCursor.java | 6 ++- .../platform/client/ClientRequestHandler.java | 7 ++- .../ClientCacheSqlFieldsQueryRequest.java | 19 +++++-- .../security/SecurityContextHolder.java | 53 +++++++++++++++++++ .../query/h2/ddl/DdlStatementsProcessor.java | 9 ++++ 6 files changed, 91 insertions(+), 6 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityContextHolder.java diff --git a/modules/core/src/main/java/org/apache/ignite/client/ClientException.java b/modules/core/src/main/java/org/apache/ignite/client/ClientException.java index 05556358fb53c..b0d9f6cc5cfee 100644 --- a/modules/core/src/main/java/org/apache/ignite/client/ClientException.java +++ b/modules/core/src/main/java/org/apache/ignite/client/ClientException.java @@ -20,7 +20,7 @@ /** * Common thin client checked exception. */ -public class ClientException extends Exception { +public class ClientException extends RuntimeException { /** Serial version uid. */ private static final long serialVersionUID = 0L; @@ -28,6 +28,7 @@ public class ClientException extends Exception { * Constructs a new exception with {@code null} as its detail message. */ public ClientException() { + // No-op. } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientQueryCursor.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientQueryCursor.java index 9367cfd52a0af..086fab875bbf3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientQueryCursor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientQueryCursor.java @@ -54,6 +54,7 @@ class ClientQueryCursor implements QueryCursor { pager.close(); } catch (Exception ignored) { + // No-op. } } @@ -76,7 +77,10 @@ class ClientQueryCursor implements QueryCursor { currPageIt = currPage.iterator(); } catch (ClientException e) { - throw new RuntimeException("Failed to retrieve query results", e); + throw e; + } + catch (Exception e) { + throw new ClientException("Failed to retrieve query results", e); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java index faa50bcd0f094..5ed0d38d8eac5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java @@ -22,6 +22,7 @@ import org.apache.ignite.internal.processors.odbc.ClientListenerRequest; import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler; import org.apache.ignite.internal.processors.odbc.ClientListenerResponse; +import org.apache.ignite.internal.processors.security.SecurityContextHolder; /** * Thin client request handler. @@ -47,8 +48,10 @@ public class ClientRequestHandler implements ClientListenerRequestHandler { /** {@inheritDoc} */ @Override public ClientListenerResponse handle(ClientListenerRequest req) { - if (authCtx != null) + if (authCtx != null) { AuthorizationContext.context(authCtx); + SecurityContextHolder.set(ctx.securityContext()); + } try { return ((ClientRequest)req).process(ctx); @@ -56,6 +59,8 @@ public class ClientRequestHandler implements ClientListenerRequestHandler { finally { if (authCtx != null) AuthorizationContext.clear(); + + SecurityContextHolder.clear(); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlFieldsQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlFieldsQueryRequest.java index 3aa95bf2a1a37..53f6353cc2a50 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlFieldsQueryRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlFieldsQueryRequest.java @@ -28,8 +28,11 @@ import org.apache.ignite.internal.processors.platform.cache.PlatformCache; import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.internal.processors.platform.client.ClientStatus; +import org.apache.ignite.internal.processors.platform.client.IgniteClientException; import org.apache.ignite.internal.processors.query.QueryUtils; -import org.apache.ignite.plugin.security.SecurityPermission; +import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.plugin.security.SecurityException; /** * Sql query request. @@ -95,7 +98,7 @@ public ClientCacheSqlFieldsQueryRequest(BinaryRawReaderEx reader) { if (qry.getSchema() == null) { String schema = QueryUtils.normalizeSchemaName(desc.cacheName(), - desc.cacheConfiguration().getSqlSchema()); + desc.cacheConfiguration().getSqlSchema()); qry.setSchema(schema); } @@ -108,7 +111,7 @@ public ClientCacheSqlFieldsQueryRequest(BinaryRawReaderEx reader) { FieldsQueryCursor cur = curs.get(0); ClientCacheFieldsQueryCursor cliCur = new ClientCacheFieldsQueryCursor( - cur, qry.getPageSize(), ctx); + cur, qry.getPageSize(), ctx); long cursorId = ctx.resources().put(cliCur); @@ -119,6 +122,16 @@ public ClientCacheSqlFieldsQueryRequest(BinaryRawReaderEx reader) { catch (Exception e) { ctx.decrementCursors(); + SecurityException securityEx = X.cause(e, SecurityException.class); + + if (securityEx != null) { + throw new IgniteClientException( + ClientStatus.SECURITY_VIOLATION, + "Client is not authorized to perform this operation", + securityEx + ); + } + throw e; } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityContextHolder.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityContextHolder.java new file mode 100644 index 0000000000000..14d70c97cbfdb --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityContextHolder.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.security; + +import org.jetbrains.annotations.Nullable; + +/** + * Thread-local security context. + */ +public class SecurityContextHolder { + /** Context. */ + private static final ThreadLocal CTX = new ThreadLocal<>(); + + /** + * Get security context. + * + * @return Security context. + */ + @Nullable public static SecurityContext get() { + return CTX.get(); + } + + /** + * Set security context. + * + * @param ctx Context. + */ + public static void set(@Nullable SecurityContext ctx) { + CTX.set(ctx); + } + + /** + * Clear security context. + */ + public static void clear() { + set(null); + } +} 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 b14896991ba01..bc5c1e0208848 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 @@ -34,6 +34,8 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.authentication.AuthorizationContext; +import org.apache.ignite.internal.processors.authentication.UserManagementOperation; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.QueryCursorImpl; import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; @@ -56,6 +58,8 @@ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser; import org.apache.ignite.internal.processors.query.h2.sql.GridSqlStatement; import org.apache.ignite.internal.processors.query.schema.SchemaOperationException; +import org.apache.ignite.internal.processors.security.SecurityContext; +import org.apache.ignite.internal.processors.security.SecurityContextHolder; import org.apache.ignite.internal.sql.command.SqlAlterTableCommand; import org.apache.ignite.internal.sql.command.SqlAlterUserCommand; import org.apache.ignite.internal.sql.command.SqlCommand; @@ -67,6 +71,7 @@ import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.plugin.security.SecurityPermission; import org.h2.command.Prepared; import org.h2.command.ddl.AlterTableAlterColumn; import org.h2.command.ddl.CreateIndex; @@ -316,6 +321,8 @@ else if (stmt0 instanceof GridSqlDropIndex) { } } else if (stmt0 instanceof GridSqlCreateTable) { + ctx.security().authorize(null, SecurityPermission.CACHE_CREATE, SecurityContextHolder.get()); + GridSqlCreateTable cmd = (GridSqlCreateTable)stmt0; if (!F.eq(QueryUtils.DFLT_SCHEMA, cmd.schemaName())) @@ -349,6 +356,8 @@ else if (stmt0 instanceof GridSqlCreateTable) { } } else if (stmt0 instanceof GridSqlDropTable) { + ctx.security().authorize(null, SecurityPermission.CACHE_DESTROY, SecurityContextHolder.get()); + GridSqlDropTable cmd = (GridSqlDropTable)stmt0; if (!F.eq(QueryUtils.DFLT_SCHEMA, cmd.schemaName())) From 2c4a7a2e366d7308485ae5cc95d4b60e66b09589 Mon Sep 17 00:00:00 2001 From: devozerov Date: Thu, 12 Apr 2018 15:13:51 +0300 Subject: [PATCH 0039/1463] IGNITE-8230: SQL: Fixed backup number propagation in CREATE TABLE command. This closes #3803. --- .../processors/query/GridQueryProcessor.java | 7 ++-- .../query/h2/sql/GridSqlCreateTable.java | 7 ++-- .../cache/index/H2DynamicTableSelfTest.java | 37 +++++++++++++++++++ 3 files changed, 45 insertions(+), 6 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java index bde9427677963..03e5254183d19 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 @@ -1483,10 +1483,10 @@ else if (op instanceof SchemaAlterTableDropColumnOperation) { @SuppressWarnings("unchecked") public void dynamicTableCreate(String schemaName, QueryEntity entity, String templateName, String cacheName, String cacheGroup, @Nullable String dataRegion, String affinityKey, @Nullable CacheAtomicityMode atomicityMode, - @Nullable CacheWriteSynchronizationMode writeSyncMode, int backups, boolean ifNotExists) + @Nullable CacheWriteSynchronizationMode writeSyncMode, @Nullable Integer backups, boolean ifNotExists) throws IgniteCheckedException { assert !F.isEmpty(templateName); - assert backups >= 0; + assert backups == null || backups >= 0; CacheConfiguration ccfg = ctx.cache().getConfigFromTemplate(templateName); @@ -1525,7 +1525,8 @@ else if (QueryUtils.TEMPLATE_REPLICATED.equalsIgnoreCase(templateName)) if (writeSyncMode != null) ccfg.setWriteSynchronizationMode(writeSyncMode); - ccfg.setBackups(backups); + if (backups != null) + ccfg.setBackups(backups); ccfg.setSqlSchema(schemaName); ccfg.setSqlEscapeAll(true); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java index 3608aedb40dc7..de86d6aa71e8c 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java @@ -22,6 +22,7 @@ import java.util.List; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.jetbrains.annotations.Nullable; /** * CREATE TABLE statement. @@ -57,7 +58,7 @@ public class GridSqlCreateTable extends GridSqlStatement { private CacheWriteSynchronizationMode writeSyncMode; /** Backups number for new cache. */ - private int backups; + private Integer backups; /** Quietly ignore this command if table already exists. */ private boolean ifNotExists; @@ -184,14 +185,14 @@ public void writeSynchronizationMode(CacheWriteSynchronizationMode writeSyncMode /** * @return Backups number for new cache. */ - public int backups() { + @Nullable public Integer backups() { return backups; } /** * @param backups Backups number for new cache. */ - public void backups(int backups) { + public void backups(Integer backups) { this.backups = backups; } 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 b20bb59611afe..82247114fc062 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 @@ -88,7 +88,14 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest { /** Bad data region name. */ public static final String DATA_REGION_NAME_BAD = "my_data_region_bad"; + /** Cache with backups. */ + private static final String CACHE_NAME_BACKUPS = CACHE_NAME + "_backups"; + + /** Number of backups for backup test. */ + private static final int DFLT_BACKUPS = 2; + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") @Override protected void beforeTestsStarted() throws Exception { super.beforeTestsStarted(); @@ -98,6 +105,8 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest { client().addCacheConfiguration(cacheConfiguration()); client().addCacheConfiguration(cacheConfiguration().setName(CACHE_NAME + "_async") .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_ASYNC)); + + client().addCacheConfiguration(cacheConfiguration().setName(CACHE_NAME_BACKUPS).setBackups(DFLT_BACKUPS)); } /** {@inheritDoc} */ @@ -108,6 +117,7 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest { } /** {@inheritDoc} */ + @SuppressWarnings("unchecked") @Override protected void beforeTest() throws Exception { super.beforeTest(); @@ -120,6 +130,7 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest { execute("DROP TABLE IF EXISTS PUBLIC.\"Person\""); execute("DROP TABLE IF EXISTS PUBLIC.\"City\""); execute("DROP TABLE IF EXISTS PUBLIC.\"NameTest\""); + execute("DROP TABLE IF EXISTS PUBLIC.\"BackupTest\""); super.afterTest(); } @@ -495,6 +506,32 @@ private void doTestCreateTable(String tplCacheName, String cacheGrp, CacheMode c doTestCreateTable(tplCacheName, cacheGrp, cacheMode, writeSyncMode, false, additionalParams); } + /** + * Test backups propagation. + * + * @throws Exception If failed. + */ + @SuppressWarnings("unchecked") + public void testBackups() throws Exception { + String cacheName = "BackupTestCache"; + + execute("CREATE TABLE \"BackupTest\" (id BIGINT PRIMARY KEY, name VARCHAR) WITH \"template=" + + CACHE_NAME_BACKUPS + ", cache_name=" + cacheName + "\""); + + CacheConfiguration ccfg = grid(0).cache(cacheName).getConfiguration(CacheConfiguration.class); + + assertEquals(DFLT_BACKUPS, ccfg.getBackups()); + + execute("DROP TABLE PUBLIC.\"BackupTest\""); + + execute("CREATE TABLE \"BackupTest\" (id BIGINT PRIMARY KEY, name VARCHAR) WITH \"template=" + + CACHE_NAME_BACKUPS + ", cache_name=" + cacheName + ", backups=1\""); + + ccfg = grid(0).cache(cacheName).getConfiguration(CacheConfiguration.class); + + assertEquals(1, ccfg.getBackups()); + } + /** * Test that {@code CREATE TABLE} with given template cache name actually creates new cache, * H2 table and type descriptor on all nodes, optionally with cache type check. From 80f4340f61742988aaf9437eb08ed76644a1c8ca Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Thu, 12 Apr 2018 14:29:43 +0300 Subject: [PATCH 0040/1463] IGNITE-7871 Fixed condition for cache partitions validation. - Fixes #3804. Signed-off-by: dpavlov (cherry picked from commit 7a1d0ea) --- .../dht/preloader/GridDhtPartitionsExchangeFuture.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 dd4a57157a1e7..af5acd64257e7 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 @@ -2760,7 +2760,7 @@ private void validatePartitionsState() { || grpCtx.config().isReadThrough() || grpCtx.config().isWriteThrough() || grpCtx.config().getCacheStoreFactory() != null - || grpCtx.config().getRebalanceDelay() != -1 + || grpCtx.config().getRebalanceDelay() == -1 || grpCtx.config().getRebalanceMode() == CacheRebalanceMode.NONE) continue; From dfe17074593d9d12cbab7b60aa73e73c37bbffb7 Mon Sep 17 00:00:00 2001 From: Anton Kurbanov Date: Thu, 12 Apr 2018 20:31:50 +0300 Subject: [PATCH 0041/1463] IGNITE-8110 GridCacheWriteBehindStore.Flusher thread uses the wrong transformation from milliseconds to nanoseconds. - Fixes #3742. Signed-off-by: dpavlov (cherry picked from commit adaedb4) --- .../store/GridCacheWriteBehindStore.java | 2 +- .../GridCacheWriteBehindStoreSelfTest.java | 41 ++++++++++++++++++- 2 files changed, 40 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java index 44cadd6b0e12a..82ff3aaabac22 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java @@ -895,7 +895,7 @@ private class Flusher extends GridWorker { protected Thread thread; /** Cache flushing frequence in nanos. */ - protected long cacheFlushFreqNanos = cacheFlushFreq * 1000; + protected long cacheFlushFreqNanos = cacheFlushFreq * 1000 * 1000; /** Writer lock. */ private final Lock flusherWriterLock = new ReentrantLock(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreSelfTest.java index 9a487a4268726..af21fc831c0c0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreSelfTest.java @@ -106,6 +106,43 @@ public void testSimpleStoreWithoutCoalescing() throws Exception { testSimpleStore(false); } + /** + * Checks that write behind cache flush frequency was correctly adjusted to nanos expecting putAllCnt to be + * less or equal than elapsed time divided by flush frequency. + * + * @throws Exception If failed. + */ + public void testSimpleStoreFlushFrequencyWithoutCoalescing() throws Exception { + initStore(1, false); + + long writeBehindFlushFreqNanos = FLUSH_FREQUENCY * 1000 * 1000; + + int threshold = store.getWriteBehindStoreBatchSize() / 10; + + try { + long start = System.nanoTime(); + + for (int i = 0; i < threshold / 2; i++) + store.write(new CacheEntryImpl<>(i, "v" + i)); + + U.sleep(FLUSH_FREQUENCY + 300); + + for (int i = threshold / 2; i < threshold; i++) + store.write(new CacheEntryImpl<>(i, "v" + i)); + + long elapsed = System.nanoTime() - start; + + U.sleep(FLUSH_FREQUENCY + 300); + + int expFlushOps = (int)(1 + elapsed / writeBehindFlushFreqNanos); + + assertTrue(delegate.getPutAllCount() <= expFlushOps); + } + finally { + shutdownStore(); + } + } + /** * Simple store test. * @@ -254,7 +291,6 @@ private void testContinuousPut(boolean writeCoalescing) throws Exception { int delegatePutCnt = delegate.getPutAllCount(); - fut.get(); log().info(">>> [putCnt = " + actualPutCnt.get() + ", delegatePutCnt=" + delegatePutCnt + "]"); @@ -262,7 +298,8 @@ private void testContinuousPut(boolean writeCoalescing) throws Exception { assertTrue("No puts were made to the underlying store", delegatePutCnt > 0); if (store.getWriteCoalescing()) { assertTrue("Too many puts were made to the underlying store", delegatePutCnt < actualPutCnt.get() / 10); - } else { + } + else { assertTrue("Too few puts cnt=" + actualPutCnt.get() + " << storePutCnt=" + delegatePutCnt, delegatePutCnt > actualPutCnt.get() / 2); } } From fe99497528cd040ab3b4d5f7bfc40e788393a5ed Mon Sep 17 00:00:00 2001 From: Andrey Kuznetsov Date: Thu, 12 Apr 2018 21:23:28 +0300 Subject: [PATCH 0042/1463] IGNITE-7983: NPE fixed in transactions Signed-off-by: Andrey Gura --- .../cache/distributed/near/GridNearTxLocal.java | 9 +++++++++ 1 file changed, 9 insertions(+) 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 33f84f030db71..fc8a9a3fb2f6c 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 @@ -3916,6 +3916,15 @@ private IgniteInternalFuture> checkMissed( throw new GridClosureException(e); } + if (isRollbackOnly()) { + if (timedOut()) + throw new GridClosureException(new IgniteTxTimeoutCheckedException( + "Transaction has been timed out: " + GridNearTxLocal.this)); + else + throw new GridClosureException(new IgniteTxRollbackCheckedException( + "Transaction has been rolled back: " + GridNearTxLocal.this)); + } + return map; } }, From 6a77dd8b182091fe4e38850098c6334597c14a6d Mon Sep 17 00:00:00 2001 From: zaleslaw Date: Fri, 13 Apr 2018 12:49:56 +0300 Subject: [PATCH 0043/1463] IGNITE-7829: Adopt kNN regression example to the new Partitioned Dataset this closes #3798 (cherry picked from commit 8550d61) --- .../ml/knn/KNNClassificationExample.java | 4 +- .../examples/ml/knn/KNNRegressionExample.java | 310 ++++++++++++++++++ .../org/apache/ignite/ml/knn/KNNUtils.java | 10 +- .../KNNClassificationModel.java | 9 +- .../knn/partitions/KNNPartitionContext.java | 28 -- .../ml/knn/partitions/package-info.java | 22 -- .../ml/knn/regression/KNNRegressionModel.java | 7 +- .../partition/LabelPartitionContext.java | 28 -- .../LabelPartitionDataBuilderOnHeap.java | 1 - .../SVMLinearBinaryClassificationModel.java | 3 + .../SVMLinearBinaryClassificationTrainer.java | 9 +- ...VMLinearMultiClassClassificationModel.java | 3 + ...LinearMultiClassClassificationTrainer.java | 8 +- .../ignite/ml/svm/SVMPartitionContext.java | 28 -- .../ignite/ml/knn/KNNClassificationTest.java | 110 ++++--- .../ignite/ml/knn/KNNRegressionTest.java | 104 +++--- ...KNNTest.java => LabeledDatasetHelper.java} | 10 +- .../ignite/ml/knn/LabeledDatasetTest.java | 2 +- 18 files changed, 453 insertions(+), 243 deletions(-) create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNRegressionExample.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/knn/partitions/KNNPartitionContext.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/knn/partitions/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/LabelPartitionContext.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMPartitionContext.java rename modules/ml/src/test/java/org/apache/ignite/ml/knn/{BaseKNNTest.java => LabeledDatasetHelper.java} (93%) 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 39a8431a18761..15375a13ff306 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 @@ -80,7 +80,7 @@ public static void main(String[] args) throws InterruptedException { double prediction = knnMdl.apply(new DenseLocalOnHeapVector(inputs)); totalAmount++; - if(groundTruth != prediction) + if (groundTruth != prediction) amountOfErrors++; System.out.printf(">>> | %.4f\t\t| %.4f\t\t|\n", prediction, groundTruth); @@ -89,7 +89,7 @@ public static void main(String[] args) throws InterruptedException { System.out.println(">>> ---------------------------------"); 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)); } }); 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 new file mode 100644 index 0000000000000..76a07cd9ce244 --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNRegressionExample.java @@ -0,0 +1,310 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.knn; + +import java.util.Arrays; +import java.util.UUID; +import javax.cache.Cache; +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.cache.query.QueryCursor; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; +import org.apache.ignite.ml.knn.classification.KNNClassificationTrainer; +import org.apache.ignite.ml.knn.classification.KNNStrategy; +import org.apache.ignite.ml.knn.regression.KNNRegressionModel; +import org.apache.ignite.ml.knn.regression.KNNRegressionTrainer; +import org.apache.ignite.ml.math.distances.ManhattanDistance; +import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; +import org.apache.ignite.thread.IgniteThread; + +/** + * Run kNN regression trainer over distributed dataset. + * + * @see KNNClassificationTrainer + */ +public class KNNRegressionExample { + /** Run example. */ + public static void main(String[] args) throws InterruptedException { + System.out.println(); + System.out.println(">>> kNN regression over cached dataset usage example started."); + // Start ignite grid. + try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { + System.out.println(">>> Ignite grid started."); + + IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), + KNNRegressionExample.class.getSimpleName(), () -> { + IgniteCache dataCache = getTestCache(ignite); + + KNNRegressionTrainer trainer = new KNNRegressionTrainer(); + + KNNRegressionModel knnMdl = (KNNRegressionModel) trainer.fit( + new CacheBasedDatasetBuilder<>(ignite, dataCache), + (k, v) -> Arrays.copyOfRange(v, 1, v.length), + (k, v) -> v[0] + ).withK(5) + .withDistanceMeasure(new ManhattanDistance()) + .withStrategy(KNNStrategy.WEIGHTED); + + int totalAmount = 0; + // Calculate mean squared error (MSE) + double mse = 0.0; + // Calculate mean absolute error (MAE) + double mae = 0.0; + + try (QueryCursor> observations = dataCache.query(new ScanQuery<>())) { + for (Cache.Entry observation : observations) { + double[] val = observation.getValue(); + double[] inputs = Arrays.copyOfRange(val, 1, val.length); + double groundTruth = val[0]; + + double prediction = knnMdl.apply(new DenseLocalOnHeapVector(inputs)); + + mse += Math.pow(prediction - groundTruth, 2.0); + mae += Math.abs(prediction - groundTruth); + + totalAmount++; + } + + mse = mse / totalAmount; + System.out.println("\n>>> Mean squared error (MSE) " + mse); + + mae = mae / totalAmount; + System.out.println("\n>>> Mean absolute error (MAE) " + mae); + } + }); + + igniteThread.start(); + igniteThread.join(); + } + } + + /** + * Fills cache with data and returns it. + * + * @param ignite Ignite instance. + * @return Filled Ignite Cache. + */ + private static IgniteCache getTestCache(Ignite ignite) { + CacheConfiguration cacheConfiguration = new CacheConfiguration<>(); + cacheConfiguration.setName("TEST_" + UUID.randomUUID()); + 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; + } + + /** The Iris dataset. */ + private static final double[][] data = { + {199, 125, 256, 6000, 256, 16, 128}, + {253, 29, 8000, 32000, 32, 8, 32}, + {132, 29, 8000, 16000, 32, 8, 16}, + {290, 26, 8000, 32000, 64, 8, 32}, + {381, 23, 16000, 32000, 64, 16, 32}, + {749, 23, 16000, 64000, 64, 16, 32}, + {1238, 23, 32000, 64000, 128, 32, 64}, + {23, 400, 1000, 3000, 0, 1, 2}, + {24, 400, 512, 3500, 4, 1, 6}, + {70, 60, 2000, 8000, 65, 1, 8}, + {117, 50, 4000, 16000, 65, 1, 8}, + {15, 350, 64, 64, 0, 1, 4}, + {64, 200, 512, 16000, 0, 4, 32}, + {23, 167, 524, 2000, 8, 4, 15}, + {29, 143, 512, 5000, 0, 7, 32}, + {22, 143, 1000, 2000, 0, 5, 16}, + {124, 110, 5000, 5000, 142, 8, 64}, + {35, 143, 1500, 6300, 0, 5, 32}, + {39, 143, 3100, 6200, 0, 5, 20}, + {40, 143, 2300, 6200, 0, 6, 64}, + {45, 110, 3100, 6200, 0, 6, 64}, + {28, 320, 128, 6000, 0, 1, 12}, + {21, 320, 512, 2000, 4, 1, 3}, + {28, 320, 256, 6000, 0, 1, 6}, + {22, 320, 256, 3000, 4, 1, 3}, + {28, 320, 512, 5000, 4, 1, 5}, + {27, 320, 256, 5000, 4, 1, 6}, + {102, 25, 1310, 2620, 131, 12, 24}, + {74, 50, 2620, 10480, 30, 12, 24}, + {138, 56, 5240, 20970, 30, 12, 24}, + {136, 64, 5240, 20970, 30, 12, 24}, + {23, 50, 500, 2000, 8, 1, 4}, + {29, 50, 1000, 4000, 8, 1, 5}, + {44, 50, 2000, 8000, 8, 1, 5}, + {30, 50, 1000, 4000, 8, 3, 5}, + {41, 50, 1000, 8000, 8, 3, 5}, + {74, 50, 2000, 16000, 8, 3, 5}, + {54, 133, 1000, 12000, 9, 3, 12}, + {41, 133, 1000, 8000, 9, 3, 12}, + {18, 810, 512, 512, 8, 1, 1}, + {28, 810, 1000, 5000, 0, 1, 1}, + {36, 320, 512, 8000, 4, 1, 5}, + {38, 200, 512, 8000, 8, 1, 8}, + {34, 700, 384, 8000, 0, 1, 1}, + {19, 700, 256, 2000, 0, 1, 1}, + {72, 140, 1000, 16000, 16, 1, 3}, + {36, 200, 1000, 8000, 0, 1, 2}, + {30, 110, 1000, 4000, 16, 1, 2}, + {56, 110, 1000, 12000, 16, 1, 2}, + {42, 220, 1000, 8000, 16, 1, 2}, + {34, 800, 256, 8000, 0, 1, 4}, + {19, 125, 512, 1000, 0, 8, 20}, + {75, 75, 2000, 8000, 64, 1, 38}, + {113, 75, 2000, 16000, 64, 1, 38}, + {157, 75, 2000, 16000, 128, 1, 38}, + {18, 90, 256, 1000, 0, 3, 10}, + {20, 105, 256, 2000, 0, 3, 10}, + {28, 105, 1000, 4000, 0, 3, 24}, + {33, 105, 2000, 4000, 8, 3, 19}, + {47, 75, 2000, 8000, 8, 3, 24}, + {54, 75, 3000, 8000, 8, 3, 48}, + {20, 175, 256, 2000, 0, 3, 24}, + {23, 300, 768, 3000, 0, 6, 24}, + {25, 300, 768, 3000, 6, 6, 24}, + {52, 300, 768, 12000, 6, 6, 24}, + {27, 300, 768, 4500, 0, 1, 24}, + {50, 300, 384, 12000, 6, 1, 24}, + {18, 300, 192, 768, 6, 6, 24}, + {53, 180, 768, 12000, 6, 1, 31}, + {23, 330, 1000, 3000, 0, 2, 4}, + {30, 300, 1000, 4000, 8, 3, 64}, + {73, 300, 1000, 16000, 8, 2, 112}, + {20, 330, 1000, 2000, 0, 1, 2}, + {25, 330, 1000, 4000, 0, 3, 6}, + {28, 140, 2000, 4000, 0, 3, 6}, + {29, 140, 2000, 4000, 0, 4, 8}, + {32, 140, 2000, 4000, 8, 1, 20}, + {175, 140, 2000, 32000, 32, 1, 20}, + {57, 140, 2000, 8000, 32, 1, 54}, + {181, 140, 2000, 32000, 32, 1, 54}, + {32, 140, 2000, 4000, 8, 1, 20}, + {82, 57, 4000, 16000, 1, 6, 12}, + {171, 57, 4000, 24000, 64, 12, 16}, + {361, 26, 16000, 32000, 64, 16, 24}, + {350, 26, 16000, 32000, 64, 8, 24}, + {220, 26, 8000, 32000, 0, 8, 24}, + {113, 26, 8000, 16000, 0, 8, 16}, + {15, 480, 96, 512, 0, 1, 1}, + {21, 203, 1000, 2000, 0, 1, 5}, + {35, 115, 512, 6000, 16, 1, 6}, + {18, 1100, 512, 1500, 0, 1, 1}, + {20, 1100, 768, 2000, 0, 1, 1}, + {20, 600, 768, 2000, 0, 1, 1}, + {28, 400, 2000, 4000, 0, 1, 1}, + {45, 400, 4000, 8000, 0, 1, 1}, + {18, 900, 1000, 1000, 0, 1, 2}, + {17, 900, 512, 1000, 0, 1, 2}, + {26, 900, 1000, 4000, 4, 1, 2}, + {28, 900, 1000, 4000, 8, 1, 2}, + {28, 900, 2000, 4000, 0, 3, 6}, + {31, 225, 2000, 4000, 8, 3, 6}, + {42, 180, 2000, 8000, 8, 1, 6}, + {76, 185, 2000, 16000, 16, 1, 6}, + {76, 180, 2000, 16000, 16, 1, 6}, + {26, 225, 1000, 4000, 2, 3, 6}, + {59, 25, 2000, 12000, 8, 1, 4}, + {65, 25, 2000, 12000, 16, 3, 5}, + {101, 17, 4000, 16000, 8, 6, 12}, + {116, 17, 4000, 16000, 32, 6, 12}, + {18, 1500, 768, 1000, 0, 0, 0}, + {20, 1500, 768, 2000, 0, 0, 0}, + {20, 800, 768, 2000, 0, 0, 0}, + {30, 50, 2000, 4000, 0, 3, 6}, + {44, 50, 2000, 8000, 8, 3, 6}, + {82, 50, 2000, 16000, 24, 1, 6}, + {128, 50, 8000, 16000, 48, 1, 10}, + {37, 100, 1000, 8000, 0, 2, 6}, + {46, 100, 1000, 8000, 24, 2, 6}, + {46, 100, 1000, 8000, 24, 3, 6}, + {80, 50, 2000, 16000, 12, 3, 16}, + {88, 50, 2000, 16000, 24, 6, 16}, + {33, 150, 512, 4000, 0, 8, 128}, + {46, 115, 2000, 8000, 16, 1, 3}, + {29, 115, 2000, 4000, 2, 1, 5}, + {53, 92, 2000, 8000, 32, 1, 6}, + {41, 92, 2000, 8000, 4, 1, 6}, + {86, 75, 4000, 16000, 16, 1, 6}, + {95, 60, 4000, 16000, 32, 1, 6}, + {107, 60, 2000, 16000, 64, 5, 8}, + {117, 60, 4000, 16000, 64, 5, 8}, + {119, 50, 4000, 16000, 64, 5, 10}, + {120, 72, 4000, 16000, 64, 8, 16}, + {48, 72, 2000, 8000, 16, 6, 8}, + {126, 40, 8000, 16000, 32, 8, 16}, + {266, 40, 8000, 32000, 64, 8, 24}, + {270, 35, 8000, 32000, 64, 8, 24}, + {426, 38, 16000, 32000, 128, 16, 32}, + {151, 48, 4000, 24000, 32, 8, 24}, + {267, 38, 8000, 32000, 64, 8, 24}, + {603, 30, 16000, 32000, 256, 16, 24}, + {19, 112, 1000, 1000, 0, 1, 4}, + {21, 84, 1000, 2000, 0, 1, 6}, + {26, 56, 1000, 4000, 0, 1, 6}, + {35, 56, 2000, 6000, 0, 1, 8}, + {41, 56, 2000, 8000, 0, 1, 8}, + {47, 56, 4000, 8000, 0, 1, 8}, + {62, 56, 4000, 12000, 0, 1, 8}, + {78, 56, 4000, 16000, 0, 1, 8}, + {80, 38, 4000, 8000, 32, 16, 32}, + {142, 38, 8000, 16000, 64, 4, 8}, + {281, 38, 8000, 24000, 160, 4, 8}, + {190, 38, 4000, 16000, 128, 16, 32}, + {21, 200, 1000, 2000, 0, 1, 2}, + {25, 200, 1000, 4000, 0, 1, 4}, + {67, 200, 2000, 8000, 64, 1, 5}, + {24, 250, 512, 4000, 0, 1, 7}, + {24, 250, 512, 4000, 0, 4, 7}, + {64, 250, 1000, 16000, 1, 1, 8}, + {25, 160, 512, 4000, 2, 1, 5}, + {20, 160, 512, 2000, 2, 3, 8}, + {29, 160, 1000, 4000, 8, 1, 14}, + {43, 160, 1000, 8000, 16, 1, 14}, + {53, 160, 2000, 8000, 32, 1, 13}, + {19, 240, 512, 1000, 8, 1, 3}, + {22, 240, 512, 2000, 8, 1, 5}, + {31, 105, 2000, 4000, 8, 3, 8}, + {41, 105, 2000, 6000, 16, 6, 16}, + {47, 105, 2000, 8000, 16, 4, 14}, + {99, 52, 4000, 16000, 32, 4, 12}, + {67, 70, 4000, 12000, 8, 6, 8}, + {81, 59, 4000, 12000, 32, 6, 12}, + {149, 59, 8000, 16000, 64, 12, 24}, + {183, 26, 8000, 24000, 32, 8, 16}, + {275, 26, 8000, 32000, 64, 12, 16}, + {382, 26, 8000, 32000, 128, 24, 32}, + {56, 116, 2000, 8000, 32, 5, 28}, + {182, 50, 2000, 32000, 24, 6, 26}, + {227, 50, 2000, 32000, 48, 26, 52}, + {341, 50, 2000, 32000, 112, 52, 104}, + {360, 50, 4000, 32000, 112, 52, 104}, + {919, 30, 8000, 64000, 96, 12, 176}, + {978, 30, 8000, 64000, 128, 12, 176}, + {24, 180, 262, 4000, 0, 1, 3}, + {37, 124, 1000, 8000, 0, 1, 8}, + {50, 98, 1000, 8000, 32, 2, 8}, + {41, 125, 2000, 8000, 0, 2, 14}, + {47, 480, 512, 8000, 32, 0, 0}, + {25, 480, 1000, 4000, 0, 0, 0} + }; +} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/KNNUtils.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/KNNUtils.java index 88fa70f8c9ea2..716eb526c8fab 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/KNNUtils.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/KNNUtils.java @@ -20,7 +20,7 @@ import org.apache.ignite.ml.dataset.Dataset; import org.apache.ignite.ml.dataset.DatasetBuilder; import org.apache.ignite.ml.dataset.PartitionDataBuilder; -import org.apache.ignite.ml.knn.partitions.KNNPartitionContext; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.structures.LabeledDataset; import org.apache.ignite.ml.structures.LabeledVector; @@ -39,18 +39,18 @@ public class KNNUtils { * @param lbExtractor Label extractor. * @return Dataset. */ - @Nullable public static Dataset> buildDataset(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { - PartitionDataBuilder> partDataBuilder + @Nullable public static Dataset> buildDataset(DatasetBuilder datasetBuilder, IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + PartitionDataBuilder> partDataBuilder = new LabeledDatasetPartitionDataBuilderOnHeap<>( featureExtractor, lbExtractor ); - Dataset> dataset = null; + Dataset> dataset = null; if (datasetBuilder != null) { dataset = datasetBuilder.build( - (upstream, upstreamSize) -> new KNNPartitionContext(), + (upstream, upstreamSize) -> new EmptyContext(), partDataBuilder ); } 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 373f822d7276a..693b81df0ecff 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 @@ -32,7 +32,7 @@ import org.apache.ignite.ml.Exporter; import org.apache.ignite.ml.Model; import org.apache.ignite.ml.dataset.Dataset; -import org.apache.ignite.ml.knn.partitions.KNNPartitionContext; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; import org.apache.ignite.ml.math.Vector; import org.apache.ignite.ml.math.distances.DistanceMeasure; import org.apache.ignite.ml.math.distances.EuclideanDistance; @@ -44,6 +44,9 @@ * kNN algorithm model to solve multi-class classification task. */ public class KNNClassificationModel implements Model, Exportable { + /** */ + private static final long serialVersionUID = -127386523291350345L; + /** Amount of nearest neighbors. */ protected int k = 5; @@ -54,13 +57,13 @@ public class KNNClassificationModel implements Model, Expo protected KNNStrategy stgy = KNNStrategy.SIMPLE; /** Dataset. */ - private Dataset> dataset; + private Dataset> dataset; /** * Builds the model via prepared dataset. * @param dataset Specially prepared object to run algorithm over it. */ - public KNNClassificationModel(Dataset> dataset) { + public KNNClassificationModel(Dataset> dataset) { this.dataset = dataset; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/partitions/KNNPartitionContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/partitions/KNNPartitionContext.java deleted file mode 100644 index 0081612754e37..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/partitions/KNNPartitionContext.java +++ /dev/null @@ -1,28 +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.knn.partitions; - -import java.io.Serializable; - -/** - * Partition context of the kNN classification algorithm. - */ -public class KNNPartitionContext implements Serializable { - /** */ - private static final long serialVersionUID = -7212307112344430126L; -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/partitions/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/partitions/package-info.java deleted file mode 100644 index 951a8490444df..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/partitions/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. - */ - -/** - * - * Contains helper classes for kNN classification algorithms. - */ -package org.apache.ignite.ml.knn.partitions; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionModel.java index cabc1438e1d77..f5def43634bfe 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionModel.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionModel.java @@ -17,8 +17,8 @@ package org.apache.ignite.ml.knn.regression; import org.apache.ignite.ml.dataset.Dataset; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; import org.apache.ignite.ml.knn.classification.KNNClassificationModel; -import org.apache.ignite.ml.knn.partitions.KNNPartitionContext; import org.apache.ignite.ml.math.Vector; import org.apache.ignite.ml.math.exceptions.UnsupportedOperationException; import org.apache.ignite.ml.structures.LabeledDataset; @@ -38,11 +38,14 @@ * */ public class KNNRegressionModel extends KNNClassificationModel { + /** */ + private static final long serialVersionUID = -721836321291120543L; + /** * Builds the model via prepared dataset. * @param dataset Specially prepared object to run algorithm over it. */ - public KNNRegressionModel(Dataset> dataset) { + public KNNRegressionModel(Dataset> dataset) { super(dataset); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/LabelPartitionContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/LabelPartitionContext.java deleted file mode 100644 index 1069ff8ab00c5..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/LabelPartitionContext.java +++ /dev/null @@ -1,28 +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.structures.partition; - -import java.io.Serializable; - -/** - * Base partition context. - */ -public class LabelPartitionContext implements Serializable { - /** */ - private static final long serialVersionUID = -7412302212344430126L; -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/LabelPartitionDataBuilderOnHeap.java b/modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/LabelPartitionDataBuilderOnHeap.java index 14c053e24acb0..4fba0289d447f 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/LabelPartitionDataBuilderOnHeap.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/LabelPartitionDataBuilderOnHeap.java @@ -22,7 +22,6 @@ 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.structures.LabeledDataset; /** * Partition data builder that builds {@link LabelPartitionDataOnHeap}. diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationModel.java index dace8c6b3bcad..f806fb8da28a1 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationModel.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationModel.java @@ -28,6 +28,9 @@ * Base class for SVM linear classification model. */ public class SVMLinearBinaryClassificationModel implements Model, Exportable, Serializable { + /** */ + private static final long serialVersionUID = -996984622291440226L; + /** Output label format. -1 and +1 for false value and raw distances from the separating hyperplane otherwise. */ private boolean isKeepingRawLabels = false; 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 7f11e209b1fd6..d56848c9d6d45 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 @@ -18,6 +18,7 @@ package org.apache.ignite.ml.svm; import java.util.concurrent.ThreadLocalRandom; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; import org.apache.ignite.ml.structures.partition.LabeledDatasetPartitionDataBuilderOnHeap; import org.apache.ignite.ml.trainers.SingleLabelDatasetTrainer; import org.apache.ignite.ml.dataset.Dataset; @@ -59,15 +60,15 @@ public class SVMLinearBinaryClassificationTrainer implements SingleLabelDatasetT assert datasetBuilder != null; - PartitionDataBuilder> partDataBuilder = new LabeledDatasetPartitionDataBuilderOnHeap<>( + PartitionDataBuilder> partDataBuilder = new LabeledDatasetPartitionDataBuilderOnHeap<>( featureExtractor, lbExtractor ); Vector weights; - try(Dataset> dataset = datasetBuilder.build( - (upstream, upstreamSize) -> new SVMPartitionContext(), + try(Dataset> dataset = datasetBuilder.build( + (upstream, upstreamSize) -> new EmptyContext(), partDataBuilder )) { final int cols = dataset.compute(data -> data.colSize(), (a, b) -> a == null ? b : a); @@ -90,7 +91,7 @@ public class SVMLinearBinaryClassificationTrainer implements SingleLabelDatasetT } /** */ - 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()); 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 5879ef095214a..bbec791e56652 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 @@ -29,6 +29,9 @@ /** Base class for multi-classification model for set of SVM classifiers. */ public class SVMLinearMultiClassClassificationModel implements Model, Exportable, Serializable { + /** */ + private static final long serialVersionUID = -667986511191350227L; + /** List of models associated with each class. */ private Map models; 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 88c342d78a201..4e081c6123e6a 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 @@ -24,12 +24,12 @@ import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.ignite.ml.dataset.primitive.context.EmptyContext; import org.apache.ignite.ml.trainers.SingleLabelDatasetTrainer; import org.apache.ignite.ml.dataset.Dataset; import org.apache.ignite.ml.dataset.DatasetBuilder; import org.apache.ignite.ml.dataset.PartitionDataBuilder; import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.structures.partition.LabelPartitionContext; import org.apache.ignite.ml.structures.partition.LabelPartitionDataBuilderOnHeap; import org.apache.ignite.ml.structures.partition.LabelPartitionDataOnHeap; @@ -89,12 +89,12 @@ public class SVMLinearMultiClassClassificationTrainer private List extractClassLabels(DatasetBuilder datasetBuilder, IgniteBiFunction lbExtractor) { assert datasetBuilder != null; - PartitionDataBuilder partDataBuilder = new LabelPartitionDataBuilderOnHeap<>(lbExtractor); + PartitionDataBuilder partDataBuilder = new LabelPartitionDataBuilderOnHeap<>(lbExtractor); List res = new ArrayList<>(); - try (Dataset dataset = datasetBuilder.build( - (upstream, upstreamSize) -> new LabelPartitionContext(), + try (Dataset dataset = datasetBuilder.build( + (upstream, upstreamSize) -> new EmptyContext(), partDataBuilder )) { final Set clsLabels = dataset.compute(data -> { diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMPartitionContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMPartitionContext.java deleted file mode 100644 index 0aee0fbf140cb..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMPartitionContext.java +++ /dev/null @@ -1,28 +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.io.Serializable; - -/** - * Partition context of the SVM classification algorithm. - */ -public class SVMPartitionContext implements Serializable { - /** */ - private static final long serialVersionUID = -7212307112344430126L; -} 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 b27fcba20d246..0877fc07cd16c 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 @@ -17,31 +17,35 @@ package org.apache.ignite.ml.knn; -import org.apache.ignite.internal.util.IgniteUtils; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.junit.Assert; +import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; import org.apache.ignite.ml.knn.classification.KNNClassificationModel; import org.apache.ignite.ml.knn.classification.KNNClassificationTrainer; import org.apache.ignite.ml.knn.classification.KNNStrategy; import org.apache.ignite.ml.math.Vector; import org.apache.ignite.ml.math.distances.EuclideanDistance; import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; - -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; +import org.junit.Test; /** Tests behaviour of KNNClassificationTest. */ -public class KNNClassificationTest extends BaseKNNTest { +public class KNNClassificationTest { + /** Precision in test checks. */ + private static final double PRECISION = 1e-2; + /** */ - public void testBinaryClassificationTest() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); + @Test + public void binaryClassificationTest() { 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}); + 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}); KNNClassificationTrainer trainer = new KNNClassificationTrainer(); @@ -54,23 +58,23 @@ public void testBinaryClassificationTest() { .withDistanceMeasure(new EuclideanDistance()) .withStrategy(KNNStrategy.SIMPLE); - Vector firstVector = new DenseLocalOnHeapVector(new double[] {2.0, 2.0}); - assertEquals(knnMdl.apply(firstVector), 1.0); - Vector secondVector = new DenseLocalOnHeapVector(new double[] {-2.0, -2.0}); - assertEquals(knnMdl.apply(secondVector), 2.0); + Vector firstVector = new DenseLocalOnHeapVector(new double[]{2.0, 2.0}); + Assert.assertEquals(knnMdl.apply(firstVector), 1.0, PRECISION); + Vector secondVector = new DenseLocalOnHeapVector(new double[]{-2.0, -2.0}); + Assert.assertEquals(knnMdl.apply(secondVector), 2.0, PRECISION); } /** */ - public void testBinaryClassificationWithSmallestKTest() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - + @Test + public void binaryClassificationWithSmallestKTest() { 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}); + + 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}); KNNClassificationTrainer trainer = new KNNClassificationTrainer(); @@ -83,23 +87,23 @@ public void testBinaryClassificationWithSmallestKTest() { .withDistanceMeasure(new EuclideanDistance()) .withStrategy(KNNStrategy.SIMPLE); - Vector firstVector = new DenseLocalOnHeapVector(new double[] {2.0, 2.0}); - assertEquals(knnMdl.apply(firstVector), 1.0); - Vector secondVector = new DenseLocalOnHeapVector(new double[] {-2.0, -2.0}); - assertEquals(knnMdl.apply(secondVector), 2.0); + Vector firstVector = new DenseLocalOnHeapVector(new double[]{2.0, 2.0}); + Assert.assertEquals(knnMdl.apply(firstVector), 1.0, PRECISION); + Vector secondVector = new DenseLocalOnHeapVector(new double[]{-2.0, -2.0}); + Assert.assertEquals(knnMdl.apply(secondVector), 2.0, PRECISION); } /** */ - public void testBinaryClassificationFarPointsWithSimpleStrategy() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - + @Test + public void binaryClassificationFarPointsWithSimpleStrategy() { 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}); + + 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(); @@ -112,21 +116,21 @@ public void testBinaryClassificationFarPointsWithSimpleStrategy() { .withDistanceMeasure(new EuclideanDistance()) .withStrategy(KNNStrategy.SIMPLE); - Vector vector = new DenseLocalOnHeapVector(new double[] {-1.01, -1.01}); - assertEquals(knnMdl.apply(vector), 2.0); + Vector vector = new DenseLocalOnHeapVector(new double[]{-1.01, -1.01}); + Assert.assertEquals(knnMdl.apply(vector), 2.0, PRECISION); } /** */ - public void testBinaryClassificationFarPointsWithWeightedStrategy() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - + @Test + public void binaryClassificationFarPointsWithWeightedStrategy() { 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}); + + 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(); @@ -139,7 +143,7 @@ public void testBinaryClassificationFarPointsWithWeightedStrategy() { .withDistanceMeasure(new EuclideanDistance()) .withStrategy(KNNStrategy.WEIGHTED); - Vector vector = new DenseLocalOnHeapVector(new double[] {-1.01, -1.01}); - assertEquals(knnMdl.apply(vector), 1.0); + Vector vector = new DenseLocalOnHeapVector(new double[]{-1.01, -1.01}); + Assert.assertEquals(knnMdl.apply(vector), 1.0, PRECISION); } } 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 66dbca9ff038e..ce9cae59cda80 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 @@ -17,7 +17,6 @@ package org.apache.ignite.ml.knn; -import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; import org.apache.ignite.ml.knn.classification.KNNStrategy; import org.apache.ignite.ml.knn.regression.KNNRegressionModel; @@ -30,28 +29,23 @@ import java.util.Arrays; import java.util.HashMap; import java.util.Map; +import org.junit.Test; /** * Tests for {@link KNNRegressionTrainer}. */ -public class KNNRegressionTest extends BaseKNNTest { +public class KNNRegressionTest { /** */ - private double[] y; - - /** */ - private double[][] x; - - /** */ - public void testSimpleRegressionWithOneNeighbour() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - + @Test + public void simpleRegressionWithOneNeighbour() { 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}); + + 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(); @@ -63,32 +57,31 @@ public void testSimpleRegressionWithOneNeighbour() { .withDistanceMeasure(new EuclideanDistance()) .withStrategy(KNNStrategy.SIMPLE); - Vector vector = new DenseLocalOnHeapVector(new double[] {0, 0, 0, 5.0, 0.0}); + Vector vector = new DenseLocalOnHeapVector(new double[]{0, 0, 0, 5.0, 0.0}); System.out.println(knnMdl.apply(vector)); Assert.assertEquals(15, knnMdl.apply(vector), 1E-12); } /** */ - public void testLongly() { - - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - + @Test + public void longly() { Map data = new HashMap<>(); - data.put(0, new double[] {60323, 83.0, 234289, 2356, 1590, 107608, 1947}); - data.put(1, new double[] {61122, 88.5, 259426, 2325, 1456, 108632, 1948}); - data.put(2, new double[] {60171, 88.2, 258054, 3682, 1616, 109773, 1949}); - data.put(3, new double[] {61187, 89.5, 284599, 3351, 1650, 110929, 1950}); - data.put(4, new double[] {63221, 96.2, 328975, 2099, 3099, 112075, 1951}); - data.put(5, new double[] {63639, 98.1, 346999, 1932, 3594, 113270, 1952}); - data.put(6, new double[] {64989, 99.0, 365385, 1870, 3547, 115094, 1953}); - data.put(7, new double[] {63761, 100.0, 363112, 3578, 3350, 116219, 1954}); - data.put(8, new double[] {66019, 101.2, 397469, 2904, 3048, 117388, 1955}); - data.put(9, new double[] {68169, 108.4, 442769, 2936, 2798, 120445, 1957}); - data.put(10, new double[] {66513, 110.8, 444546, 4681, 2637, 121950, 1958}); - data.put(11, new double[] {68655, 112.6, 482704, 3813, 2552, 123366, 1959}); - data.put(12, new double[] {69564, 114.2, 502601, 3931, 2514, 125368, 1960}); - data.put(13, new double[] {69331, 115.7, 518173, 4806, 2572, 127852, 1961}); - data.put(14, new double[] {70551, 116.9, 554894, 4007, 2827, 130081, 1962}); + + data.put(0, new double[]{60323, 83.0, 234289, 2356, 1590, 107608, 1947}); + data.put(1, new double[]{61122, 88.5, 259426, 2325, 1456, 108632, 1948}); + data.put(2, new double[]{60171, 88.2, 258054, 3682, 1616, 109773, 1949}); + data.put(3, new double[]{61187, 89.5, 284599, 3351, 1650, 110929, 1950}); + data.put(4, new double[]{63221, 96.2, 328975, 2099, 3099, 112075, 1951}); + data.put(5, new double[]{63639, 98.1, 346999, 1932, 3594, 113270, 1952}); + data.put(6, new double[]{64989, 99.0, 365385, 1870, 3547, 115094, 1953}); + data.put(7, new double[]{63761, 100.0, 363112, 3578, 3350, 116219, 1954}); + data.put(8, new double[]{66019, 101.2, 397469, 2904, 3048, 117388, 1955}); + data.put(9, new double[]{68169, 108.4, 442769, 2936, 2798, 120445, 1957}); + data.put(10, new double[]{66513, 110.8, 444546, 4681, 2637, 121950, 1958}); + data.put(11, new double[]{68655, 112.6, 482704, 3813, 2552, 123366, 1959}); + data.put(12, new double[]{69564, 114.2, 502601, 3931, 2514, 125368, 1960}); + data.put(13, new double[]{69331, 115.7, 518173, 4806, 2572, 127852, 1961}); + data.put(14, new double[]{70551, 116.9, 554894, 4007, 2827, 130081, 1962}); KNNRegressionTrainer trainer = new KNNRegressionTrainer(); @@ -100,31 +93,30 @@ public void testLongly() { .withDistanceMeasure(new EuclideanDistance()) .withStrategy(KNNStrategy.SIMPLE); - Vector vector = new DenseLocalOnHeapVector(new double[] {104.6, 419180, 2822, 2857, 118734, 1956}); + Vector vector = new DenseLocalOnHeapVector(new double[]{104.6, 419180, 2822, 2857, 118734, 1956}); System.out.println(knnMdl.apply(vector)); Assert.assertEquals(67857, knnMdl.apply(vector), 2000); } /** */ public void testLonglyWithWeightedStrategy() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - Map data = new HashMap<>(); - data.put(0, new double[] {60323, 83.0, 234289, 2356, 1590, 107608, 1947}); - data.put(1, new double[] {61122, 88.5, 259426, 2325, 1456, 108632, 1948}); - data.put(2, new double[] {60171, 88.2, 258054, 3682, 1616, 109773, 1949}); - data.put(3, new double[] {61187, 89.5, 284599, 3351, 1650, 110929, 1950}); - data.put(4, new double[] {63221, 96.2, 328975, 2099, 3099, 112075, 1951}); - data.put(5, new double[] {63639, 98.1, 346999, 1932, 3594, 113270, 1952}); - data.put(6, new double[] {64989, 99.0, 365385, 1870, 3547, 115094, 1953}); - data.put(7, new double[] {63761, 100.0, 363112, 3578, 3350, 116219, 1954}); - data.put(8, new double[] {66019, 101.2, 397469, 2904, 3048, 117388, 1955}); - data.put(9, new double[] {68169, 108.4, 442769, 2936, 2798, 120445, 1957}); - data.put(10, new double[] {66513, 110.8, 444546, 4681, 2637, 121950, 1958}); - data.put(11, new double[] {68655, 112.6, 482704, 3813, 2552, 123366, 1959}); - data.put(12, new double[] {69564, 114.2, 502601, 3931, 2514, 125368, 1960}); - data.put(13, new double[] {69331, 115.7, 518173, 4806, 2572, 127852, 1961}); - data.put(14, new double[] {70551, 116.9, 554894, 4007, 2827, 130081, 1962}); + + data.put(0, new double[]{60323, 83.0, 234289, 2356, 1590, 107608, 1947}); + data.put(1, new double[]{61122, 88.5, 259426, 2325, 1456, 108632, 1948}); + data.put(2, new double[]{60171, 88.2, 258054, 3682, 1616, 109773, 1949}); + data.put(3, new double[]{61187, 89.5, 284599, 3351, 1650, 110929, 1950}); + data.put(4, new double[]{63221, 96.2, 328975, 2099, 3099, 112075, 1951}); + data.put(5, new double[]{63639, 98.1, 346999, 1932, 3594, 113270, 1952}); + data.put(6, new double[]{64989, 99.0, 365385, 1870, 3547, 115094, 1953}); + data.put(7, new double[]{63761, 100.0, 363112, 3578, 3350, 116219, 1954}); + data.put(8, new double[]{66019, 101.2, 397469, 2904, 3048, 117388, 1955}); + data.put(9, new double[]{68169, 108.4, 442769, 2936, 2798, 120445, 1957}); + data.put(10, new double[]{66513, 110.8, 444546, 4681, 2637, 121950, 1958}); + data.put(11, new double[]{68655, 112.6, 482704, 3813, 2552, 123366, 1959}); + data.put(12, new double[]{69564, 114.2, 502601, 3931, 2514, 125368, 1960}); + data.put(13, new double[]{69331, 115.7, 518173, 4806, 2572, 127852, 1961}); + data.put(14, new double[]{70551, 116.9, 554894, 4007, 2827, 130081, 1962}); KNNRegressionTrainer trainer = new KNNRegressionTrainer(); @@ -136,7 +128,7 @@ public void testLonglyWithWeightedStrategy() { .withDistanceMeasure(new EuclideanDistance()) .withStrategy(KNNStrategy.SIMPLE); - Vector vector = new DenseLocalOnHeapVector(new double[] {104.6, 419180, 2822, 2857, 118734, 1956}); + Vector vector = new DenseLocalOnHeapVector(new double[]{104.6, 419180, 2822, 2857, 118734, 1956}); System.out.println(knnMdl.apply(vector)); Assert.assertEquals(67857, knnMdl.apply(vector), 2000); } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/BaseKNNTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetHelper.java similarity index 93% rename from modules/ml/src/test/java/org/apache/ignite/ml/knn/BaseKNNTest.java rename to modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetHelper.java index aeac2cf2e8b8d..a25b303480780 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/BaseKNNTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetHelper.java @@ -29,7 +29,7 @@ /** * Base class for decision trees test. */ -public class BaseKNNTest extends GridCommonAbstractTest { +public class LabeledDatasetHelper extends GridCommonAbstractTest { /** Count of nodes. */ private static final int NODE_COUNT = 4; @@ -42,7 +42,7 @@ public class BaseKNNTest extends GridCommonAbstractTest { /** * Default constructor. */ - public BaseKNNTest() { + public LabeledDatasetHelper() { super(false); } @@ -75,12 +75,10 @@ LabeledDataset loadDatasetFromTxt(String rsrcPath, boolean isFallOnBadData) { Path path = Paths.get(this.getClass().getClassLoader().getResource(rsrcPath).toURI()); try { return LabeledDatasetLoader.loadFromTxtFile(path, SEPARATOR, false, isFallOnBadData); - } - catch (IOException e) { + } catch (IOException e) { e.printStackTrace(); } - } - catch (URISyntaxException e) { + } catch (URISyntaxException e) { e.printStackTrace(); return null; } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetTest.java index cdd5dc414fc03..77d40a61a0d4f 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetTest.java @@ -34,7 +34,7 @@ import org.apache.ignite.ml.structures.preprocessing.LabeledDatasetLoader; /** Tests behaviour of KNNClassificationTest. */ -public class LabeledDatasetTest extends BaseKNNTest implements ExternalizableTest { +public class LabeledDatasetTest extends LabeledDatasetHelper implements ExternalizableTest { /** */ private static final String KNN_IRIS_TXT = "datasets/knn/iris.txt"; From 687ae653bd66745c49ba9f85a169e27191ddc16c Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Fri, 13 Apr 2018 12:28:19 +0300 Subject: [PATCH 0044/1463] IGNITE-8240 .NET: Use default scheduler when starting Tasks This closes #3812 --- .gitignore | 1 + .../IgniteSessionStateStoreProviderTest.cs | 7 +- .../BenchmarkRunner.cs | 1 - .../Binary/BinaryDynamicRegistrationTest.cs | 2 +- .../Cache/CacheAbstractTransactionalTest.cs | 5 +- .../Client/ClientConnectionTest.cs | 3 +- .../Apache.Ignite.Core.Tests/EventsTest.cs | 9 +-- .../ExceptionsTest.cs | 3 +- .../IgniteStartStopTest.cs | 5 +- .../Apache.Ignite.Core.Tests/MessagingTest.cs | 5 +- .../Apache.Ignite.Core.csproj | 1 + .../Impl/Client/ClientSocket.cs | 6 +- .../Impl/Common/TaskRunner.cs | 70 +++++++++++++++++++ .../Impl/Datastream/DataStreamerBatch.cs | 2 +- .../Impl/Datastream/DataStreamerImpl.cs | 2 +- .../Apache.Ignite.Core/Impl/Events/Events.cs | 2 +- .../Impl/Transactions/TransactionImpl.cs | 3 +- 17 files changed, 103 insertions(+), 24 deletions(-) create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/TaskRunner.cs diff --git a/.gitignore b/.gitignore index 535a8ff093818..47220b27df9b8 100644 --- a/.gitignore +++ b/.gitignore @@ -29,6 +29,7 @@ git-patch-prop-local.sh **/dotnet/libs/ *.classname* *.exe +.mvn/ #Visual Studio files *.[Oo]bj diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs index 25700c6aa3a5f..08c44a6b0f261 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs @@ -28,6 +28,7 @@ namespace Apache.Ignite.AspNet.Tests using System.Web.SessionState; using Apache.Ignite.Core; using Apache.Ignite.Core.Common; + using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Tests; using NUnit.Framework; @@ -265,7 +266,7 @@ public void TestCaching() Assert.AreEqual(SessionStateActions.None, actions); // Try to get it in a different thread. - Task.Factory.StartNew(() => + TaskRunner.Run(() => { object lockId1; // do not overwrite lockId res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId1, out actions); @@ -277,7 +278,7 @@ public void TestCaching() }).Wait(); // Try to get it in a different thread. - Task.Factory.StartNew(() => + TaskRunner.Run(() => { object lockId1; // do not overwrite lockId res = provider.GetItemExclusive(HttpContext, Id, out locked, out lockAge, out lockId1, out actions); @@ -292,7 +293,7 @@ public void TestCaching() provider.ReleaseItemExclusive(HttpContext, Id, lockId); // Make sure it is accessible in a different thread. - Task.Factory.StartNew(() => + TaskRunner.Run(() => { res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions); Assert.IsNotNull(res); diff --git a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/BenchmarkRunner.cs b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/BenchmarkRunner.cs index e152ffb8a81a8..fb2fbd28063f5 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/BenchmarkRunner.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/BenchmarkRunner.cs @@ -22,7 +22,6 @@ namespace Apache.Ignite.Benchmarks using System.IO; using System.Text; using Apache.Ignite.Benchmarks.Interop; - using Apache.Ignite.Benchmarks.ThinClient; ///

    /// Benchmark runner. diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryDynamicRegistrationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryDynamicRegistrationTest.cs index e635bd12cdb20..272a0cae819f6 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryDynamicRegistrationTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryDynamicRegistrationTest.cs @@ -399,7 +399,7 @@ public void TestRegistrationMultithreaded([Values(true, false)] bool useTypeName }; var tasks = Enumerable.Range(0, threads) - .Select(x => Task.Factory.StartNew(registerType)) + .Select(x => TaskRunner.Run(registerType)) .ToArray(); Task.WaitAll(tasks); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTransactionalTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTransactionalTest.cs index 2602a02efb75c..3d0168c08d6cb 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTransactionalTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTransactionalTest.cs @@ -25,6 +25,7 @@ namespace Apache.Ignite.Core.Tests.Cache using System.Transactions; using Apache.Ignite.Core.Cache; using Apache.Ignite.Core.Cache.Configuration; + using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Transactions; using NUnit.Framework; @@ -563,8 +564,8 @@ public void TestTxDeadlockDetection() var aex = Assert.Throws(() => Task.WaitAll(new[] { - Task.Factory.StartNew(() => increment(keys0)), - Task.Factory.StartNew(() => increment(keys0.Reverse().ToArray())) + TaskRunner.Run(() => increment(keys0)), + TaskRunner.Run(() => increment(keys0.Reverse().ToArray())) }, TimeSpan.FromSeconds(40))); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs index 2ea17a8739227..cb30f40c874be 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs @@ -31,6 +31,7 @@ namespace Apache.Ignite.Core.Tests.Client using Apache.Ignite.Core.Client; using Apache.Ignite.Core.Client.Cache; using Apache.Ignite.Core.Configuration; + using Apache.Ignite.Core.Impl.Common; using NUnit.Framework; /// @@ -310,7 +311,7 @@ public void TestServerConnectionAborted() var evt = new ManualResetEventSlim(); var ignite = Ignition.Start(TestUtils.GetTestConfiguration()); - var putGetTask = Task.Factory.StartNew(() => + var putGetTask = TaskRunner.Run(() => { using (var client = StartClient()) { diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs index a7c05344e963f..e9bac023a41bf 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs @@ -34,6 +34,7 @@ namespace Apache.Ignite.Core.Tests using Apache.Ignite.Core.Common; using Apache.Ignite.Core.Events; using Apache.Ignite.Core.Impl; + using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Impl.Events; using Apache.Ignite.Core.Resource; using Apache.Ignite.Core.Tests.Compute; @@ -385,14 +386,14 @@ public void TestWaitForLocal() /// private static IEnumerable, int[], Task>> GetWaitTasks(IEvents events) { - yield return (filter, types) => Task.Factory.StartNew(() => events.WaitForLocal(types)); - yield return (filter, types) => Task.Factory.StartNew(() => events.WaitForLocal(types.ToList())); + yield return (filter, types) => TaskRunner.Run(() => events.WaitForLocal(types)); + yield return (filter, types) => TaskRunner.Run(() => events.WaitForLocal(types.ToList())); yield return (filter, types) => events.WaitForLocalAsync(types); yield return (filter, types) => events.WaitForLocalAsync(types.ToList()); - yield return (filter, types) => Task.Factory.StartNew(() => events.WaitForLocal(filter, types)); - yield return (filter, types) => Task.Factory.StartNew(() => events.WaitForLocal(filter, types.ToList())); + yield return (filter, types) => TaskRunner.Run(() => events.WaitForLocal(filter, types)); + yield return (filter, types) => TaskRunner.Run(() => events.WaitForLocal(filter, types.ToList())); yield return (filter, types) => events.WaitForLocalAsync(filter, types); yield return (filter, types) => events.WaitForLocalAsync(filter, types.ToList()); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs index f7568ef19e38f..0b06ea34bde97 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs @@ -29,6 +29,7 @@ namespace Apache.Ignite.Core.Tests using Apache.Ignite.Core.Cluster; using Apache.Ignite.Core.Common; using Apache.Ignite.Core.Compute; + using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Services; using Apache.Ignite.Core.Transactions; using NUnit.Framework; @@ -348,7 +349,7 @@ private static void TestPartialUpdateException(bool async, Func(); // Do cache puts in parallel - var putTask = Task.Factory.StartNew(() => + var putTask = TaskRunner.Run(() => { try { diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs index 792b33d484cda..f9c1cad8055cd 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs @@ -23,6 +23,7 @@ namespace Apache.Ignite.Core.Tests using System.Threading; using System.Threading.Tasks; using Apache.Ignite.Core.Common; + using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Messaging; using Apache.Ignite.Core.Tests.Process; using NUnit.Framework; @@ -207,7 +208,7 @@ public void TestStartStopLeak() if (i % 2 == 0) // Try to stop ignite from another thread. { - Task.Factory.StartNew(() => grid.Dispose()).Wait(); + TaskRunner.Run(() => grid.Dispose()).Wait(); } else { @@ -306,7 +307,7 @@ public void TestProcessorInit() // Spam message subscriptions on a separate thread // to test race conditions during processor init on remote node - var listenTask = Task.Factory.StartNew(() => + var listenTask = TaskRunner.Run(() => { var filter = new MessageListener(); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs index e644e317d5f97..7db4eef09f44a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs @@ -27,6 +27,7 @@ namespace Apache.Ignite.Core.Tests using Apache.Ignite.Core.Cache.Configuration; using Apache.Ignite.Core.Cluster; using Apache.Ignite.Core.Common; + using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Messaging; using Apache.Ignite.Core.Resource; using Apache.Ignite.Core.Tests.Cache; @@ -252,7 +253,7 @@ public void TestLocalListenMultithreaded() var messaging = _grid1.GetMessaging(); - var senders = Task.Factory.StartNew(() => TestUtils.RunMultiThreaded(() => + var senders = TaskRunner.Run(() => TestUtils.RunMultiThreaded(() => { messaging.Send(NextMessage()); Thread.Sleep(50); @@ -423,7 +424,7 @@ public void TestRemoteListenMultithreaded() var messaging = _grid1.GetMessaging(); - var senders = Task.Factory.StartNew(() => TestUtils.RunMultiThreaded(() => + var senders = TaskRunner.Run(() => TestUtils.RunMultiThreaded(() => { MessagingTestHelper.ClearReceived(int.MaxValue); messaging.Send(NextMessage()); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj index ec84a38fa77a7..93c45c34d4caa 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj @@ -72,6 +72,7 @@ + diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs index 27d8f0bea092a..bce681f8ca557 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs @@ -28,11 +28,11 @@ namespace Apache.Ignite.Core.Impl.Client using System.Net.Sockets; using System.Threading; using System.Threading.Tasks; - using System.Xml.Schema; using Apache.Ignite.Core.Client; using Apache.Ignite.Core.Common; using Apache.Ignite.Core.Impl.Binary; using Apache.Ignite.Core.Impl.Binary.IO; + using Apache.Ignite.Core.Impl.Common; /// /// Wrapper over framework socket for Ignite thin client operations. @@ -117,7 +117,7 @@ public ClientSocket(IgniteClientConfiguration clientConfiguration, ClientProtoco } // Continuously and asynchronously wait for data from server. - Task.Factory.StartNew(WaitForMessages); + TaskRunner.Run(WaitForMessages); } /// @@ -174,7 +174,7 @@ public Task DoOutInOpAsync(ClientOp opId, Action writeActio var task = SendRequestAsync(ref reqMsg); // Decode. - return task.ContinueWith(responseTask => DecodeResponse(responseTask.Result, readFunc, errorFunc)); + return task.ContWith(responseTask => DecodeResponse(responseTask.Result, readFunc, errorFunc)); } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/TaskRunner.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/TaskRunner.cs new file mode 100644 index 0000000000000..51a7c6a98bd43 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/TaskRunner.cs @@ -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. + */ + +namespace Apache.Ignite.Core.Impl.Common +{ + using System; + using System.Threading; + using System.Threading.Tasks; + + /// + /// Extensions for classes. + /// Fixes the issue with being used by defaut by system APIs. + /// + internal static class TaskRunner + { + /// + /// ContinueWith using default scheduler. + /// + public static Task ContWith(this Task task, + Func, TNewResult> continuationFunction) + { + IgniteArgumentCheck.NotNull(task, "task"); + + return task.ContinueWith(continuationFunction, TaskScheduler.Default); + } + + /// + /// ContinueWith using default scheduler. + /// + public static Task ContWith(this Task task, + Action continuationFunction) + { + IgniteArgumentCheck.NotNull(task, "task"); + + return task.ContinueWith(continuationFunction, TaskScheduler.Default); + } + + /// + /// Run new task using default scheduler. + /// + public static Task Run(Action action) + { + return Task.Factory.StartNew(action, CancellationToken.None, TaskCreationOptions.None, + TaskScheduler.Default); + } + + /// + /// Run new task using default scheduler. + /// + public static Task Run(Func func) + { + return Task.Factory.StartNew(func, CancellationToken.None, TaskCreationOptions.None, + TaskScheduler.Default); + } + } +} \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerBatch.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerBatch.cs index 38a8ea885ac38..002670113071c 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerBatch.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerBatch.cs @@ -69,7 +69,7 @@ public DataStreamerBatch(DataStreamerBatch prev) if (prev != null) Thread.MemoryBarrier(); // Prevent "prev" field escape. - _fut.Task.ContinueWith(x => ParentsCompleted()); + _fut.Task.ContWith(x => ParentsCompleted()); } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs index 555c6e6c05b7c..7aaa84a8009e9 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs @@ -897,7 +897,7 @@ public void Stop() /// public void RunThread() { - Task.Factory.StartNew(Run); + TaskRunner.Run(Run); } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs index a81523a3e13f2..04cc210a3370e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs @@ -241,7 +241,7 @@ public Task WaitForLocalAsync(IEventFilter filter, params int[] types) if (hnd != null) { // Dispose handle as soon as future ends. - task.ContinueWith(x => Ignite.HandleRegistry.Release(hnd.Value)); + task.ContWith(x => Ignite.HandleRegistry.Release(hnd.Value)); } return task; diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionImpl.cs index 0b04a6803a975..c80085969c093 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionImpl.cs @@ -21,6 +21,7 @@ namespace Apache.Ignite.Core.Impl.Transactions using System.Globalization; using System.Threading; using System.Threading.Tasks; + using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Transactions; /// @@ -457,7 +458,7 @@ private void ThrowIfClosed() /// private Task CloseWhenComplete(Task task) { - return task.ContinueWith(x => Close()); + return task.ContWith(x => Close()); } /** */ From 0a64e4affae9ec2e16c3a99128985f6bd9c788cb Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Fri, 13 Apr 2018 12:44:17 +0300 Subject: [PATCH 0045/1463] IGNITE-8042: .NET: Thin client: authentication support - fix naming and inspections --- .../Client/ClientConnectionTest.cs | 12 ++++++------ .../Client/IgniteClientConfiguration.cs | 4 ++-- .../IgniteClientConfigurationSection.xsd | 2 +- .../Apache.Ignite.Core/Impl/Client/ClientSocket.cs | 14 +++++++------- 4 files changed, 16 insertions(+), 16 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs index cb30f40c874be..0a6b1a76a71eb 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs @@ -98,11 +98,11 @@ public void TestAuthenticationEmptyCredentials() cliCfg.Password = "ignite"; - cliCfg.Username = null; + cliCfg.UserName = null; ex = Assert.Throws(() => { Ignition.StartClient(cliCfg); }); Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.Username cannot be null")); - cliCfg.Username = ""; + cliCfg.UserName = ""; ex = Assert.Throws(() => { Ignition.StartClient(cliCfg); }); Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.Username cannot be empty")); } @@ -118,12 +118,12 @@ public void TestAuthenticationInvalidCredentials() { var cliCfg = SecureClientConfig(); - cliCfg.Username = "invalid"; + cliCfg.UserName = "invalid"; var ex = Assert.Throws(() => { Ignition.StartClient(cliCfg); }); Assert.True(ex.StatusCode == ClientStatusCode.AuthenticationFailed); - cliCfg.Username = "ignite"; + cliCfg.UserName = "ignite"; cliCfg.Password = "invalid"; ex = Assert.Throws(() => { Ignition.StartClient(cliCfg); }); @@ -165,7 +165,7 @@ public void TestAuthentication() var cliCfg = SecureClientConfig(); - cliCfg.Username = "my_User"; + cliCfg.UserName = "my_User"; cliCfg.Password = "my_Password"; using (var cli = Ignition.StartClient(cliCfg)) @@ -532,7 +532,7 @@ private static IgniteClientConfiguration SecureClientConfig() return new IgniteClientConfiguration() { Host = "localhost", - Username = "ignite", + UserName = "ignite", Password = "ignite" }; } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs index 32524955ee4c2..80f26cfa5e7d6 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs @@ -91,7 +91,7 @@ public IgniteClientConfiguration(IgniteClientConfiguration cfg) : this() BinaryProcessor = cfg.BinaryProcessor; SslStreamFactory = cfg.SslStreamFactory; - Username = cfg.Username; + UserName = cfg.UserName; Password = cfg.Password; } @@ -151,7 +151,7 @@ public IgniteClientConfiguration(IgniteClientConfiguration cfg) : this() /// /// Username to be used to connect to secured cluster. /// - public string Username { get; set; } + public string UserName { get; set; } /// /// Password to be used to connect to secured cluster. diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd index 7e6caff6c4d19..b9a04b8e0d75d 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd @@ -237,7 +237,7 @@ Socket operation timeout. Zero or negative for infinite timeout. - + Username to be used to connect to secured cluster. diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs index bce681f8ca557..11d79420ed365 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs @@ -124,11 +124,11 @@ public ClientSocket(IgniteClientConfiguration clientConfiguration, ClientProtoco /// Validate configuration. /// /// Configuration. - private void Validate(IgniteClientConfiguration cfg) + private static void Validate(IgniteClientConfiguration cfg) { - if (cfg.Username != null) + if (cfg.UserName != null) { - if (cfg.Username.Length == 0) + if (cfg.UserName.Length == 0) throw new IgniteClientException("IgniteClientConfiguration.Username cannot be empty."); if (cfg.Password == null) @@ -140,8 +140,8 @@ private void Validate(IgniteClientConfiguration cfg) if (cfg.Password.Length == 0) throw new IgniteClientException("IgniteClientConfiguration.Password cannot be empty."); - if (cfg.Username == null) - throw new IgniteClientException("IgniteClientConfiguration.Username cannot be null when Password is set."); + if (cfg.UserName == null) + throw new IgniteClientException("IgniteClientConfiguration.UserName cannot be null when Password is set."); } } @@ -262,7 +262,7 @@ private static T DecodeResponse(BinaryHeapStream stream, Func private void Handshake(IgniteClientConfiguration clientConfiguration, ClientProtocolVersion version) { - bool auth = version.CompareTo(Ver110) >= 0 && clientConfiguration.Username != null; + bool auth = version.CompareTo(Ver110) >= 0 && clientConfiguration.UserName != null; // Send request. int messageLen; @@ -284,7 +284,7 @@ private void Handshake(IgniteClientConfiguration clientConfiguration, ClientProt { var writer = BinaryUtils.Marshaller.StartMarshal(stream); - writer.WriteString(clientConfiguration.Username); + writer.WriteString(clientConfiguration.UserName); writer.WriteString(clientConfiguration.Password); BinaryUtils.Marshaller.FinishMarshal(writer); From 6072af825ca1507ad9d8143ca73e556539960e1d Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Fri, 13 Apr 2018 13:36:20 +0300 Subject: [PATCH 0046/1463] IGNITE-8042: .NET: Thin client: authentication support - fix TestAuthenticationEmptyCredentials --- .../Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs index 0a6b1a76a71eb..67d1c52cfe001 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs @@ -100,7 +100,7 @@ public void TestAuthenticationEmptyCredentials() cliCfg.UserName = null; ex = Assert.Throws(() => { Ignition.StartClient(cliCfg); }); - Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.Username cannot be null")); + Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.UserName cannot be null")); cliCfg.UserName = ""; ex = Assert.Throws(() => { Ignition.StartClient(cliCfg); }); From 78e7414cf568ef7e3f7567fdb74334012896b632 Mon Sep 17 00:00:00 2001 From: Dmitriy Shabalin Date: Fri, 13 Apr 2018 17:55:02 +0700 Subject: [PATCH 0047/1463] IGNITE-8245 Fixed input appearance position with error. (cherry picked from commit 56e3f43) --- .../frontend/app/primitives/form-field/index.scss | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/modules/web-console/frontend/app/primitives/form-field/index.scss b/modules/web-console/frontend/app/primitives/form-field/index.scss index 7d9ea1ffd655d..1035adecfc6c9 100644 --- a/modules/web-console/frontend/app/primitives/form-field/index.scss +++ b/modules/web-console/frontend/app/primitives/form-field/index.scss @@ -240,6 +240,18 @@ box-shadow: none; } } + + // Added right offset to appearance of input for invalid password + & > input[type='password'].ng-invalid.ng-touched { + padding-right: 36px; + } + + // Added right offset to appearance of dropdown for invalid data + & > button.select-toggle.ng-invalid.ng-touched { + &:after { + right: 36px; + } + } } &__errors { From db3099e079022a8acc081b284f96583280742567 Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Fri, 13 Apr 2018 18:08:35 +0700 Subject: [PATCH 0048/1463] IGNITE-8248 Fixed npe in Web Console agent int case of self-signed certificate. (cherry picked from commit 8a42f64) --- .../apache/ignite/console/agent/AgentLauncher.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java index 4db26bacc05b6..385ce08a12289 100644 --- a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java +++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java @@ -32,6 +32,7 @@ import java.net.URISyntaxException; import java.net.URL; import java.net.UnknownHostException; +import java.security.cert.X509Certificate; import java.util.Arrays; import java.util.Collection; import java.util.List; @@ -122,18 +123,16 @@ private static TrustManager[] getTrustManagers() { return new TrustManager[] { new X509TrustManager() { /** {@inheritDoc} */ - @Override public java.security.cert.X509Certificate[] getAcceptedIssuers() { - return null; + @Override public X509Certificate[] getAcceptedIssuers() { + return new X509Certificate[0]; } /** {@inheritDoc} */ - @Override public void checkClientTrusted( - java.security.cert.X509Certificate[] certs, String authType) { + @Override public void checkClientTrusted(X509Certificate[] certs, String authType) { } /** {@inheritDoc} */ - @Override public void checkServerTrusted( - java.security.cert.X509Certificate[] certs, String authType) { + @Override public void checkServerTrusted(X509Certificate[] certs, String authType) { } }}; } From e904b7b12a9bf4e038d13e39e7d85eaccfb40d65 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Fri, 13 Apr 2018 18:01:00 +0300 Subject: [PATCH 0049/1463] IGNITE-8256 Fixed simulated node failure in the test --- .../cache/distributed/dht/TxRecoveryStoreEnabledTest.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java index 060af21f8d363..30ac83d5f50b9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java @@ -32,6 +32,7 @@ import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.failure.StopNodeFailureHandler; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; @@ -81,6 +82,8 @@ public class TxRecoveryStoreEnabledTest extends GridCommonAbstractTest { cfg.setCacheConfiguration(ccfg); + cfg.setFailureHandler(new StopNodeFailureHandler()); + return cfg; } From 9be980d084b19a604d7f811bf2c73a28b35b0cd6 Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Fri, 13 Apr 2018 18:02:37 +0300 Subject: [PATCH 0050/1463] IGNITE-8233: KNN and SVM algorithms don't work when partition doesn't contain data. this closes #3807 (cherry picked from commit ee9ca06) --- .../dataset/impl/cache/CacheBasedDataset.java | 14 +- .../dataset/impl/cache/util/ComputeUtils.java | 9 +- .../ml/dataset/impl/local/LocalDataset.java | 16 ++- .../impl/local/LocalDatasetBuilder.java | 8 +- .../KNNClassificationModel.java | 32 +++-- .../impl/cache/CacheBasedDatasetTest.java | 9 +- .../ignite/ml/knn/KNNClassificationTest.java | 120 +++++++++-------- .../ignite/ml/knn/KNNRegressionTest.java | 122 ++++++++++-------- .../ignite/ml/knn/LabeledDatasetHelper.java | 45 +------ .../ignite/ml/knn/LabeledDatasetTest.java | 54 ++++---- 10 files changed, 226 insertions(+), 203 deletions(-) diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java index 463d4964d09b1..7428faf28fc6c 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java @@ -101,12 +101,16 @@ public CacheBasedDataset(Ignite ignite, IgniteCache upstreamCache, partDataBuilder ); - R res = map.apply(ctx, data, part); + if (data != null) { + R res = map.apply(ctx, data, part); - // Saves partition context after update. - ComputeUtils.saveContext(Ignition.localIgnite(), datasetCacheName, part, ctx); + // Saves partition context after update. + ComputeUtils.saveContext(Ignition.localIgnite(), datasetCacheName, part, ctx); - return res; + return res; + } + + return null; }, reduce, identity); } @@ -125,7 +129,7 @@ public CacheBasedDataset(Ignite ignite, IgniteCache upstreamCache, partDataBuilder ); - return map.apply(data, part); + return data != null ? map.apply(data, part) : null; }, reduce, identity); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java index 0785db28711bb..ce2fcfdcf8440 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java @@ -163,9 +163,14 @@ public static D getData( qry.setPartition(part); long cnt = upstreamCache.localSizeLong(part); - try (QueryCursor> cursor = upstreamCache.query(qry)) { - return partDataBuilder.build(new UpstreamCursorAdapter<>(cursor.iterator(), cnt), cnt, ctx); + + if (cnt > 0) { + try (QueryCursor> cursor = upstreamCache.query(qry)) { + return partDataBuilder.build(new UpstreamCursorAdapter<>(cursor.iterator(), cnt), cnt, ctx); + } } + + return null; }); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java index c08b7dedaba17..e312b202a1259 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java @@ -55,8 +55,12 @@ public class LocalDataset imple R identity) { R res = identity; - for (int part = 0; part < ctx.size(); part++) - res = reduce.apply(res, map.apply(ctx.get(part), data.get(part), part)); + for (int part = 0; part < ctx.size(); part++) { + D partData = data.get(part); + + if (partData != null) + res = reduce.apply(res, map.apply(ctx.get(part), partData, part)); + } return res; } @@ -65,8 +69,12 @@ public class LocalDataset imple @Override public R compute(IgniteBiFunction map, IgniteBinaryOperator reduce, R identity) { R res = identity; - for (int part = 0; part < data.size(); part++) - res = reduce.apply(res, map.apply(data.get(part), part)); + for (int part = 0; part < data.size(); part++) { + D partData = data.get(part); + + if (partData != null) + res = reduce.apply(res, map.apply(partData, part)); + } return res; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java index 0dc1ed6b5ecbd..cfc1801f26282 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java @@ -69,16 +69,16 @@ public LocalDatasetBuilder(Map upstreamMap, int partitions) { for (int part = 0; part < partitions; part++) { int cnt = part == partitions - 1 ? upstreamMap.size() - ptr : Math.min(partSize, upstreamMap.size() - ptr); - C ctx = partCtxBuilder.build( + C ctx = cnt > 0 ? partCtxBuilder.build( new IteratorWindow<>(firstKeysIter, k -> new UpstreamEntry<>(k, upstreamMap.get(k)), cnt), cnt - ); + ) : null; - D data = partDataBuilder.build( + D data = cnt > 0 ? partDataBuilder.build( new IteratorWindow<>(secondKeysIter, k -> new UpstreamEntry<>(k, upstreamMap.get(k)), cnt), cnt, ctx - ); + ) : null; ctxList.add(ctx); dataList.add(data); 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 693b81df0ecff..0f0cc9fed544c 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 @@ -151,19 +151,29 @@ private LabeledDataset buildLabeledDatasetOnListOfVectors */ @NotNull private LabeledVector[] getKClosestVectors(LabeledDataset trainingData, TreeMap> distanceIdxPairs) { - LabeledVector[] res = new LabeledVector[k]; - int i = 0; - final Iterator iter = distanceIdxPairs.keySet().iterator(); - while (i < k) { - double key = iter.next(); - Set idxs = distanceIdxPairs.get(key); - for (Integer idx : idxs) { - res[i] = trainingData.getRow(idx); - i++; - if (i >= k) - break; // go to next while-loop iteration + LabeledVector[] res; + + if (trainingData.rowSize() <= k) { + res = new LabeledVector[trainingData.rowSize()]; + for (int i = 0; i < trainingData.rowSize(); i++) + res[i] = trainingData.getRow(i); + } + else { + res = new LabeledVector[k]; + int i = 0; + final Iterator iter = distanceIdxPairs.keySet().iterator(); + while (i < k) { + double key = iter.next(); + Set idxs = distanceIdxPairs.get(key); + for (Integer idx : idxs) { + res[i] = trainingData.getRow(idx); + i++; + if (i >= k) + break; // go to next while-loop iteration + } } } + return res; } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java index dc0e160e3ca19..16ba04403310d 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java @@ -38,6 +38,7 @@ import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.ml.dataset.primitive.data.SimpleDatasetData; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; /** @@ -81,9 +82,9 @@ public void testPartitionExchangeDuringComputeCall() { CacheBasedDatasetBuilder builder = new CacheBasedDatasetBuilder<>(ignite, upstreamCache); - CacheBasedDataset dataset = builder.build( + CacheBasedDataset dataset = builder.build( (upstream, upstreamSize) -> upstreamSize, - (upstream, upstreamSize, ctx) -> null + (upstream, upstreamSize, ctx) -> new SimpleDatasetData(new double[0], 0) ); assertTrue("Before computation all partitions should not be reserved", @@ -133,9 +134,9 @@ public void testPartitionExchangeDuringComputeWithCtxCall() { CacheBasedDatasetBuilder builder = new CacheBasedDatasetBuilder<>(ignite, upstreamCache); - CacheBasedDataset dataset = builder.build( + CacheBasedDataset dataset = builder.build( (upstream, upstreamSize) -> upstreamSize, - (upstream, upstreamSize, ctx) -> null + (upstream, upstreamSize, ctx) -> new SimpleDatasetData(new double[0], 0) ); assertTrue("Before computation all partitions should not be reserved", 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 0877fc07cd16c..004718e238ba1 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 @@ -17,11 +17,11 @@ package org.apache.ignite.ml.knn; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; -import org.junit.Assert; -import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; import org.apache.ignite.ml.knn.classification.KNNClassificationModel; import org.apache.ignite.ml.knn.classification.KNNClassificationTrainer; import org.apache.ignite.ml.knn.classification.KNNStrategy; @@ -29,121 +29,137 @@ import org.apache.ignite.ml.math.distances.EuclideanDistance; import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static junit.framework.TestCase.assertEquals; /** Tests behaviour of KNNClassificationTest. */ +@RunWith(Parameterized.class) public class KNNClassificationTest { - /** Precision in test checks. */ - private static final double PRECISION = 1e-2; + /** Number of parts to be tested. */ + private static final int[] partsToBeTested = new int[] {1, 2, 3, 4, 5, 7, 100}; + + /** Number of partitions. */ + @Parameterized.Parameter + public int parts; + + /** Parameters. */ + @Parameterized.Parameters(name = "Data divided on {0} partitions, training with batch size {1}") + public static Iterable data() { + List res = new ArrayList<>(); + + for (int part : partsToBeTested) + res.add(new Integer[] {part}); + + return res; + } /** */ @Test - public void binaryClassificationTest() { - + public void testBinaryClassificationTest() { 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}); + 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}); KNNClassificationTrainer trainer = new KNNClassificationTrainer(); KNNClassificationModel knnMdl = trainer.fit( data, - 2, + parts, (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1), (k, v) -> v[2] ).withK(3) .withDistanceMeasure(new EuclideanDistance()) .withStrategy(KNNStrategy.SIMPLE); - Vector firstVector = new DenseLocalOnHeapVector(new double[]{2.0, 2.0}); - Assert.assertEquals(knnMdl.apply(firstVector), 1.0, PRECISION); - Vector secondVector = new DenseLocalOnHeapVector(new double[]{-2.0, -2.0}); - Assert.assertEquals(knnMdl.apply(secondVector), 2.0, PRECISION); + Vector firstVector = new DenseLocalOnHeapVector(new double[] {2.0, 2.0}); + assertEquals(knnMdl.apply(firstVector), 1.0); + Vector secondVector = new DenseLocalOnHeapVector(new double[] {-2.0, -2.0}); + assertEquals(knnMdl.apply(secondVector), 2.0); } /** */ @Test - public void binaryClassificationWithSmallestKTest() { + public void testBinaryClassificationWithSmallestKTest() { 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}); + 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}); KNNClassificationTrainer trainer = new KNNClassificationTrainer(); KNNClassificationModel knnMdl = trainer.fit( data, - 2, + parts, (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1), (k, v) -> v[2] ).withK(1) .withDistanceMeasure(new EuclideanDistance()) .withStrategy(KNNStrategy.SIMPLE); - Vector firstVector = new DenseLocalOnHeapVector(new double[]{2.0, 2.0}); - Assert.assertEquals(knnMdl.apply(firstVector), 1.0, PRECISION); - Vector secondVector = new DenseLocalOnHeapVector(new double[]{-2.0, -2.0}); - Assert.assertEquals(knnMdl.apply(secondVector), 2.0, PRECISION); + Vector firstVector = new DenseLocalOnHeapVector(new double[] {2.0, 2.0}); + assertEquals(knnMdl.apply(firstVector), 1.0); + Vector secondVector = new DenseLocalOnHeapVector(new double[] {-2.0, -2.0}); + assertEquals(knnMdl.apply(secondVector), 2.0); } /** */ @Test - public void binaryClassificationFarPointsWithSimpleStrategy() { + public void testBinaryClassificationFarPointsWithSimpleStrategy() { 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}); + 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 knnMdl = trainer.fit( data, - 2, + parts, (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1), (k, v) -> v[2] ).withK(3) .withDistanceMeasure(new EuclideanDistance()) .withStrategy(KNNStrategy.SIMPLE); - Vector vector = new DenseLocalOnHeapVector(new double[]{-1.01, -1.01}); - Assert.assertEquals(knnMdl.apply(vector), 2.0, PRECISION); + Vector vector = new DenseLocalOnHeapVector(new double[] {-1.01, -1.01}); + assertEquals(knnMdl.apply(vector), 2.0); } /** */ @Test - public void binaryClassificationFarPointsWithWeightedStrategy() { + public void testBinaryClassificationFarPointsWithWeightedStrategy() { 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}); + 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 knnMdl = trainer.fit( data, - 2, + parts, (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1), (k, v) -> v[2] ).withK(3) .withDistanceMeasure(new EuclideanDistance()) .withStrategy(KNNStrategy.WEIGHTED); - Vector vector = new DenseLocalOnHeapVector(new double[]{-1.01, -1.01}); - Assert.assertEquals(knnMdl.apply(vector), 1.0, PRECISION); + Vector vector = new DenseLocalOnHeapVector(new double[] {-1.01, -1.01}); + assertEquals(knnMdl.apply(vector), 1.0); } } 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 ce9cae59cda80..0c26ba9d4a771 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 @@ -17,6 +17,11 @@ package org.apache.ignite.ml.knn; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; import org.apache.ignite.ml.knn.classification.KNNStrategy; import org.apache.ignite.ml.knn.regression.KNNRegressionModel; @@ -25,110 +30,125 @@ import org.apache.ignite.ml.math.distances.EuclideanDistance; import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; import org.junit.Assert; - -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; /** * Tests for {@link KNNRegressionTrainer}. */ +@RunWith(Parameterized.class) public class KNNRegressionTest { + /** Number of parts to be tested. */ + private static final int[] partsToBeTested = new int[] {1, 2, 3, 4, 5, 7, 100}; + + /** Number of partitions. */ + @Parameterized.Parameter + public int parts; + + /** Parameters. */ + @Parameterized.Parameters(name = "Data divided on {0} partitions, training with batch size {1}") + public static Iterable data() { + List res = new ArrayList<>(); + + for (int part : partsToBeTested) + res.add(new Integer[] {part}); + + return res; + } + /** */ @Test - public void simpleRegressionWithOneNeighbour() { + public void testSimpleRegressionWithOneNeighbour() { 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}); + 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 knnMdl = (KNNRegressionModel) trainer.fit( - new LocalDatasetBuilder<>(data, 2), + new LocalDatasetBuilder<>(data, parts), (k, v) -> Arrays.copyOfRange(v, 1, v.length), (k, v) -> v[0] ).withK(1) .withDistanceMeasure(new EuclideanDistance()) .withStrategy(KNNStrategy.SIMPLE); - Vector vector = new DenseLocalOnHeapVector(new double[]{0, 0, 0, 5.0, 0.0}); + Vector vector = new DenseLocalOnHeapVector(new double[] {0, 0, 0, 5.0, 0.0}); System.out.println(knnMdl.apply(vector)); Assert.assertEquals(15, knnMdl.apply(vector), 1E-12); } /** */ @Test - public void longly() { + public void testLongly() { Map data = new HashMap<>(); - - data.put(0, new double[]{60323, 83.0, 234289, 2356, 1590, 107608, 1947}); - data.put(1, new double[]{61122, 88.5, 259426, 2325, 1456, 108632, 1948}); - data.put(2, new double[]{60171, 88.2, 258054, 3682, 1616, 109773, 1949}); - data.put(3, new double[]{61187, 89.5, 284599, 3351, 1650, 110929, 1950}); - data.put(4, new double[]{63221, 96.2, 328975, 2099, 3099, 112075, 1951}); - data.put(5, new double[]{63639, 98.1, 346999, 1932, 3594, 113270, 1952}); - data.put(6, new double[]{64989, 99.0, 365385, 1870, 3547, 115094, 1953}); - data.put(7, new double[]{63761, 100.0, 363112, 3578, 3350, 116219, 1954}); - data.put(8, new double[]{66019, 101.2, 397469, 2904, 3048, 117388, 1955}); - data.put(9, new double[]{68169, 108.4, 442769, 2936, 2798, 120445, 1957}); - data.put(10, new double[]{66513, 110.8, 444546, 4681, 2637, 121950, 1958}); - data.put(11, new double[]{68655, 112.6, 482704, 3813, 2552, 123366, 1959}); - data.put(12, new double[]{69564, 114.2, 502601, 3931, 2514, 125368, 1960}); - data.put(13, new double[]{69331, 115.7, 518173, 4806, 2572, 127852, 1961}); - data.put(14, new double[]{70551, 116.9, 554894, 4007, 2827, 130081, 1962}); + data.put(0, new double[] {60323, 83.0, 234289, 2356, 1590, 107608, 1947}); + data.put(1, new double[] {61122, 88.5, 259426, 2325, 1456, 108632, 1948}); + data.put(2, new double[] {60171, 88.2, 258054, 3682, 1616, 109773, 1949}); + data.put(3, new double[] {61187, 89.5, 284599, 3351, 1650, 110929, 1950}); + data.put(4, new double[] {63221, 96.2, 328975, 2099, 3099, 112075, 1951}); + data.put(5, new double[] {63639, 98.1, 346999, 1932, 3594, 113270, 1952}); + data.put(6, new double[] {64989, 99.0, 365385, 1870, 3547, 115094, 1953}); + data.put(7, new double[] {63761, 100.0, 363112, 3578, 3350, 116219, 1954}); + data.put(8, new double[] {66019, 101.2, 397469, 2904, 3048, 117388, 1955}); + data.put(9, new double[] {68169, 108.4, 442769, 2936, 2798, 120445, 1957}); + data.put(10, new double[] {66513, 110.8, 444546, 4681, 2637, 121950, 1958}); + data.put(11, new double[] {68655, 112.6, 482704, 3813, 2552, 123366, 1959}); + data.put(12, new double[] {69564, 114.2, 502601, 3931, 2514, 125368, 1960}); + data.put(13, new double[] {69331, 115.7, 518173, 4806, 2572, 127852, 1961}); + data.put(14, new double[] {70551, 116.9, 554894, 4007, 2827, 130081, 1962}); KNNRegressionTrainer trainer = new KNNRegressionTrainer(); KNNRegressionModel knnMdl = (KNNRegressionModel) trainer.fit( - new LocalDatasetBuilder<>(data, 2), + new LocalDatasetBuilder<>(data, parts), (k, v) -> Arrays.copyOfRange(v, 1, v.length), (k, v) -> v[0] ).withK(3) .withDistanceMeasure(new EuclideanDistance()) .withStrategy(KNNStrategy.SIMPLE); - Vector vector = new DenseLocalOnHeapVector(new double[]{104.6, 419180, 2822, 2857, 118734, 1956}); + Vector vector = new DenseLocalOnHeapVector(new double[] {104.6, 419180, 2822, 2857, 118734, 1956}); System.out.println(knnMdl.apply(vector)); Assert.assertEquals(67857, knnMdl.apply(vector), 2000); } /** */ + @Test public void testLonglyWithWeightedStrategy() { Map data = new HashMap<>(); - - data.put(0, new double[]{60323, 83.0, 234289, 2356, 1590, 107608, 1947}); - data.put(1, new double[]{61122, 88.5, 259426, 2325, 1456, 108632, 1948}); - data.put(2, new double[]{60171, 88.2, 258054, 3682, 1616, 109773, 1949}); - data.put(3, new double[]{61187, 89.5, 284599, 3351, 1650, 110929, 1950}); - data.put(4, new double[]{63221, 96.2, 328975, 2099, 3099, 112075, 1951}); - data.put(5, new double[]{63639, 98.1, 346999, 1932, 3594, 113270, 1952}); - data.put(6, new double[]{64989, 99.0, 365385, 1870, 3547, 115094, 1953}); - data.put(7, new double[]{63761, 100.0, 363112, 3578, 3350, 116219, 1954}); - data.put(8, new double[]{66019, 101.2, 397469, 2904, 3048, 117388, 1955}); - data.put(9, new double[]{68169, 108.4, 442769, 2936, 2798, 120445, 1957}); - data.put(10, new double[]{66513, 110.8, 444546, 4681, 2637, 121950, 1958}); - data.put(11, new double[]{68655, 112.6, 482704, 3813, 2552, 123366, 1959}); - data.put(12, new double[]{69564, 114.2, 502601, 3931, 2514, 125368, 1960}); - data.put(13, new double[]{69331, 115.7, 518173, 4806, 2572, 127852, 1961}); - data.put(14, new double[]{70551, 116.9, 554894, 4007, 2827, 130081, 1962}); + data.put(0, new double[] {60323, 83.0, 234289, 2356, 1590, 107608, 1947}); + data.put(1, new double[] {61122, 88.5, 259426, 2325, 1456, 108632, 1948}); + data.put(2, new double[] {60171, 88.2, 258054, 3682, 1616, 109773, 1949}); + data.put(3, new double[] {61187, 89.5, 284599, 3351, 1650, 110929, 1950}); + data.put(4, new double[] {63221, 96.2, 328975, 2099, 3099, 112075, 1951}); + data.put(5, new double[] {63639, 98.1, 346999, 1932, 3594, 113270, 1952}); + data.put(6, new double[] {64989, 99.0, 365385, 1870, 3547, 115094, 1953}); + data.put(7, new double[] {63761, 100.0, 363112, 3578, 3350, 116219, 1954}); + data.put(8, new double[] {66019, 101.2, 397469, 2904, 3048, 117388, 1955}); + data.put(9, new double[] {68169, 108.4, 442769, 2936, 2798, 120445, 1957}); + data.put(10, new double[] {66513, 110.8, 444546, 4681, 2637, 121950, 1958}); + data.put(11, new double[] {68655, 112.6, 482704, 3813, 2552, 123366, 1959}); + data.put(12, new double[] {69564, 114.2, 502601, 3931, 2514, 125368, 1960}); + data.put(13, new double[] {69331, 115.7, 518173, 4806, 2572, 127852, 1961}); + data.put(14, new double[] {70551, 116.9, 554894, 4007, 2827, 130081, 1962}); KNNRegressionTrainer trainer = new KNNRegressionTrainer(); KNNRegressionModel knnMdl = (KNNRegressionModel) trainer.fit( - new LocalDatasetBuilder<>(data, 2), + new LocalDatasetBuilder<>(data, parts), (k, v) -> Arrays.copyOfRange(v, 1, v.length), (k, v) -> v[0] ).withK(3) .withDistanceMeasure(new EuclideanDistance()) .withStrategy(KNNStrategy.SIMPLE); - Vector vector = new DenseLocalOnHeapVector(new double[]{104.6, 419180, 2822, 2857, 118734, 1956}); + Vector vector = new DenseLocalOnHeapVector(new double[] {104.6, 419180, 2822, 2857, 118734, 1956}); System.out.println(knnMdl.apply(vector)); Assert.assertEquals(67857, knnMdl.apply(vector), 2000); } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetHelper.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetHelper.java index a25b303480780..dbcdb99e62b34 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetHelper.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetHelper.java @@ -21,64 +21,33 @@ import java.net.URISyntaxException; import java.nio.file.Path; import java.nio.file.Paths; -import org.apache.ignite.Ignite; import org.apache.ignite.ml.structures.LabeledDataset; import org.apache.ignite.ml.structures.preprocessing.LabeledDatasetLoader; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; /** * Base class for decision trees test. */ -public class LabeledDatasetHelper extends GridCommonAbstractTest { - /** Count of nodes. */ - private static final int NODE_COUNT = 4; - +public class LabeledDatasetHelper { /** Separator. */ private static final String SEPARATOR = "\t"; - /** Grid instance. */ - protected Ignite ignite; - - /** - * Default constructor. - */ - public LabeledDatasetHelper() { - super(false); - } - - /** - * {@inheritDoc} - */ - @Override protected void beforeTest() throws Exception { - ignite = grid(NODE_COUNT); - } - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - for (int i = 1; i <= NODE_COUNT; i++) - startGrid(i); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() throws Exception { - stopAllGrids(); - } - /** * Loads labeled dataset from file with .txt extension. * * @param rsrcPath path to dataset. * @return null if path is incorrect. */ - LabeledDataset loadDatasetFromTxt(String rsrcPath, boolean isFallOnBadData) { + public static LabeledDataset loadDatasetFromTxt(String rsrcPath, boolean isFallOnBadData) { try { - Path path = Paths.get(this.getClass().getClassLoader().getResource(rsrcPath).toURI()); + Path path = Paths.get(LabeledDatasetHelper.class.getClassLoader().getResource(rsrcPath).toURI()); try { return LabeledDatasetLoader.loadFromTxtFile(path, SEPARATOR, false, isFallOnBadData); - } catch (IOException e) { + } + catch (IOException e) { e.printStackTrace(); } - } catch (URISyntaxException e) { + } + catch (URISyntaxException e) { e.printStackTrace(); return null; } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetTest.java index 77d40a61a0d4f..e986740a378c9 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetTest.java @@ -21,7 +21,6 @@ import java.net.URISyntaxException; import java.nio.file.Path; import java.nio.file.Paths; -import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.ml.math.ExternalizableTest; import org.apache.ignite.ml.math.Vector; import org.apache.ignite.ml.math.exceptions.CardinalityException; @@ -32,9 +31,13 @@ import org.apache.ignite.ml.structures.LabeledDatasetTestTrainPair; import org.apache.ignite.ml.structures.LabeledVector; import org.apache.ignite.ml.structures.preprocessing.LabeledDatasetLoader; +import org.junit.Test; + +import static junit.framework.TestCase.assertEquals; +import static junit.framework.TestCase.fail; /** Tests behaviour of KNNClassificationTest. */ -public class LabeledDatasetTest extends LabeledDatasetHelper implements ExternalizableTest { +public class LabeledDatasetTest implements ExternalizableTest { /** */ private static final String KNN_IRIS_TXT = "datasets/knn/iris.txt"; @@ -51,9 +54,8 @@ public class LabeledDatasetTest extends LabeledDatasetHelper implements External private static final String IRIS_MISSED_DATA = "datasets/knn/missed_data.txt"; /** */ + @Test public void testFeatureNames() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - double[][] mtx = new double[][] { {1.0, 1.0}, @@ -71,9 +73,8 @@ public void testFeatureNames() { } /** */ + @Test public void testAccessMethods() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - double[][] mtx = new double[][] { {1.0, 1.0}, @@ -98,9 +99,8 @@ public void testAccessMethods() { } /** */ + @Test public void testFailOnYNull() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - double[][] mtx = new double[][] { {1.0, 1.0}, @@ -122,9 +122,8 @@ public void testFailOnYNull() { } /** */ + @Test public void testFailOnXNull() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - double[][] mtx = new double[][] {}; double[] lbs = new double[] {1.0, 1.0, 1.0, 2.0, 2.0, 2.0}; @@ -140,18 +139,17 @@ public void testFailOnXNull() { } /** */ + @Test public void testLoadingCorrectTxtFile() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - LabeledDataset training = loadDatasetFromTxt(KNN_IRIS_TXT, false); + LabeledDataset training = LabeledDatasetHelper.loadDatasetFromTxt(KNN_IRIS_TXT, false); assertEquals(training.rowSize(), 150); } /** */ + @Test public void testLoadingEmptyFile() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - try { - loadDatasetFromTxt(EMPTY_TXT, false); + LabeledDatasetHelper.loadDatasetFromTxt(EMPTY_TXT, false); fail("EmptyFileException"); } catch (EmptyFileException e) { @@ -161,11 +159,10 @@ public void testLoadingEmptyFile() { } /** */ + @Test public void testLoadingFileWithFirstEmptyRow() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - try { - loadDatasetFromTxt(NO_DATA_TXT, false); + LabeledDatasetHelper.loadDatasetFromTxt(NO_DATA_TXT, false); fail("NoDataException"); } catch (NoDataException e) { @@ -175,19 +172,17 @@ public void testLoadingFileWithFirstEmptyRow() { } /** */ + @Test public void testLoadingFileWithIncorrectData() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - - LabeledDataset training = loadDatasetFromTxt(IRIS_INCORRECT_TXT, false); + LabeledDataset training = LabeledDatasetHelper.loadDatasetFromTxt(IRIS_INCORRECT_TXT, false); assertEquals(149, training.rowSize()); } /** */ + @Test public void testFailOnLoadingFileWithIncorrectData() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - try { - loadDatasetFromTxt(IRIS_INCORRECT_TXT, true); + LabeledDatasetHelper.loadDatasetFromTxt(IRIS_INCORRECT_TXT, true); fail("FileParsingException"); } catch (FileParsingException e) { @@ -198,9 +193,8 @@ public void testFailOnLoadingFileWithIncorrectData() { } /** */ + @Test public void testLoadingFileWithMissedData() throws URISyntaxException, IOException { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - Path path = Paths.get(this.getClass().getClassLoader().getResource(IRIS_MISSED_DATA).toURI()); LabeledDataset training = LabeledDatasetLoader.loadFromTxtFile(path, ",", false, false); @@ -209,9 +203,8 @@ public void testLoadingFileWithMissedData() throws URISyntaxException, IOExcepti } /** */ + @Test public void testSplitting() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - double[][] mtx = new double[][] { {1.0, 1.0}, @@ -246,9 +239,8 @@ public void testSplitting() { } /** */ + @Test public void testLabels() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - double[][] mtx = new double[][] { {1.0, 1.0}, @@ -267,8 +259,6 @@ public void testLabels() { /** */ @Override public void testExternalization() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - double[][] mtx = new double[][] { {1.0, 1.0}, From a9ec54be9c51fe8ff04ca5d21160c27255640007 Mon Sep 17 00:00:00 2001 From: dmitrievanthony Date: Fri, 13 Apr 2018 18:08:08 +0300 Subject: [PATCH 0051/1463] IGNITE-8232: ML package cleanup for 2.5 release this closes #3806 (cherry picked from commit 47cfdc2) --- .../examples/ml/nn/MLPTrainerExample.java | 2 +- ...dLinearRegressionWithQRTrainerExample.java | 137 ------ ...> LinearRegressionLSQRTrainerExample.java} | 4 +- ...nLSQRTrainerWithNormalizationExample.java} | 2 +- ...=> LinearRegressionSGDTrainerExample.java} | 9 +- .../java/org/apache/ignite/ml/Trainer.java | 36 -- .../ignite/ml/estimators/Estimators.java | 50 --- .../ignite/ml/estimators/package-info.java | 22 - .../ml/math/functions/IgniteBiFunction.java | 8 +- .../LinSysPartitionDataBuilderOnHeap.java | 86 ---- .../isolve/LinSysPartitionDataOnHeap.java | 65 --- .../ml/math/isolve/lsqr/AbstractLSQR.java | 3 +- .../ml/math/isolve/lsqr/LSQROnHeap.java | 27 +- .../org/apache/ignite/ml/nn/MLPTrainer.java | 1 - .../group => nn}/UpdatesStrategy.java | 2 +- .../ml/optimization/GradientDescent.java | 202 --------- .../ml/optimization/GradientFunction.java | 31 -- .../LeastSquaresGradientFunction.java | 33 -- .../SparseDistributedMatrixMapReducer.java | 84 ---- .../ml/optimization/util/package-info.java | 22 - .../linear/LinearRegressionLSQRTrainer.java | 10 +- .../linear/LinearRegressionQRTrainer.java | 72 ---- .../linear/LinearRegressionSGDTrainer.java | 7 +- .../apache/ignite/ml/trainers/Trainer.java | 33 -- .../trainers/group/BaseLocalProcessorJob.java | 146 ------- .../ignite/ml/trainers/group/ConstModel.java | 46 -- .../ml/trainers/group/GroupTrainer.java | 208 --------- .../group/GroupTrainerBaseProcessorTask.java | 144 ------- .../trainers/group/GroupTrainerCacheKey.java | 125 ------ .../GroupTrainerEntriesProcessorTask.java | 64 --- .../ml/trainers/group/GroupTrainerInput.java | 37 -- .../group/GroupTrainerKeysProcessorTask.java | 62 --- .../trainers/group/GroupTrainingContext.java | 98 ----- .../group/LocalEntriesProcessorJob.java | 85 ---- .../trainers/group/LocalKeysProcessorJob.java | 78 ---- .../ml/trainers/group/Metaoptimizer.java | 93 ---- .../group/MetaoptimizerDistributedStep.java | 97 ----- .../group/MetaoptimizerGroupTrainer.java | 132 ------ .../ml/trainers/group/ResultAndUpdates.java | 178 -------- .../ml/trainers/group/UpdateStrategies.java | 47 -- .../ml/trainers/group/chain/Chains.java | 56 --- .../group/chain/ComputationsChain.java | 246 ----------- .../chain/DistributedEntryProcessingStep.java | 34 -- .../chain/DistributedKeyProcessingStep.java | 33 -- .../trainers/group/chain/DistributedStep.java | 70 --- .../trainers/group/chain/EntryAndContext.java | 70 --- .../trainers/group/chain/HasTrainingUUID.java | 32 -- .../trainers/group/chain/KeyAndContext.java | 67 --- .../ml/trainers/group/chain/package-info.java | 22 - .../ml/trainers/group/package-info.java | 22 - .../apache/ignite/ml/IgniteMLTestSuite.java | 4 - .../ml/math/isolve/lsqr/LSQROnHeapTest.java | 14 +- .../ml/nn/MLPTrainerIntegrationTest.java | 1 - .../apache/ignite/ml/nn/MLPTrainerTest.java | 1 - .../MLPTrainerMnistIntegrationTest.java | 2 +- .../nn/performance/MLPTrainerMnistTest.java | 2 +- .../ml/optimization/GradientDescentTest.java | 64 --- .../optimization/OptimizationTestSuite.java | 33 -- ...SparseDistributedMatrixMapReducerTest.java | 135 ------ .../ml/regressions/RegressionsTestSuite.java | 3 - .../linear/ArtificialRegressionDatasets.java | 404 ------------------ ...tributedLinearRegressionQRTrainerTest.java | 36 -- ...tributedLinearRegressionQRTrainerTest.java | 36 -- .../GenericLinearRegressionTrainerTest.java | 206 --------- ...reAbstractLinearRegressionTrainerTest.java | 127 ------ .../LinearRegressionSGDTrainerTest.java | 2 +- .../LocalLinearRegressionQRTrainerTest.java | 36 -- .../group/DistributedWorkersChainTest.java | 189 -------- .../ml/trainers/group/GroupTrainerTest.java | 90 ---- .../group/SimpleGroupTrainerInput.java | 63 --- .../ml/trainers/group/TestGroupTrainer.java | 144 ------- .../group/TestGroupTrainerLocalContext.java | 85 ---- .../group/TestGroupTrainingCache.java | 70 --- .../group/TestGroupTrainingSecondCache.java | 56 --- .../ml/trainers/group/TestLocalContext.java | 51 --- .../trainers/group/TestTrainingLoopStep.java | 65 --- .../group/TrainersGroupTestSuite.java | 32 -- ...eOLSMultipleLinearRegressionBenchmark.java | 69 --- .../yardstick/ml/regression/package-info.java | 22 - 79 files changed, 54 insertions(+), 5228 deletions(-) delete mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java rename examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/{DistributedLinearRegressionWithLSQRTrainerExample.java => LinearRegressionLSQRTrainerExample.java} (97%) rename examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/{DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample.java => LinearRegressionLSQRTrainerWithNormalizationExample.java} (99%) rename examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/{DistributedLinearRegressionWithSGDTrainerExample.java => LinearRegressionSGDTrainerExample.java} (95%) delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/estimators/Estimators.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/estimators/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataBuilderOnHeap.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataOnHeap.java rename modules/ml/src/main/java/org/apache/ignite/ml/{trainers/group => nn}/UpdatesStrategy.java (98%) delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/optimization/GradientDescent.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/optimization/GradientFunction.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/optimization/LeastSquaresGradientFunction.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/optimization/util/SparseDistributedMatrixMapReducer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/optimization/util/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionQRTrainer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/Trainer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/BaseLocalProcessorJob.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/ConstModel.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerBaseProcessorTask.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerCacheKey.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerEntriesProcessorTask.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerInput.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerKeysProcessorTask.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainingContext.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/LocalEntriesProcessorJob.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/LocalKeysProcessorJob.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/Metaoptimizer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/MetaoptimizerDistributedStep.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/MetaoptimizerGroupTrainer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/ResultAndUpdates.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/UpdateStrategies.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/Chains.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/ComputationsChain.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedEntryProcessingStep.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedKeyProcessingStep.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedStep.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/EntryAndContext.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/HasTrainingUUID.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/KeyAndContext.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/package-info.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/package-info.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/optimization/GradientDescentTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/optimization/OptimizationTestSuite.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/optimization/util/SparseDistributedMatrixMapReducerTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/ArtificialRegressionDatasets.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/BlockDistributedLinearRegressionQRTrainerTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/DistributedLinearRegressionQRTrainerTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GenericLinearRegressionTrainerTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GridAwareAbstractLinearRegressionTrainerTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LocalLinearRegressionQRTrainerTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/DistributedWorkersChainTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/GroupTrainerTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/SimpleGroupTrainerInput.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainer.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainerLocalContext.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainingCache.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainingSecondCache.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestLocalContext.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestTrainingLoopStep.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TrainersGroupTestSuite.java delete mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/regression/IgniteOLSMultipleLinearRegressionBenchmark.java delete mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/regression/package-info.java 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 ce44cc64b935e..5d1ac38a4d3cb 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 @@ -32,7 +32,7 @@ import org.apache.ignite.ml.optimization.LossFunctions; import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDParameterUpdate; import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDUpdateCalculator; -import org.apache.ignite.ml.trainers.group.UpdatesStrategy; +import org.apache.ignite.ml.nn.UpdatesStrategy; import org.apache.ignite.thread.IgniteThread; /** diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java deleted file mode 100644 index 98d5e4e84ced1..0000000000000 --- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java +++ /dev/null @@ -1,137 +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.regression.linear; - -import org.apache.ignite.Ignite; -import org.apache.ignite.Ignition; -import org.apache.ignite.examples.ml.math.matrix.SparseDistributedMatrixExample; -import org.apache.ignite.ml.Trainer; -import org.apache.ignite.ml.math.Matrix; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.ml.math.impls.vector.SparseDistributedVector; -import org.apache.ignite.ml.regressions.linear.LinearRegressionModel; -import org.apache.ignite.ml.regressions.linear.LinearRegressionQRTrainer; -import org.apache.ignite.thread.IgniteThread; - -import java.util.Arrays; - -/** - * Run linear regression model over distributed matrix. - * - * @see LinearRegressionQRTrainer - */ -public class DistributedLinearRegressionWithQRTrainerExample { - /** */ - private static final double[][] data = { - {8, 78, 284, 9.100000381, 109}, - {9.300000191, 68, 433, 8.699999809, 144}, - {7.5, 70, 739, 7.199999809, 113}, - {8.899999619, 96, 1792, 8.899999619, 97}, - {10.19999981, 74, 477, 8.300000191, 206}, - {8.300000191, 111, 362, 10.89999962, 124}, - {8.800000191, 77, 671, 10, 152}, - {8.800000191, 168, 636, 9.100000381, 162}, - {10.69999981, 82, 329, 8.699999809, 150}, - {11.69999981, 89, 634, 7.599999905, 134}, - {8.5, 149, 631, 10.80000019, 292}, - {8.300000191, 60, 257, 9.5, 108}, - {8.199999809, 96, 284, 8.800000191, 111}, - {7.900000095, 83, 603, 9.5, 182}, - {10.30000019, 130, 686, 8.699999809, 129}, - {7.400000095, 145, 345, 11.19999981, 158}, - {9.600000381, 112, 1357, 9.699999809, 186}, - {9.300000191, 131, 544, 9.600000381, 177}, - {10.60000038, 80, 205, 9.100000381, 127}, - {9.699999809, 130, 1264, 9.199999809, 179}, - {11.60000038, 140, 688, 8.300000191, 80}, - {8.100000381, 154, 354, 8.399999619, 103}, - {9.800000191, 118, 1632, 9.399999619, 101}, - {7.400000095, 94, 348, 9.800000191, 117}, - {9.399999619, 119, 370, 10.39999962, 88}, - {11.19999981, 153, 648, 9.899999619, 78}, - {9.100000381, 116, 366, 9.199999809, 102}, - {10.5, 97, 540, 10.30000019, 95}, - {11.89999962, 176, 680, 8.899999619, 80}, - {8.399999619, 75, 345, 9.600000381, 92}, - {5, 134, 525, 10.30000019, 126}, - {9.800000191, 161, 870, 10.39999962, 108}, - {9.800000191, 111, 669, 9.699999809, 77}, - {10.80000019, 114, 452, 9.600000381, 60}, - {10.10000038, 142, 430, 10.69999981, 71}, - {10.89999962, 238, 822, 10.30000019, 86}, - {9.199999809, 78, 190, 10.69999981, 93}, - {8.300000191, 196, 867, 9.600000381, 106}, - {7.300000191, 125, 969, 10.5, 162}, - {9.399999619, 82, 499, 7.699999809, 95}, - {9.399999619, 125, 925, 10.19999981, 91}, - {9.800000191, 129, 353, 9.899999619, 52}, - {3.599999905, 84, 288, 8.399999619, 110}, - {8.399999619, 183, 718, 10.39999962, 69}, - {10.80000019, 119, 540, 9.199999809, 57}, - {10.10000038, 180, 668, 13, 106}, - {9, 82, 347, 8.800000191, 40}, - {10, 71, 345, 9.199999809, 50}, - {11.30000019, 118, 463, 7.800000191, 35}, - {11.30000019, 121, 728, 8.199999809, 86}, - {12.80000019, 68, 383, 7.400000095, 57}, - {10, 112, 316, 10.39999962, 57}, - {6.699999809, 109, 388, 8.899999619, 94} - }; - - /** Run example. */ - public static void main(String[] args) throws InterruptedException { - System.out.println(); - System.out.println(">>> Linear regression model over sparse distributed matrix API usage example started."); - // Start ignite grid. - try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { - System.out.println(">>> Ignite grid started."); - // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread - // because we create ignite cache internally. - IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), - SparseDistributedMatrixExample.class.getSimpleName(), () -> { - - // Create SparseDistributedMatrix, new cache will be created automagically. - System.out.println(">>> Create new SparseDistributedMatrix inside IgniteThread."); - SparseDistributedMatrix distributedMatrix = new SparseDistributedMatrix(data); - - System.out.println(">>> Create new linear regression trainer object."); - Trainer trainer = new LinearRegressionQRTrainer(); - - System.out.println(">>> Perform the training to get the model."); - LinearRegressionModel mdl = trainer.train(distributedMatrix); - System.out.println(">>> Linear regression model: " + mdl); - - System.out.println(">>> ---------------------------------"); - System.out.println(">>> | Prediction\t| Ground Truth\t|"); - System.out.println(">>> ---------------------------------"); - for (double[] observation : data) { - Vector inputs = new SparseDistributedVector(Arrays.copyOfRange(observation, 1, observation.length)); - double prediction = mdl.apply(inputs); - double groundTruth = observation[0]; - System.out.printf(">>> | %.4f\t\t| %.4f\t\t|\n", prediction, groundTruth); - } - System.out.println(">>> ---------------------------------"); - }); - - igniteThread.start(); - - igniteThread.join(); - } - } -} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerExample.java similarity index 97% rename from examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java rename to examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerExample.java index 25aec0cbba400..276d43fcd67ef 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerExample.java @@ -38,7 +38,7 @@ * * @see LinearRegressionLSQRTrainer */ -public class DistributedLinearRegressionWithLSQRTrainerExample { +public class LinearRegressionLSQRTrainerExample { /** */ private static final double[][] data = { {8, 78, 284, 9.100000381, 109}, @@ -107,7 +107,7 @@ public static void main(String[] args) throws InterruptedException { // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread // because we create ignite cache internally. IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), - DistributedLinearRegressionWithLSQRTrainerExample.class.getSimpleName(), () -> { + LinearRegressionLSQRTrainerExample.class.getSimpleName(), () -> { IgniteCache dataCache = getTestCache(ignite); System.out.println(">>> Create new linear regression trainer object."); diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerWithNormalizationExample.java similarity index 99% rename from examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample.java rename to examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerWithNormalizationExample.java index 99e657781b131..0358f44135e00 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerWithNormalizationExample.java @@ -44,7 +44,7 @@ * @see NormalizationTrainer * @see NormalizationPreprocessor */ -public class DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample { +public class LinearRegressionLSQRTrainerWithNormalizationExample { /** */ private static final double[][] data = { {8, 78, 284, 9.100000381, 109}, diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithSGDTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionSGDTrainerExample.java similarity index 95% rename from examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithSGDTrainerExample.java rename to examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionSGDTrainerExample.java index 44366e1aafc33..ce6ad3b4df287 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithSGDTrainerExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionSGDTrainerExample.java @@ -28,9 +28,8 @@ import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate; import org.apache.ignite.ml.optimization.updatecalculators.RPropUpdateCalculator; import org.apache.ignite.ml.regressions.linear.LinearRegressionModel; -import org.apache.ignite.ml.regressions.linear.LinearRegressionQRTrainer; import org.apache.ignite.ml.regressions.linear.LinearRegressionSGDTrainer; -import org.apache.ignite.ml.trainers.group.UpdatesStrategy; +import org.apache.ignite.ml.nn.UpdatesStrategy; import org.apache.ignite.thread.IgniteThread; import javax.cache.Cache; @@ -40,9 +39,9 @@ /** * Run linear regression model over distributed matrix. * - * @see LinearRegressionQRTrainer + * @see LinearRegressionSGDTrainer */ -public class DistributedLinearRegressionWithSGDTrainerExample { +public class LinearRegressionSGDTrainerExample { /** */ private static final double[][] data = { {8, 78, 284, 9.100000381, 109}, @@ -110,7 +109,7 @@ public static void main(String[] args) throws InterruptedException { // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread // because we create ignite cache internally. IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), - DistributedLinearRegressionWithSGDTrainerExample.class.getSimpleName(), () -> { + LinearRegressionSGDTrainerExample.class.getSimpleName(), () -> { IgniteCache dataCache = getTestCache(ignite); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java deleted file mode 100644 index f53b80110cb4c..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.ml; - -/** - * Interface for Trainers. Trainer is just a function which produces model from the data. - * - * @param Type of produced model. - * @param Type of data needed for model producing. - */ -// TODO: IGNITE-7659: Reduce multiple Trainer interfaces to one -@Deprecated -public interface Trainer { - /** - * Returns model based on data - * - * @param data data to build model - * @return model - */ - M train(T data); -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/estimators/Estimators.java b/modules/ml/src/main/java/org/apache/ignite/ml/estimators/Estimators.java deleted file mode 100644 index b2731ffd0879e..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/estimators/Estimators.java +++ /dev/null @@ -1,50 +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.estimators; - -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.Function; -import java.util.stream.Stream; -import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.ml.Model; -import org.apache.ignite.ml.math.functions.IgniteTriFunction; - -/** Estimators. */ -public class Estimators { - /** Simple implementation of mean squared error estimator. */ - public static IgniteTriFunction, Stream>, Function, Double> MSE() { - return (model, stream, f) -> stream.mapToDouble(dp -> { - double diff = f.apply(dp.get2()) - f.apply(model.apply(dp.get1())); - return diff * diff; - }).average().orElse(0); - } - - /** Simple implementation of errors percentage estimator. */ - public static IgniteTriFunction, Stream>, Function, Double> errorsPercentage() { - return (model, stream, f) -> { - AtomicLong total = new AtomicLong(0); - - long cnt = stream. - peek((ib) -> total.incrementAndGet()). - filter(dp -> !model.apply(dp.get1()).equals(dp.get2())). - count(); - - return (double)cnt / total.get(); - }; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/estimators/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/estimators/package-info.java deleted file mode 100644 index c03827f3e32e0..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/estimators/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. - */ - -/** - * - * Contains estimation algorithms. - */ -package org.apache.ignite.ml.estimators; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/functions/IgniteBiFunction.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/functions/IgniteBiFunction.java index dc49739426881..45fd035710e79 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/math/functions/IgniteBiFunction.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/functions/IgniteBiFunction.java @@ -18,6 +18,7 @@ package org.apache.ignite.ml.math.functions; import java.io.Serializable; +import java.util.Objects; import java.util.function.BiFunction; /** @@ -25,5 +26,10 @@ * * @see java.util.function.BiFunction */ -public interface IgniteBiFunction extends BiFunction, Serializable { +public interface IgniteBiFunction extends BiFunction, Serializable { + /** {@inheritDoc} */ + default IgniteBiFunction andThen(IgniteFunction after) { + Objects.requireNonNull(after); + return (T t, U u) -> after.apply(apply(t, u)); + } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataBuilderOnHeap.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataBuilderOnHeap.java deleted file mode 100644 index e80b935a165ca..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataBuilderOnHeap.java +++ /dev/null @@ -1,86 +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.math.isolve; - -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; - -/** - * Linear system partition data builder that builds {@link LinSysPartitionDataOnHeap}. - * - * @param Type of a key in upstream data. - * @param Type of a value in upstream data. - * @param Type of a partition context. - */ -public class LinSysPartitionDataBuilderOnHeap - implements PartitionDataBuilder { - /** */ - private static final long serialVersionUID = -7820760153954269227L; - - /** Extractor of X matrix row. */ - private final IgniteBiFunction xExtractor; - - /** Extractor of Y vector value. */ - private final IgniteBiFunction yExtractor; - - /** - * Constructs a new instance of linear system partition data builder. - * - * @param xExtractor Extractor of X matrix row. - * @param yExtractor Extractor of Y vector value. - */ - public LinSysPartitionDataBuilderOnHeap(IgniteBiFunction xExtractor, - IgniteBiFunction yExtractor) { - this.xExtractor = xExtractor; - this.yExtractor = yExtractor; - } - - /** {@inheritDoc} */ - @Override public LinSysPartitionDataOnHeap build(Iterator> upstreamData, long upstreamDataSize, - C ctx) { - // Prepares the matrix of features in flat column-major format. - int xCols = -1; - double[] x = null;//new double[Math.toIntExact(upstreamDataSize * cols)]; - double[] y = new double[Math.toIntExact(upstreamDataSize)]; - - int ptr = 0; - while (upstreamData.hasNext()) { - UpstreamEntry entry = upstreamData.next(); - double[] row = xExtractor.apply(entry.getKey(), entry.getValue()); - - if (xCols < 0) { - xCols = row.length; - x = new double[Math.toIntExact(upstreamDataSize * xCols)]; - } - else - assert row.length == xCols : "X extractor must return exactly " + xCols + " columns"; - - for (int i = 0; i < xCols; i++) - x[Math.toIntExact(i * upstreamDataSize) + ptr] = row[i]; - - y[ptr] = yExtractor.apply(entry.getKey(), entry.getValue()); - - ptr++; - } - - return new LinSysPartitionDataOnHeap(x, y, Math.toIntExact(upstreamDataSize)); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataOnHeap.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataOnHeap.java deleted file mode 100644 index 89c8e441f92fe..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataOnHeap.java +++ /dev/null @@ -1,65 +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.math.isolve; - -/** - * On Heap partition data that keeps part of a linear system. - */ -public class LinSysPartitionDataOnHeap implements AutoCloseable { - /** Part of X matrix. */ - private final double[] x; - - /** Part of Y vector. */ - private final double[] y; - - /** Number of rows. */ - private final int rows; - - /** - * Constructs a new instance of linear system partition data. - * - * @param x Part of X matrix. - * @param y Part of Y vector. - * @param rows Number of rows. - */ - public LinSysPartitionDataOnHeap(double[] x, double[] y, int rows) { - this.x = x; - this.rows = rows; - this.y = y; - } - - /** */ - public double[] getX() { - return x; - } - - /** */ - public int getRows() { - return rows; - } - - /** */ - public double[] getY() { - return y; - } - - /** {@inheritDoc} */ - @Override public void close() { - // Do nothing, GC will clean up. - } -} 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 8d190cd74373a..d1d3219f2b1f5 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 @@ -19,6 +19,7 @@ import com.github.fommil.netlib.BLAS; import java.util.Arrays; +import org.apache.ignite.ml.math.Precision; /** * Basic implementation of the LSQR algorithm without assumptions about dataset storage format or data processing @@ -30,7 +31,7 @@ // TODO: IGNITE-7660: Refactor LSQR algorithm public abstract class AbstractLSQR { /** The smallest representable positive number such that 1.0 + eps != 1.0. */ - private static final double eps = Double.longBitsToDouble(Double.doubleToLongBits(1.0) | 1) - 1.0; + private static final double eps = Precision.EPSILON; /** BLAS (Basic Linear Algebra Subprograms) instance. */ private static BLAS blas = BLAS.getInstance(); 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 b1cc4c957e1b1..e138cf3ff3db9 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 @@ -22,14 +22,14 @@ import org.apache.ignite.ml.dataset.Dataset; import org.apache.ignite.ml.dataset.DatasetBuilder; import org.apache.ignite.ml.dataset.PartitionDataBuilder; -import org.apache.ignite.ml.math.isolve.LinSysPartitionDataOnHeap; +import org.apache.ignite.ml.dataset.primitive.data.SimpleLabeledDatasetData; /** * Distributed implementation of LSQR algorithm based on {@link AbstractLSQR} and {@link Dataset}. */ public class LSQROnHeap extends AbstractLSQR implements AutoCloseable { /** Dataset. */ - private final Dataset dataset; + private final Dataset dataset; /** * Constructs a new instance of OnHeap LSQR algorithm implementation. @@ -38,7 +38,7 @@ public class LSQROnHeap extends AbstractLSQR implements AutoCloseable { * @param partDataBuilder Partition data builder. */ public LSQROnHeap(DatasetBuilder datasetBuilder, - PartitionDataBuilder partDataBuilder) { + PartitionDataBuilder partDataBuilder) { this.dataset = datasetBuilder.build( (upstream, upstreamSize) -> new LSQRPartitionContext(), partDataBuilder @@ -48,20 +48,20 @@ public LSQROnHeap(DatasetBuilder datasetBuilder, /** {@inheritDoc} */ @Override protected double bnorm() { return dataset.computeWithCtx((ctx, data) -> { - ctx.setU(Arrays.copyOf(data.getY(), data.getY().length)); + ctx.setU(Arrays.copyOf(data.getLabels(), data.getLabels().length)); - return BLAS.getInstance().dnrm2(data.getY().length, data.getY(), 1); + return BLAS.getInstance().dnrm2(data.getLabels().length, data.getLabels(), 1); }, (a, b) -> a == null ? b : b == null ? a : Math.sqrt(a * a + b * b)); } /** {@inheritDoc} */ @Override protected double beta(double[] x, double alfa, double beta) { return dataset.computeWithCtx((ctx, data) -> { - if (data.getX() == null) + if (data.getFeatures() == null) return null; - int cols = data.getX().length / data.getRows(); - BLAS.getInstance().dgemv("N", data.getRows(), cols, alfa, data.getX(), + int cols = data.getFeatures().length / data.getRows(); + BLAS.getInstance().dgemv("N", data.getRows(), cols, alfa, data.getFeatures(), Math.max(1, data.getRows()), x, 1, beta, ctx.getU(), 1); return BLAS.getInstance().dnrm2(ctx.getU().length, ctx.getU(), 1); @@ -71,13 +71,13 @@ public LSQROnHeap(DatasetBuilder datasetBuilder, /** {@inheritDoc} */ @Override protected double[] iter(double bnorm, double[] target) { double[] res = dataset.computeWithCtx((ctx, data) -> { - if (data.getX() == null) + if (data.getFeatures() == null) return null; - int cols = data.getX().length / data.getRows(); + int cols = data.getFeatures().length / data.getRows(); BLAS.getInstance().dscal(ctx.getU().length, 1 / bnorm, ctx.getU(), 1); double[] v = new double[cols]; - BLAS.getInstance().dgemv("T", data.getRows(), cols, 1.0, data.getX(), + BLAS.getInstance().dgemv("T", data.getRows(), cols, 1.0, data.getFeatures(), Math.max(1, data.getRows()), ctx.getU(), 1, 0, v, 1); return v; @@ -101,7 +101,10 @@ else if (b == null) * @return number of columns */ @Override protected int getColumns() { - return dataset.compute(data -> data.getX() == null ? null : data.getX().length / data.getRows(), (a, b) -> a == null ? b : a); + return dataset.compute( + data -> data.getFeatures() == null ? null : data.getFeatures().length / data.getRows(), + (a, b) -> a == null ? b : a + ); } /** {@inheritDoc} */ 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 fe955cbe696de..d12a276b2bc0e 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 @@ -33,7 +33,6 @@ import org.apache.ignite.ml.nn.initializers.RandomInitializer; import org.apache.ignite.ml.optimization.updatecalculators.ParameterUpdateCalculator; import org.apache.ignite.ml.trainers.MultiLabelDatasetTrainer; -import org.apache.ignite.ml.trainers.group.UpdatesStrategy; import org.apache.ignite.ml.util.Utils; import java.io.Serializable; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/UpdatesStrategy.java b/modules/ml/src/main/java/org/apache/ignite/ml/nn/UpdatesStrategy.java similarity index 98% rename from modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/UpdatesStrategy.java rename to modules/ml/src/main/java/org/apache/ignite/ml/nn/UpdatesStrategy.java index 5288dbfbf6c1c..e48d94665d3da 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/UpdatesStrategy.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/nn/UpdatesStrategy.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.ml.trainers.group; +package org.apache.ignite.ml.nn; import java.io.Serializable; import java.util.List; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/GradientDescent.java b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/GradientDescent.java deleted file mode 100644 index 15ed914e4ee16..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/GradientDescent.java +++ /dev/null @@ -1,202 +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.optimization; - -import org.apache.ignite.ml.math.Matrix; -import org.apache.ignite.ml.math.StorageConstants; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; -import org.apache.ignite.ml.math.impls.vector.FunctionVector; -import org.apache.ignite.ml.optimization.util.SparseDistributedMatrixMapReducer; - -/** - * Gradient descent optimizer. - */ -public class GradientDescent { - /** - * Function which computes gradient of the loss function at any given point. - */ - private final GradientFunction lossGradient; - - /** - * Weights updater applied on every gradient descent step to decide how weights should be changed. - */ - private final Updater updater; - - /** - * Max number of gradient descent iterations. - */ - private int maxIterations = 1000; - - /** - * Convergence tolerance is condition which decides iteration termination. - */ - private double convergenceTol = 1e-8; - - /** - * New gradient descent instance based of loss function and updater. - * - * @param lossGradient Function which computes gradient of the loss function at any given point - * @param updater Weights updater applied on every gradient descent step to decide how weights should be changed - */ - public GradientDescent(GradientFunction lossGradient, Updater updater) { - this.lossGradient = lossGradient; - this.updater = updater; - } - - /** - * Sets max number of gradient descent iterations. - * - * @param maxIterations Max number of gradient descent iterations - * @return This gradient descent instance - */ - public GradientDescent withMaxIterations(int maxIterations) { - assert maxIterations >= 0; - - this.maxIterations = maxIterations; - - return this; - } - - /** - * Sets convergence tolerance. - * - * @param convergenceTol Condition which decides iteration termination - * @return This gradient descent instance - */ - public GradientDescent withConvergenceTol(double convergenceTol) { - assert convergenceTol >= 0; - - this.convergenceTol = convergenceTol; - - return this; - } - - /** - * Computes point where loss function takes minimal value. - * - * @param data Inputs parameters of loss function - * @param initWeights Initial weights - * @return Point where loss function takes minimal value - */ - public Vector optimize(Matrix data, Vector initWeights) { - Vector weights = initWeights, oldWeights = null, oldGradient = null; - IgniteFunction gradientFunction = getLossGradientFunction(data); - - for (int iteration = 0; iteration < maxIterations; iteration++) { - Vector gradient = gradientFunction.apply(weights); - Vector newWeights = updater.compute(oldWeights, oldGradient, weights, gradient, iteration); - - if (isConverged(weights, newWeights)) - return newWeights; - else { - oldGradient = gradient; - oldWeights = weights; - weights = newWeights; - } - } - return weights; - } - - /** - * Calculates gradient based in distributed matrix using {@link SparseDistributedMatrixMapReducer}. - * - * @param data Distributed matrix - * @param weights Point to calculate gradient - * @return Gradient - */ - private Vector calculateDistributedGradient(SparseDistributedMatrix data, Vector weights) { - SparseDistributedMatrixMapReducer mapReducer = new SparseDistributedMatrixMapReducer(data); - return mapReducer.mapReduce( - (matrix, args) -> { - Matrix inputs = extractInputs(matrix); - Vector groundTruth = extractGroundTruth(matrix); - - return lossGradient.compute(inputs, groundTruth, args); - }, - gradients -> { - int cnt = 0; - Vector resGradient = new DenseLocalOnHeapVector(data.columnSize()); - - for (Vector gradient : gradients) { - if (gradient != null) { - resGradient = resGradient.plus(gradient); - cnt++; - } - } - - return resGradient.divide(cnt); - }, - weights); - } - - /** - * Tests if gradient descent process converged. - * - * @param weights Weights - * @param newWeights New weights - * @return {@code true} if process has converged, otherwise {@code false} - */ - private boolean isConverged(Vector weights, Vector newWeights) { - if (convergenceTol == 0) - return false; - else { - double solutionVectorDiff = weights.minus(newWeights).kNorm(2.0); - return solutionVectorDiff < convergenceTol * Math.max(newWeights.kNorm(2.0), 1.0); - } - } - - /** - * Extracts first column with ground truth from the data set matrix. - * - * @param data data to build model - * @return Ground truth vector - */ - private Vector extractGroundTruth(Matrix data) { - return data.getCol(0); - } - - /** - * Extracts all inputs from data set matrix and updates matrix so that first column contains value 1.0. - * - * @param data data to build model - * @return Inputs matrix - */ - private Matrix extractInputs(Matrix data) { - data = data.copy(); - data.assignColumn(0, new FunctionVector(data.rowSize(), row -> 1.0)); - return data; - } - - /** Makes carrying of the gradient function and fixes data matrix. */ - private IgniteFunction getLossGradientFunction(Matrix data) { - if (data instanceof SparseDistributedMatrix) { - SparseDistributedMatrix distributedMatrix = (SparseDistributedMatrix)data; - - if (distributedMatrix.getStorage().storageMode() == StorageConstants.ROW_STORAGE_MODE) - return weights -> calculateDistributedGradient(distributedMatrix, weights); - } - - Matrix inputs = extractInputs(data); - Vector groundTruth = extractGroundTruth(data); - - return weights -> lossGradient.compute(inputs, groundTruth, weights); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/GradientFunction.java b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/GradientFunction.java deleted file mode 100644 index a6a1e71a38701..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/GradientFunction.java +++ /dev/null @@ -1,31 +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.optimization; - -import java.io.Serializable; -import org.apache.ignite.ml.math.Matrix; -import org.apache.ignite.ml.math.Vector; - -/** - * Function which computes gradient of the loss function at any given point. - */ -@FunctionalInterface -public interface GradientFunction extends Serializable { - /** */ - Vector compute(Matrix inputs, Vector groundTruth, Vector pnt); -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/LeastSquaresGradientFunction.java b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/LeastSquaresGradientFunction.java deleted file mode 100644 index 4d90e3b994a43..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/LeastSquaresGradientFunction.java +++ /dev/null @@ -1,33 +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.optimization; - -import org.apache.ignite.ml.math.Matrix; -import org.apache.ignite.ml.math.Vector; - -/** - * Function which computes gradient of least square loss function. - */ -public class LeastSquaresGradientFunction implements GradientFunction { - /** - * {@inheritDoc} - */ - @Override public Vector compute(Matrix inputs, Vector groundTruth, Vector pnt) { - return inputs.transpose().times(inputs.times(pnt).minus(groundTruth)); - } -} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/util/SparseDistributedMatrixMapReducer.java b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/util/SparseDistributedMatrixMapReducer.java deleted file mode 100644 index 20f861e84305e..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/util/SparseDistributedMatrixMapReducer.java +++ /dev/null @@ -1,84 +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.optimization.util; - -import java.util.Collection; -import java.util.Map; -import org.apache.ignite.Ignite; -import org.apache.ignite.Ignition; -import org.apache.ignite.cache.affinity.Affinity; -import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.ml.math.Matrix; -import org.apache.ignite.ml.math.distributed.keys.RowColMatrixKey; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.ml.math.impls.storage.matrix.SparseDistributedMatrixStorage; - -/** - * Wrapper of {@link SparseDistributedMatrix} which allow to perform computation on every node containing a part of the - * distributed matrix, get results and then reduce them. - */ -public class SparseDistributedMatrixMapReducer { - /** */ - private final SparseDistributedMatrix distributedMatrix; - - /** */ - public SparseDistributedMatrixMapReducer( - SparseDistributedMatrix distributedMatrix) { - this.distributedMatrix = distributedMatrix; - } - - /** */ - public R mapReduce(IgniteBiFunction mapper, IgniteFunction, R> reducer, T args) { - Ignite ignite = Ignition.localIgnite(); - SparseDistributedMatrixStorage storage = (SparseDistributedMatrixStorage)distributedMatrix.getStorage(); - - int colSize = distributedMatrix.columnSize(); - - Collection results = ignite - .compute(ignite.cluster().forDataNodes(storage.cacheName())) - .broadcast(arguments -> { - Ignite locIgnite = Ignition.localIgnite(); - - Affinity affinity = locIgnite.affinity(storage.cacheName()); - ClusterNode locNode = locIgnite.cluster().localNode(); - - Map> keys = affinity.mapKeysToNodes(storage.getAllKeys()); - Collection locKeys = keys.get(locNode); - - if (locKeys != null) { - int idx = 0; - Matrix locMatrix = new DenseLocalOnHeapMatrix(locKeys.size(), colSize); - - for (RowColMatrixKey key : locKeys) { - Map row = storage.cache().get(key); - - for (Map.Entry cell : row.entrySet()) - locMatrix.set(idx, cell.getKey(), cell.getValue()); - - idx++; - } - return mapper.apply(locMatrix, arguments); - } - return null; - }, args); - return reducer.apply(results); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/util/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/util/package-info.java deleted file mode 100644 index cb01ab6268a90..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/util/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. - */ - -/** - * - * Contains util classes used in optimization package. - */ -package org.apache.ignite.ml.optimization.util; \ No newline at end of file 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 9526db1e366ae..095aa31354ef6 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 @@ -17,18 +17,17 @@ package org.apache.ignite.ml.regressions.linear; +import java.util.Arrays; import org.apache.ignite.ml.dataset.DatasetBuilder; +import org.apache.ignite.ml.dataset.primitive.builder.data.SimpleLabeledDatasetDataBuilder; import org.apache.ignite.ml.math.Vector; import org.apache.ignite.ml.math.functions.IgniteBiFunction; import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; -import org.apache.ignite.ml.math.isolve.LinSysPartitionDataBuilderOnHeap; import org.apache.ignite.ml.math.isolve.lsqr.AbstractLSQR; import org.apache.ignite.ml.math.isolve.lsqr.LSQROnHeap; import org.apache.ignite.ml.math.isolve.lsqr.LSQRResult; import org.apache.ignite.ml.trainers.SingleLabelDatasetTrainer; -import java.util.Arrays; - /** * Trainer of the linear regression model based on LSQR algorithm. * @@ -43,7 +42,10 @@ public class LinearRegressionLSQRTrainer implements SingleLabelDatasetTrainer
  • lsqr = new LSQROnHeap<>( datasetBuilder, - new LinSysPartitionDataBuilderOnHeap<>(new FeatureExtractorWrapper<>(featureExtractor), lbExtractor) + new SimpleLabeledDatasetDataBuilder<>( + new FeatureExtractorWrapper<>(featureExtractor), + lbExtractor.andThen(e -> new double[]{e}) + ) )) { res = lsqr.solve(0, 1e-12, 1e-12, 1e8, -1, false, null); } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionQRTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionQRTrainer.java deleted file mode 100644 index 5de3cda5aee0d..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionQRTrainer.java +++ /dev/null @@ -1,72 +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.regressions.linear; - -import org.apache.ignite.ml.Trainer; -import org.apache.ignite.ml.math.Matrix; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.decompositions.QRDSolver; -import org.apache.ignite.ml.math.decompositions.QRDecomposition; -import org.apache.ignite.ml.math.impls.vector.FunctionVector; - -/** - * Linear regression trainer based on least squares loss function and QR decomposition. - */ -public class LinearRegressionQRTrainer implements Trainer { - /** - * {@inheritDoc} - */ - @Override public LinearRegressionModel train(Matrix data) { - Vector groundTruth = extractGroundTruth(data); - Matrix inputs = extractInputs(data); - - QRDecomposition decomposition = new QRDecomposition(inputs); - QRDSolver solver = new QRDSolver(decomposition.getQ(), decomposition.getR()); - - Vector variables = solver.solve(groundTruth); - Vector weights = variables.viewPart(1, variables.size() - 1); - - double intercept = variables.get(0); - - return new LinearRegressionModel(weights, intercept); - } - - /** - * Extracts first column with ground truth from the data set matrix. - * - * @param data data to build model - * @return Ground truth vector - */ - private Vector extractGroundTruth(Matrix data) { - return data.getCol(0); - } - - /** - * Extracts all inputs from data set matrix and updates matrix so that first column contains value 1.0. - * - * @param data data to build model - * @return Inputs matrix - */ - private Matrix extractInputs(Matrix data) { - data = data.copy(); - - data.assignColumn(0, new FunctionVector(data.rowSize(), row -> 1.0)); - - return data; - } -} 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 9be3fdd2a8aa2..98b888587e1b1 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 @@ -30,7 +30,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.apache.ignite.ml.trainers.group.UpdatesStrategy; +import org.apache.ignite.ml.nn.UpdatesStrategy; import java.io.Serializable; import java.util.Arrays; @@ -110,6 +110,9 @@ public LinearRegressionSGDTrainer(UpdatesStrategy { - /** - * Train the model based on provided data. - * - * @param data Data for training. - * @return Trained model. - */ - public M train(T data); -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/BaseLocalProcessorJob.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/BaseLocalProcessorJob.java deleted file mode 100644 index e20a55a9507e6..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/BaseLocalProcessorJob.java +++ /dev/null @@ -1,146 +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.trainers.group; - -import java.io.Serializable; -import java.util.List; -import java.util.Objects; -import java.util.UUID; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteException; -import org.apache.ignite.Ignition; -import org.apache.ignite.cache.affinity.Affinity; -import org.apache.ignite.compute.ComputeJob; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.functions.IgniteSupplier; - -/** - * Base job for group training. - * It's purpose is to apply worker to each element (cache key or cache entry) of given cache specified - * by keySupplier. Worker produces {@link ResultAndUpdates} object which contains 'side effects' which are updates - * needed to apply to caches and computation result. - * After we get all {@link ResultAndUpdates} we merge all 'update' parts of them for each node - * and apply them on corresponding node, also we reduce all 'result' by some given reducer. - * - * @param Type of keys of cache used for group trainer. - * @param Type of values of cache used for group trainer. - * @param Type of elements to which workers are applier. - * @param Type of result of worker. - */ -public abstract class BaseLocalProcessorJob implements ComputeJob { - /** - * UUID of group training. - */ - protected UUID trainingUUID; - - /** - * Worker. - */ - protected IgniteFunction> worker; - - /** - * Supplier of keys determining elements to which worker should be applied. - */ - protected IgniteSupplier>> keySupplier; - - /** - * Operator used to reduce results from worker. - */ - protected IgniteFunction, R> reducer; - - /** - * Name of cache used for training. - */ - protected String cacheName; - - /** - * Construct instance of this class with given arguments. - * - * @param worker Worker. - * @param keySupplier Supplier of keys. - * @param reducer Reducer. - * @param trainingUUID UUID of training. - * @param cacheName Name of cache used for training. - */ - public BaseLocalProcessorJob( - IgniteFunction> worker, - IgniteSupplier>> keySupplier, - IgniteFunction, R> reducer, - UUID trainingUUID, String cacheName) { - this.worker = worker; - this.keySupplier = keySupplier; - this.reducer = reducer; - this.trainingUUID = trainingUUID; - this.cacheName = cacheName; - } - - /** {@inheritDoc} */ - @Override public void cancel() { - // NO-OP. - } - - /** {@inheritDoc} */ - @Override public R execute() throws IgniteException { - List> resultsAndUpdates = toProcess(). - map(worker). - collect(Collectors.toList()); - - ResultAndUpdates totalRes = ResultAndUpdates.sum(reducer, resultsAndUpdates.stream().filter(Objects::nonNull).collect(Collectors.toList())); - - totalRes.applyUpdates(ignite()); - - return totalRes.result(); - } - - /** - * Get stream of elements to process. - * - * @return Stream of elements to process. - */ - protected abstract Stream toProcess(); - - /** - * Ignite instance. - * - * @return Ignite instance. - */ - protected static Ignite ignite() { - return Ignition.localIgnite(); - } - - /** - * Get cache used for training. - * - * @return Cache used for training. - */ - protected IgniteCache, V> cache() { - return ignite().getOrCreateCache(cacheName); - } - - /** - * Get affinity function for cache used in group training. - * - * @return Affinity function for cache used in group training. - */ - protected Affinity affinity() { - return ignite().affinity(cacheName); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/ConstModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/ConstModel.java deleted file mode 100644 index 75f817915a09c..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/ConstModel.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.ml.trainers.group; - -import org.apache.ignite.ml.Model; - -/** - * Model which outputs given constant. - * - * @param Type of constant. - */ -public class ConstModel implements Model { - /** - * Constant to be returned by this model. - */ - private T c; - - /** - * Create instance of this class specified by input parameters. - * - * @param c Constant to be returned by this model. - */ - public ConstModel(T c) { - this.c = c; - } - - /** {@inheritDoc} */ - @Override public T apply(T val) { - return c; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainer.java deleted file mode 100644 index fb34bf7443211..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainer.java +++ /dev/null @@ -1,208 +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.trainers.group; - -import java.io.Serializable; -import java.util.List; -import java.util.UUID; -import java.util.stream.Stream; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.ml.Model; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.functions.IgniteSupplier; -import org.apache.ignite.ml.trainers.Trainer; -import org.apache.ignite.ml.trainers.group.chain.ComputationsChain; -import org.apache.ignite.ml.trainers.group.chain.EntryAndContext; -import org.apache.ignite.ml.trainers.group.chain.HasTrainingUUID; - -/** - * Class encapsulating synchronous distributed group training. - * Training is performed by following scheme: - * 1. For specified set of keys distributed initialization is done. For each key some initialization result is returned. - * 2. All initialization results are processed locally and reduced into some object of type I. - * 3. While 'shouldContinue' condition is true, training loop step is executed. - * 4. After loop is finished, data from each key from final key set is collected. - * 5. Data collected on previous step is transformed into a model which is returned as final result. - * Note that all methods returning functions, suppliers etc should return values with minimal dependencies because they are serialized - * with all dependent objects. - * - * @param Type of local context of the training. - * @param Type of data in {@link GroupTrainerCacheKey} keys on which the training is done. - * @param Type of cache values on which the training is done. - * @param Type of data returned after initializing of distributed context. - * @param Type of result returned after training from each node. - * @param Type of data which is fed into each training loop step and returned from it. - * @param Type of model returned after training. - * @param Type of input to this trainer. - * @param Type of distributed context which is needed for forming final result which is send from each node to trainer for final model creation. - */ -abstract class GroupTrainer, G> implements Trainer { - /** - * Cache on which training is performed. For example it can be cache of neural networks. - */ - protected IgniteCache, V> cache; - - /** - * Ignite instance. - */ - protected Ignite ignite; - - /** - * Construct an instance of this class. - * - * @param cache Cache on which training is performed. - * @param ignite Ignite instance. - */ - GroupTrainer( - IgniteCache, V> cache, - Ignite ignite) { - this.cache = cache; - this.ignite = ignite; - } - - /** {@inheritDoc} */ - @Override public final M train(T data) { - UUID trainingUUID = UUID.randomUUID(); - LC locCtx = initialLocalContext(data, trainingUUID); - - GroupTrainingContext ctx = new GroupTrainingContext<>(locCtx, cache, ignite); - ComputationsChain chain = (i, c) -> i; - IgniteFunction, ResultAndUpdates> distributedInitializer - = distributedInitializer(data); - - init(data, trainingUUID); - - M res = chain. - thenDistributedForKeys(distributedInitializer, (t, lc) -> data.initialKeys(trainingUUID), - reduceDistributedInitData()). - thenLocally(this::locallyProcessInitData). - thenWhile(this::shouldContinue, trainingLoopStep()). - thenDistributedForEntries(this::extractContextForFinalResultCreation, finalResultsExtractor(), - this::finalResultKeys, finalResultsReducer()). - thenLocally(this::mapFinalResult). - process(data, ctx); - - cleanup(locCtx); - - return res; - } - - /** - * Create initial local context from data given as input to trainer. - * - * @param data Data given as input to this trainer. - * @param trainingUUID UUID of this training. - * @return Initial local context. - */ - protected abstract LC initialLocalContext(T data, UUID trainingUUID); - - /** Override in subclasses if needed. */ - protected void init(T data, UUID trainingUUID) { - } - - /** - * Get function for initialization for each of keys specified in initial key set. - * - * @param data Data given to this trainer as input. - * @return Function for initialization for each of keys specified in initial key set. - */ - protected abstract IgniteFunction, ResultAndUpdates> distributedInitializer(T data); - - /** - * Get reducer to reduce data collected from initialization of each key specified in initial key set. - * - * @return Reducer to reduce data collected from initialization of each key specified in initial key set. - */ - protected abstract IgniteFunction, IN> reduceDistributedInitData(); - - /** - * Transform data from initialization step into data which is fed as input to first step of training loop. - * - * @param data Data from initialization step. - * @param locCtx Local context. - * @return Data which is fed as input to first step of training loop. - */ - protected abstract I locallyProcessInitData(IN data, LC locCtx); - - /** - * Training loop step. - * - * @return Result of training loop step. - */ - protected abstract ComputationsChain trainingLoopStep(); - - /** - * Condition specifying if training loop should continue. - * - * @param data First time, data returned by locallyProcessInitData then data returned by last step of loop. - * @param locCtx Local context. - * @return Boolean value indicating if training loop should continue. - */ - protected abstract boolean shouldContinue(I data, LC locCtx); - - /** - * Extract context for final result creation. Each key from the final keys set will be processed with - * finalResultsExtractor. While entry data (i.e. key and value) for each key varies, some data can be common for all - * processed entries. This data is called context. - * - * @param data Data returned from last training loop step. - * @param locCtx Local context. - * @return Context. - */ - protected abstract IgniteSupplier extractContextForFinalResultCreation(I data, LC locCtx); - - /** - * Keys for final result creation. - * - * @param data Data returned from the last training loop step. - * @param locCtx Local context. - * @return Stream of keys for final result creation. - */ - protected abstract IgniteSupplier>> finalResultKeys(I data, LC locCtx); - - /** - * Get function for extracting final result from each key specified in finalResultKeys. - * - * @return Function for extracting final result from each key specified in finalResultKeys. - */ - protected abstract IgniteFunction, ResultAndUpdates> finalResultsExtractor(); - - /** - * Get function for reducing final results. - * - * @return Function for reducing final results. - */ - protected abstract IgniteFunction, R> finalResultsReducer(); - - /** - * Map final result to model which is returned by trainer. - * - * @param res Final result. - * @param locCtx Local context. - * @return Model resulted from training. - */ - protected abstract M mapFinalResult(R res, LC locCtx); - - /** - * Performs cleanups of temporary objects created by this trainer. - * - * @param locCtx Local context. - */ - protected abstract void cleanup(LC locCtx); -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerBaseProcessorTask.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerBaseProcessorTask.java deleted file mode 100644 index b192f421f1a88..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerBaseProcessorTask.java +++ /dev/null @@ -1,144 +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.trainers.group; - -import java.io.Serializable; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.UUID; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteException; -import org.apache.ignite.cache.affinity.Affinity; -import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.compute.ComputeJob; -import org.apache.ignite.compute.ComputeJobResult; -import org.apache.ignite.compute.ComputeTaskAdapter; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.functions.IgniteSupplier; -import org.jetbrains.annotations.Nullable; - -/** - * Base task for group trainer. - * - * @param Type of cache keys of cache used for training. - * @param Type of cache values of cache used for training. - * @param Type of context (common part of data needed for computation). - * @param Type of arguments of workers. - * @param Type of computation result. - */ -public abstract class GroupTrainerBaseProcessorTask extends ComputeTaskAdapter { - /** - * Context supplier. - */ - protected final IgniteSupplier ctxSupplier; - - /** - * UUID of training. - */ - protected final UUID trainingUUID; - - /** - * Worker. - */ - protected IgniteFunction> worker; - - /** - * Reducer used for reducing of computations on specified keys. - */ - protected final IgniteFunction, R> reducer; - - /** - * Name of cache on which training is done. - */ - protected final String cacheName; - - /** - * Supplier of keys on which worker should be executed. - */ - protected final IgniteSupplier>> keysSupplier; - - /** - * Ignite instance. - */ - protected final Ignite ignite; - - /** - * Construct an instance of this class with specified parameters. - * - * @param trainingUUID UUID of training. - * @param ctxSupplier Supplier of context. - * @param worker Function calculated on each of specified keys. - * @param keysSupplier Supplier of keys on which training is done. - * @param reducer Reducer used for reducing results of computation performed on each of specified keys. - * @param cacheName Name of cache on which training is done. - * @param ignite Ignite instance. - */ - public GroupTrainerBaseProcessorTask(UUID trainingUUID, - IgniteSupplier ctxSupplier, - IgniteFunction> worker, - IgniteSupplier>> keysSupplier, - IgniteFunction, R> reducer, - String cacheName, - Ignite ignite) { - this.trainingUUID = trainingUUID; - this.ctxSupplier = ctxSupplier; - this.worker = worker; - this.keysSupplier = keysSupplier; - this.reducer = reducer; - this.cacheName = cacheName; - this.ignite = ignite; - } - - /** {@inheritDoc} */ - @Nullable @Override public Map map(List subgrid, - @Nullable Void arg) throws IgniteException { - Map res = new HashMap<>(); - - for (ClusterNode node : subgrid) { - BaseLocalProcessorJob job = createJob(); - res.put(job, node); - } - - return res; - } - - /** {@inheritDoc} */ - @Nullable @Override public R reduce(List results) throws IgniteException { - return reducer.apply(results.stream().map(res -> (R)res.getData()).filter(Objects::nonNull).collect(Collectors.toList())); - } - - /** - * Create job for execution on subgrid. - * - * @return Job for execution on subgrid. - */ - protected abstract BaseLocalProcessorJob createJob(); - - /** - * Get affinity function of cache on which training is done. - * - * @return Affinity function of cache on which training is done. - */ - protected Affinity affinity() { - return ignite.affinity(cacheName); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerCacheKey.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerCacheKey.java deleted file mode 100644 index 5e4cb76b6a663..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerCacheKey.java +++ /dev/null @@ -1,125 +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.trainers.group; - -import java.util.UUID; -import org.apache.ignite.cache.affinity.AffinityKeyMapped; - -/** - * Class used as a key for caches on which {@link GroupTrainer} works. - * Structurally it is a triple: (nodeLocalEntityIndex, trainingUUID, data); - * nodeLocalEntityIndex is used to map key to node; - * trainingUUID is id of training; - * data is some custom data stored in this key, for example if we want to store three neural networks on one node - * for training with training UUID == trainingUUID, we can use keys - * (1, trainingUUID, networkIdx1), (1, trainingUUID, networkIdx2), (1, trainingUUID, networkIdx3). - * - * @param Type of data part of this key. - */ -public class GroupTrainerCacheKey { - /** - * Part of key for key-to-node affinity. - */ - @AffinityKeyMapped - private Long nodeLocEntityIdx; - - /** - * UUID of training. - */ - private UUID trainingUUID; - - /** - * Data. - */ - K data; - - /** - * Construct instance of this class. - * - * @param nodeLocEntityIdx Part of key for key-to-node affinity. - * @param data Data. - * @param trainingUUID Training UUID. - */ - public GroupTrainerCacheKey(long nodeLocEntityIdx, K data, UUID trainingUUID) { - this.nodeLocEntityIdx = nodeLocEntityIdx; - this.trainingUUID = trainingUUID; - this.data = data; - } - - /** - * Construct instance of this class. - * - * @param nodeLocEntityIdx Part of key for key-to-node affinity. - * @param data Data. - * @param trainingUUID Training UUID. - */ - public GroupTrainerCacheKey(int nodeLocEntityIdx, K data, UUID trainingUUID) { - this((long)nodeLocEntityIdx, data, trainingUUID); - } - - /** - * Get part of key used for key-to-node affinity. - * - * @return Part of key used for key-to-node affinity. - */ - public Long nodeLocalEntityIndex() { - return nodeLocEntityIdx; - } - - /** - * Get UUID of training. - * - * @return UUID of training. - */ - public UUID trainingUUID() { - return trainingUUID; - } - - /** - * Get data. - * - * @return Data. - */ - public K data() { - return data; - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - - GroupTrainerCacheKey key = (GroupTrainerCacheKey)o; - - if (nodeLocEntityIdx != null ? !nodeLocEntityIdx.equals(key.nodeLocEntityIdx) : key.nodeLocEntityIdx != null) - return false; - if (trainingUUID != null ? !trainingUUID.equals(key.trainingUUID) : key.trainingUUID != null) - return false; - return data != null ? data.equals(key.data) : key.data == null; - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - int res = nodeLocEntityIdx != null ? nodeLocEntityIdx.hashCode() : 0; - res = 31 * res + (trainingUUID != null ? trainingUUID.hashCode() : 0); - res = 31 * res + (data != null ? data.hashCode() : 0); - return res; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerEntriesProcessorTask.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerEntriesProcessorTask.java deleted file mode 100644 index daa396fac5dc4..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerEntriesProcessorTask.java +++ /dev/null @@ -1,64 +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.trainers.group; - -import java.io.Serializable; -import java.util.List; -import java.util.UUID; -import java.util.stream.Stream; -import org.apache.ignite.Ignite; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.functions.IgniteSupplier; -import org.apache.ignite.ml.trainers.group.chain.EntryAndContext; - -/** - * Task for processing entries of cache used for training. - * - * @param Type of cache keys of cache used for training. - * @param Type of cache values of cache used for training. - * @param Type of context (common part of data needed for computation). - * @param Type of computation result. - */ -public class GroupTrainerEntriesProcessorTask - extends GroupTrainerBaseProcessorTask, R> { - /** - * Construct instance of this class with given parameters. - * - * @param trainingUUID UUID of training. - * @param ctxSupplier Supplier of context. - * @param worker Function calculated on each of specified keys. - * @param keysSupplier Supplier of keys on which training is done. - * @param reducer Reducer used for reducing results of computation performed on each of specified keys. - * @param cacheName Name of cache on which training is done. - * @param ignite Ignite instance. - */ - public GroupTrainerEntriesProcessorTask(UUID trainingUUID, - IgniteSupplier ctxSupplier, - IgniteFunction, ResultAndUpdates> worker, - IgniteSupplier>> keysSupplier, - IgniteFunction, R> reducer, - String cacheName, - Ignite ignite) { - super(trainingUUID, ctxSupplier, worker, keysSupplier, reducer, cacheName, ignite); - } - - /** {@inheritDoc} */ - @Override protected BaseLocalProcessorJob, R> createJob() { - return new LocalEntriesProcessorJob<>(ctxSupplier, worker, keysSupplier, reducer, trainingUUID, cacheName); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerInput.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerInput.java deleted file mode 100644 index ae75f16091b2c..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerInput.java +++ /dev/null @@ -1,37 +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.trainers.group; - -import java.util.UUID; -import java.util.stream.Stream; -import org.apache.ignite.ml.math.functions.IgniteSupplier; - -/** - * Interface for {@link GroupTrainer} inputs. - * - * @param Types of cache keys used for group training. - */ -public interface GroupTrainerInput { - /** - * Get supplier of stream of keys used for initialization of {@link GroupTrainer}. - * - * @param trainingUUID UUID of training. - * @return Supplier of stream of keys used for initialization of {@link GroupTrainer}. - */ - IgniteSupplier>> initialKeys(UUID trainingUUID); -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerKeysProcessorTask.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerKeysProcessorTask.java deleted file mode 100644 index 7ac18f8881ec2..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerKeysProcessorTask.java +++ /dev/null @@ -1,62 +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.trainers.group; - -import java.io.Serializable; -import java.util.List; -import java.util.UUID; -import java.util.stream.Stream; -import org.apache.ignite.Ignite; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.functions.IgniteSupplier; -import org.apache.ignite.ml.trainers.group.chain.KeyAndContext; - -/** - * Task for processing entries of cache used for training. - * - * @param Type of cache keys of cache used for training. - * @param Type of context (common part of data needed for computation). - * @param Type of computation result. - */ -public class GroupTrainerKeysProcessorTask extends GroupTrainerBaseProcessorTask, R> { - /** - * Construct instance of this class with specified parameters. - * - * @param trainingUUID UUID of training. - * @param ctxSupplier Context supplier. - * @param worker Function calculated on each of specified keys. - * @param keysSupplier Supplier of keys on which computations should be done. - * @param reducer Reducer used for reducing results of computation performed on each of specified keys. - * @param cacheName Name of cache on which training is done. - * @param ignite Ignite instance. - */ - public GroupTrainerKeysProcessorTask(UUID trainingUUID, - IgniteSupplier ctxSupplier, - IgniteFunction, ResultAndUpdates> worker, - IgniteSupplier>> keysSupplier, - IgniteFunction, R> reducer, - String cacheName, - Ignite ignite) { - super(trainingUUID, ctxSupplier, worker, keysSupplier, reducer, cacheName, ignite); - } - - /** {@inheritDoc} */ - @Override protected BaseLocalProcessorJob, R> createJob() { - return new LocalKeysProcessorJob<>(ctxSupplier, worker, keysSupplier, reducer, trainingUUID, cacheName); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainingContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainingContext.java deleted file mode 100644 index cbd04b24fb475..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainingContext.java +++ /dev/null @@ -1,98 +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.trainers.group; - -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.ml.trainers.group.chain.HasTrainingUUID; - -/** - * Context for group training. - * - * @param Type of keys of cache used for group training. - * @param Type of values of cache used for group training. - * @param Type of local context used for training. - */ -public class GroupTrainingContext { - /** - * Local context. - */ - private L locCtx; - - /** - * Cache used for training. - */ - private IgniteCache, V> cache; - - /** - * Ignite instance. - */ - private Ignite ignite; - - /** - * Construct instance of this class. - * - * @param locCtx Local context. - * @param cache Information about cache used for training. - * @param ignite Ignite instance. - */ - public GroupTrainingContext(L locCtx, IgniteCache, V> cache, Ignite ignite) { - this.locCtx = locCtx; - this.cache = cache; - this.ignite = ignite; - } - - /** - * Construct new training context with same parameters but with new cache. - * - * @param newCache New cache. - * @param Type of keys of new cache. - * @param Type of values of new cache. - * @return New training context with same parameters but with new cache. - */ - public GroupTrainingContext withCache(IgniteCache, V1> newCache) { - return new GroupTrainingContext<>(locCtx, newCache, ignite); - } - - /** - * Get local context. - * - * @return Local context. - */ - public L localContext() { - return locCtx; - } - - /** - * Get cache used for training. - * - * @return Cache used for training. - */ - public IgniteCache, V> cache() { - return cache; - } - - /** - * Get ignite instance. - * - * @return Ignite instance. - */ - public Ignite ignite() { - return ignite; - } -} \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/LocalEntriesProcessorJob.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/LocalEntriesProcessorJob.java deleted file mode 100644 index d035aa5b26822..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/LocalEntriesProcessorJob.java +++ /dev/null @@ -1,85 +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.trainers.group; - -import java.io.Serializable; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.UUID; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.functions.IgniteSupplier; -import org.apache.ignite.ml.trainers.group.chain.EntryAndContext; - -/** - * {@link BaseLocalProcessorJob} specified to entry processing. - * - * @param Type of cache used for group training. - * @param Type of values used for group training. - * @param Type of context. - * @param Type of result returned by worker. - */ -public class LocalEntriesProcessorJob extends BaseLocalProcessorJob, R> { - /** - * Supplier of context for worker. - */ - private final IgniteSupplier ctxSupplier; - - /** - * Construct an instance of this class. - * - * @param ctxSupplier Supplier for context for worker. - * @param worker Worker. - * @param keySupplier Supplier of keys. - * @param reducer Reducer. - * @param trainingUUID UUID for training. - * @param cacheName Name of cache used for training. - */ - public LocalEntriesProcessorJob(IgniteSupplier ctxSupplier, - IgniteFunction, ResultAndUpdates> worker, - IgniteSupplier>> keySupplier, - IgniteFunction, R> reducer, - UUID trainingUUID, String cacheName) { - super(worker, keySupplier, reducer, trainingUUID, cacheName); - this.ctxSupplier = ctxSupplier; - } - - /** {@inheritDoc} */ - @Override protected Stream> toProcess() { - C ctx = ctxSupplier.get(); - - return selectLocalEntries().map(e -> new EntryAndContext<>(e, ctx)); - } - - /** - * Select entries for processing by worker. - * - * @return Entries for processing by worker. - */ - private Stream, V>> selectLocalEntries() { - Set> keys = keySupplier.get(). - filter(k -> Objects.requireNonNull(affinity().mapKeyToNode(k)).isLocal()). - filter(k -> k.trainingUUID().equals(trainingUUID)). - collect(Collectors.toSet()); - - return cache().getAll(keys).entrySet().stream(); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/LocalKeysProcessorJob.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/LocalKeysProcessorJob.java deleted file mode 100644 index cad53c937bd66..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/LocalKeysProcessorJob.java +++ /dev/null @@ -1,78 +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.trainers.group; - -import java.io.Serializable; -import java.util.List; -import java.util.Objects; -import java.util.UUID; -import java.util.stream.Stream; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.functions.IgniteSupplier; -import org.apache.ignite.ml.trainers.group.chain.KeyAndContext; - -/** - * {@link BaseLocalProcessorJob} specified to keys processing. - * - * @param Type of cache used for group training. - * @param Type of values used for group training. - * @param Type of context. - * @param Type of result returned by worker. - */ -public class LocalKeysProcessorJob extends BaseLocalProcessorJob, R> { - /** - * Supplier of worker context. - */ - private final IgniteSupplier ctxSupplier; - - /** - * Construct instance of this class with given arguments. - * - * @param worker Worker. - * @param keySupplier Supplier of keys. - * @param reducer Reducer. - * @param trainingUUID UUID of training. - * @param cacheName Name of cache used for training. - */ - public LocalKeysProcessorJob(IgniteSupplier ctxSupplier, - IgniteFunction, ResultAndUpdates> worker, - IgniteSupplier>> keySupplier, - IgniteFunction, R> reducer, - UUID trainingUUID, String cacheName) { - super(worker, keySupplier, reducer, trainingUUID, cacheName); - this.ctxSupplier = ctxSupplier; - } - - /** {@inheritDoc} */ - @Override protected Stream> toProcess() { - C ctx = ctxSupplier.get(); - - return selectLocalKeys().map(k -> new KeyAndContext<>(k, ctx)); - } - - /** - * Get subset of keys provided by keySupplier which are mapped to node on which code is executed. - * - * @return Subset of keys provided by keySupplier which are mapped to node on which code is executed. - */ - private Stream> selectLocalKeys() { - return keySupplier.get(). - filter(k -> Objects.requireNonNull(affinity().mapKeyToNode(k)).isLocal()). - filter(k -> k.trainingUUID().equals(trainingUUID)); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/Metaoptimizer.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/Metaoptimizer.java deleted file mode 100644 index 0ab6d32eb0184..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/Metaoptimizer.java +++ /dev/null @@ -1,93 +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.trainers.group; - -import java.util.List; -import org.apache.ignite.ml.math.functions.IgniteFunction; - -/** - * Class encapsulating data transformations in group training in {@link MetaoptimizerGroupTrainer}, which is adapter of - * {@link GroupTrainer}. - * - * @param Local context of {@link GroupTrainer}. - * @param Type of data which is processed in training loop step. - * @param Type of data returned by training loop step data processor. - * @param Type of data to which data returned by distributed initialization is mapped. - * @param Type of data returned by initialization. - * @param Type of data to which data returned by data processor is mapped. - */ -public interface Metaoptimizer { - /** - * Get function used to reduce distributed initialization results. - * - * @return Function used to reduce distributed initialization results. - */ - IgniteFunction, D> initialReducer(); - - /** - * Maps data returned by distributed initialization to data consumed by training loop step. - * - * @param data Data returned by distributed initialization. - * @param locCtx Local context. - * @return Mapping of data returned by distributed initialization to data consumed by training loop step. - */ - I locallyProcessInitData(D data, LC locCtx); - - /** - * Preprocess data for {@link MetaoptimizerGroupTrainer#dataProcessor()}. - * - * @return Preprocessed data for {@link MetaoptimizerGroupTrainer#dataProcessor()}. - */ - default IgniteFunction distributedPreprocessor() { - return x -> x; - } - - /** - * Get function used to map values returned by {@link MetaoptimizerGroupTrainer#dataProcessor()}. - * - * @return Function used to map values returned by {@link MetaoptimizerGroupTrainer#dataProcessor()}. - */ - IgniteFunction distributedPostprocessor(); - - /** - * Get binary operator used for reducing results returned by distributedPostprocessor. - * - * @return Binary operator used for reducing results returned by distributedPostprocessor. - */ - IgniteFunction, O> postProcessReducer(); - - /** - * Transform data returned by distributed part of training loop step into input fed into distributed part of training - * loop step. - * - * @param input Type of output of distributed part of training loop step. - * @param locCtx Local context. - * @return Result of transform data returned by distributed part of training loop step into input fed into distributed part of training - * loop step. - */ - I localProcessor(O input, LC locCtx); - - /** - * Returns value of predicate 'should training loop continue given previous step output and local context'. - * - * @param input Input of previous step. - * @param locCtx Local context. - * @return Value of predicate 'should training loop continue given previous step output and local context'. - */ - boolean shouldContinue(I input, LC locCtx); -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/MetaoptimizerDistributedStep.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/MetaoptimizerDistributedStep.java deleted file mode 100644 index 08e1f47d5ccb0..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/MetaoptimizerDistributedStep.java +++ /dev/null @@ -1,97 +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.trainers.group; - -import java.io.Serializable; -import java.util.List; -import java.util.stream.Stream; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.functions.IgniteSupplier; -import org.apache.ignite.ml.trainers.group.chain.DistributedEntryProcessingStep; -import org.apache.ignite.ml.trainers.group.chain.EntryAndContext; -import org.apache.ignite.ml.trainers.group.chain.HasTrainingUUID; - -/** - * Distributed step based on {@link Metaoptimizer}. - * - * @param Type of local context. - * @param Type of data in {@link GroupTrainerCacheKey}. - * @param Type of values of cache on which training is done. - * @param Type of distributed context. - * @param Type of data to which data returned by distributed initialization is mapped (see {@link Metaoptimizer}). - * @param Type of data to which data returned by data processor is mapped (see {@link Metaoptimizer}). - * @param Type of data which is processed in training loop step (see {@link Metaoptimizer}). - * @param Type of data returned by training loop step data processor (see {@link Metaoptimizer}). - * @param Type of data returned by initialization (see {@link Metaoptimizer}). - */ -class MetaoptimizerDistributedStep implements DistributedEntryProcessingStep { - /** - * {@link Metaoptimizer}. - */ - private final Metaoptimizer metaoptimizer; - - /** - * {@link MetaoptimizerGroupTrainer} for which this distributed step is used. - */ - private final MetaoptimizerGroupTrainer trainer; - - /** - * Construct instance of this class with given parameters. - * - * @param metaoptimizer Metaoptimizer. - * @param trainer {@link MetaoptimizerGroupTrainer} for which this distributed step is used. - */ - public MetaoptimizerDistributedStep(Metaoptimizer metaoptimizer, - MetaoptimizerGroupTrainer trainer) { - this.metaoptimizer = metaoptimizer; - this.trainer = trainer; - } - - /** {@inheritDoc} */ - @Override public IgniteSupplier remoteContextSupplier(I input, L locCtx) { - return trainer.remoteContextExtractor(input, locCtx); - } - - /** {@inheritDoc} */ - @Override public IgniteFunction, ResultAndUpdates> worker() { - IgniteFunction> dataProcessor = trainer.dataProcessor(); - IgniteFunction preprocessor = metaoptimizer.distributedPreprocessor(); - IgniteFunction postprocessor = metaoptimizer.distributedPostprocessor(); - IgniteFunction, X> ctxExtractor = trainer.trainingLoopStepDataExtractor(); - - return entryAndCtx -> { - X apply = ctxExtractor.apply(entryAndCtx); - preprocessor.apply(apply); - ResultAndUpdates res = dataProcessor.apply(apply); - O postprocessRes = postprocessor.apply(res.result()); - - return ResultAndUpdates.of(postprocessRes).setUpdates(res.updates()); - }; - } - - /** {@inheritDoc} */ - @Override public IgniteSupplier>> keys(I input, L locCtx) { - return trainer.keysToProcessInTrainingLoop(locCtx); - } - - /** {@inheritDoc} */ - @Override public IgniteFunction, O> reducer() { - return metaoptimizer.postProcessReducer(); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/MetaoptimizerGroupTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/MetaoptimizerGroupTrainer.java deleted file mode 100644 index bebfe3edcd817..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/MetaoptimizerGroupTrainer.java +++ /dev/null @@ -1,132 +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.trainers.group; - -import java.io.Serializable; -import java.util.List; -import java.util.UUID; -import java.util.stream.Stream; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.ml.Model; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.functions.IgniteSupplier; -import org.apache.ignite.ml.trainers.group.chain.Chains; -import org.apache.ignite.ml.trainers.group.chain.ComputationsChain; -import org.apache.ignite.ml.trainers.group.chain.EntryAndContext; -import org.apache.ignite.ml.trainers.group.chain.HasTrainingUUID; - -/** - * Group trainer using {@link Metaoptimizer}. - * Main purpose of this trainer is to extract various transformations (normalizations for example) of data which is processed - * in the training loop step into distinct entity called metaoptimizer and only fix the main part of logic in - * trainers extending this class. This way we'll be able to quickly switch between this transformations by using different metaoptimizers - * without touching main logic. - * - * @param Type of local context. - * @param Type of data in {@link GroupTrainerCacheKey} keys on which the training is done. - * @param Type of values of cache used in group training. - * @param Data type which is returned by distributed initializer. - * @param Type of final result returned by nodes on which training is done. - * @param Type of data which is fed into each training loop step and returned from it. - * @param Type of model returned after training. - * @param Type of input of this trainer. - * @param Type of distributed context which is needed for forming final result which is send from each node to trainer for final model creation. - * @param Type of output of postprocessor. - * @param Type of data which is processed by dataProcessor. - * @param Type of data which is returned by postprocessor. - */ -public abstract class MetaoptimizerGroupTrainer, - G, O extends Serializable, X, Y> extends - GroupTrainer { - /** - * Metaoptimizer. - */ - private Metaoptimizer metaoptimizer; - - /** - * Construct instance of this class. - * - * @param cache Cache on which group trainer is done. - * @param ignite Ignite instance. - */ - public MetaoptimizerGroupTrainer(Metaoptimizer metaoptimizer, - IgniteCache, V> cache, - Ignite ignite) { - super(cache, ignite); - this.metaoptimizer = metaoptimizer; - } - - /** - * Get function used to map EntryAndContext to type which is processed by dataProcessor. - * - * @return Function used to map EntryAndContext to type which is processed by dataProcessor. - */ - protected abstract IgniteFunction, X> trainingLoopStepDataExtractor(); - - /** - * Get supplier of keys which should be processed by training loop. - * - * @param locCtx Local text. - * @return Supplier of keys which should be processed by training loop. - */ - protected abstract IgniteSupplier>> keysToProcessInTrainingLoop(LC locCtx); - - /** - * Get supplier of context used in training loop step. - * - * @param input Input. - * @param ctx Local context. - * @return Supplier of context used in training loop step. - */ - protected abstract IgniteSupplier remoteContextExtractor(I input, LC ctx); - - /** {@inheritDoc} */ - @Override protected void init(T data, UUID trainingUUID) { - } - - /** - * Get function used to process data in training loop step. - * - * @return Function used to process data in training loop step. - */ - protected abstract IgniteFunction> dataProcessor(); - - /** {@inheritDoc} */ - @Override protected ComputationsChain trainingLoopStep() { - ComputationsChain chain = Chains.create(new MetaoptimizerDistributedStep<>(metaoptimizer, this)); - return chain.thenLocally(metaoptimizer::localProcessor); - } - - /** {@inheritDoc} */ - @Override protected I locallyProcessInitData(IN data, LC locCtx) { - return metaoptimizer.locallyProcessInitData(data, locCtx); - } - - /** {@inheritDoc} */ - @Override protected boolean shouldContinue(I data, LC locCtx) { - return metaoptimizer.shouldContinue(data, locCtx); - } - - /** {@inheritDoc} */ - @Override protected IgniteFunction, IN> reduceDistributedInitData() { - return metaoptimizer.initialReducer(); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/ResultAndUpdates.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/ResultAndUpdates.java deleted file mode 100644 index 9ed18af5242e9..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/ResultAndUpdates.java +++ /dev/null @@ -1,178 +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.trainers.group; - -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.concurrent.ConcurrentHashMap; -import java.util.stream.Collectors; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.ml.math.functions.IgniteFunction; - -/** - * Class containing result of computation and updates which should be made for caches. - * Purpose of this class is mainly performance optimization: suppose we have multiple computations which run in parallel - * and do some updates to caches. It is more efficient to collect all changes from all this computations and perform them - * in batch. - * - * @param Type of computation result. - */ -public class ResultAndUpdates { - /** - * Result of computation. - */ - private R res; - - /** - * Updates in the form cache name -> (key -> new value). - */ - private Map updates = new ConcurrentHashMap<>(); - - /** - * Construct an instance of this class. - * - * @param res Computation result. - */ - public ResultAndUpdates(R res) { - this.res = res; - } - - /** - * Construct an instance of this class. - * - * @param res Computation result. - * @param updates Map of updates in the form cache name -> (key -> new value). - */ - ResultAndUpdates(R res, Map updates) { - this.res = res; - this.updates = updates; - } - - /** - * Construct an empty result. - * - * @param Result type. - * @return Empty result. - */ - public static ResultAndUpdates empty() { - return new ResultAndUpdates<>(null); - } - - /** - * Construct {@link ResultAndUpdates} object from given result. - * - * @param res Result of computation. - * @param Type of result of computation. - * @return ResultAndUpdates object. - */ - public static ResultAndUpdates of(R res) { - return new ResultAndUpdates<>(res); - } - - /** - * Add a cache update to this object. - * - * @param cache Cache to be updated. - * @param key Key of cache to be updated. - * @param val New value. - * @param Type of key of cache to be updated. - * @param New value. - * @return This object. - */ - @SuppressWarnings("unchecked") - public ResultAndUpdates updateCache(IgniteCache cache, K key, V val) { - String name = cache.getName(); - - updates.computeIfAbsent(name, s -> new ConcurrentHashMap()); - updates.get(name).put(key, val); - - return this; - } - - /** - * Get result of computation. - * - * @return Result of computation. - */ - public R result() { - return res; - } - - /** - * Sum collection of ResultAndUpdate into one: results are reduced by specified binary operator and updates are merged. - * - * @param reducer Reducer used to combine computation results. - * @param resultsAndUpdates ResultAndUpdates to be combined with. - * @param Type of computation result. - * @return Sum of collection ResultAndUpdate objects. - */ - @SuppressWarnings("unchecked") - static ResultAndUpdates sum(IgniteFunction, R> reducer, - Collection> resultsAndUpdates) { - Map allUpdates = new HashMap<>(); - - for (ResultAndUpdates ru : resultsAndUpdates) { - for (String cacheName : ru.updates.keySet()) { - allUpdates.computeIfAbsent(cacheName, s -> new HashMap()); - - allUpdates.get(cacheName).putAll(ru.updates.get(cacheName)); - } - } - - List results = resultsAndUpdates.stream().map(ResultAndUpdates::result).filter(Objects::nonNull).collect(Collectors.toList()); - - return new ResultAndUpdates<>(reducer.apply(results), allUpdates); - } - - /** - * Get updates map. - * - * @return Updates map. - */ - public Map updates() { - return updates; - } - - /** - * Set updates map. - * - * @param updates New updates map. - * @return This object. - */ - ResultAndUpdates setUpdates(Map updates) { - this.updates = updates; - return this; - } - - /** - * Apply updates to caches. - * - * @param ignite Ignite instance. - */ - void applyUpdates(Ignite ignite) { - for (Map.Entry entry : updates.entrySet()) { - IgniteCache cache = ignite.getOrCreateCache(entry.getKey()); - - cache.putAll(entry.getValue()); - } - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/UpdateStrategies.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/UpdateStrategies.java deleted file mode 100644 index 33ec96a5ecd45..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/UpdateStrategies.java +++ /dev/null @@ -1,47 +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.trainers.group; - -import org.apache.ignite.ml.optimization.SmoothParametrized; -import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate; -import org.apache.ignite.ml.optimization.updatecalculators.RPropUpdateCalculator; -import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDParameterUpdate; -import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDUpdateCalculator; - -/** - * Holder class for various update strategies. - */ -public class UpdateStrategies { - /** - * Simple GD update strategy. - * - * @return GD update strategy. - */ - public static UpdatesStrategy GD() { - return new UpdatesStrategy<>(new SimpleGDUpdateCalculator(), SimpleGDParameterUpdate::sumLocal, SimpleGDParameterUpdate::avg); - } - - /** - * RProp update strategy. - * - * @return RProp update strategy. - */ - public static UpdatesStrategy RProp() { - return new UpdatesStrategy<>(new RPropUpdateCalculator(), RPropParameterUpdate::sumLocal, RPropParameterUpdate::avg); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/Chains.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/Chains.java deleted file mode 100644 index db4f13f96c4d8..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/Chains.java +++ /dev/null @@ -1,56 +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.trainers.group.chain; - -import java.io.Serializable; - -/** - * Class containing methods creating {@link ComputationsChain}. - */ -public class Chains { - /** - * Create computation chain consisting of one returning its input as output. - * - * @param Type of local context of created chain. - * @param Type of keys of cache used in computation chain. - * @param Type of values of cache used in computation chain. - * @param Type of input to computation chain. - * @return Computation chain consisting of one returning its input as output. - */ - public static ComputationsChain create() { - return (input, context) -> input; - } - - /** - * Create {@link ComputationsChain} from {@link DistributedEntryProcessingStep}. - * - * @param step Distributed chain step. - * @param Type of local context of created chain. - * @param Type of keys of cache used in computation chain. - * @param Type of values of cache used in computation chain. - * @param Type of context used by worker in {@link DistributedEntryProcessingStep}. - * @param Type of input to computation chain. - * @param Type of output of computation chain. - * @return Computation created from {@link DistributedEntryProcessingStep}. - */ - public static ComputationsChain create( - DistributedEntryProcessingStep step) { - ComputationsChain chain = create(); - return chain.thenDistributedForEntries(step); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/ComputationsChain.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/ComputationsChain.java deleted file mode 100644 index 3c3bdab833863..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/ComputationsChain.java +++ /dev/null @@ -1,246 +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.trainers.group.chain; - -import java.io.Serializable; -import java.util.List; -import java.util.UUID; -import java.util.stream.Stream; -import javax.cache.processor.EntryProcessor; -import org.apache.ignite.Ignite; -import org.apache.ignite.cluster.ClusterGroup; -import org.apache.ignite.lang.IgniteBiPredicate; -import org.apache.ignite.ml.math.functions.Functions; -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.trainers.group.GroupTrainerCacheKey; -import org.apache.ignite.ml.trainers.group.GroupTrainerEntriesProcessorTask; -import org.apache.ignite.ml.trainers.group.GroupTrainerKeysProcessorTask; -import org.apache.ignite.ml.trainers.group.GroupTrainingContext; -import org.apache.ignite.ml.trainers.group.ResultAndUpdates; - -/** - * This class encapsulates convenient way for creating computations chain for distributed model training. - * Chain is meant in the sense that output of each non-final computation is fed as input to next computation. - * Chain is basically a bi-function from context and input to output, context is separated from input - * because input is specific to each individual step and context is something which is convenient to have access to in each of steps. - * Context is separated into two parts: local context and remote context. - * There are two kinds of computations: local and distributed. - * Local steps are just functions from two arguments: input and local context. - * Distributed steps are more sophisticated, but basically can be thought as functions of form - * localContext -> (function of remote context -> output), locally we fix local context and get function - * (function of remote context -> output) which is executed distributed. - * Chains are composable through 'then' method. - * - * @param Type of local context. - * @param Type of cache keys. - * @param Type of cache values. - * @param Type of input of this chain. - * @param Type of output of this chain. - * // TODO: IGNITE-7405 check if it is possible to integrate with {@link EntryProcessor}. - */ -@FunctionalInterface -public interface ComputationsChain { - /** - * Process given input and {@link GroupTrainingContext}. - * - * @param input Computation chain input. - * @param ctx {@link GroupTrainingContext}. - * @return Result of processing input and context. - */ - O process(I input, GroupTrainingContext ctx); - - /** - * Add a local step to this chain. - * - * @param locStep Local step. - * @param Output of local step. - * @return Composition of this chain and local step. - */ - default ComputationsChain thenLocally(IgniteBiFunction locStep) { - ComputationsChain nextStep = (input, context) -> locStep.apply(input, context.localContext()); - return then(nextStep); - } - - /** - * Add a distributed step which works in the following way: - * 1. apply local context and input to local context extractor and keys supplier to get corresponding suppliers; - * 2. on each node_n - * 2.1. get context object. - * 2.2. for each entry_i e located on node_n with key_i from keys stream compute worker((context, entry_i)) and get - * (cachesUpdates_i, result_i). - * 2.3. for all i on node_n merge cacheUpdates_i and apply them. - * 2.4. for all i on node_n, reduce result_i into result_n. - * 3. get all result_n, reduce them into result and return result. - * - * @param Type of worker output. - * @param Type of context used by worker. - * @param workerCtxExtractor Extractor of context for worker. - * @param worker Function computed on each entry of cache used for training. Second argument is context: - * common part of data which is independent from key. - * @param ks Function from chain input and local context to supplier of keys for worker. - * @param reducer Function used for reducing results of worker. - * @return Combination of this chain and distributed step specified by given parameters. - */ - default ComputationsChain thenDistributedForEntries( - IgniteBiFunction> workerCtxExtractor, - IgniteFunction, ResultAndUpdates> worker, - IgniteBiFunction>>> ks, - IgniteFunction, O1> reducer) { - ComputationsChain nextStep = (input, context) -> { - L locCtx = context.localContext(); - IgniteSupplier>> keysSupplier = ks.apply(input, locCtx); - - Ignite ignite = context.ignite(); - UUID trainingUUID = context.localContext().trainingUUID(); - String cacheName = context.cache().getName(); - ClusterGroup grp = ignite.cluster().forDataNodes(cacheName); - - // Apply first two arguments locally because it is common for all nodes. - IgniteSupplier extractor = Functions.curry(workerCtxExtractor).apply(input).apply(locCtx); - - return ignite.compute(grp).execute(new GroupTrainerEntriesProcessorTask<>(trainingUUID, extractor, worker, keysSupplier, reducer, cacheName, ignite), null); - }; - return then(nextStep); - } - - /** - * Add a distributed step which works in the following way: - * 1. apply local context and input to local context extractor and keys supplier to get corresponding suppliers; - * 2. on each node_n - * 2.1. get context object. - * 2.2. for each key_i from keys stream such that key_i located on node_n compute worker((context, entry_i)) and get - * (cachesUpdates_i, result_i). - * 2.3. for all i on node_n merge cacheUpdates_i and apply them. - * 2.4. for all i on node_n, reduce result_i into result_n. - * 3. get all result_n, reduce them into result and return result. - * - * @param Type of worker output. - * @param Type of context used by worker. - * @param workerCtxExtractor Extractor of context for worker. - * @param worker Function computed on each entry of cache used for training. Second argument is context: - * common part of data which is independent from key. - * @param keysSupplier Function from chain input and local context to supplier of keys for worker. - * @param reducer Function used for reducing results of worker. - * @return Combination of this chain and distributed step specified by given parameters. - */ - default ComputationsChain thenDistributedForKeys( - IgniteBiFunction> workerCtxExtractor, - IgniteFunction, ResultAndUpdates> worker, - IgniteBiFunction>>> keysSupplier, - IgniteFunction, O1> reducer) { - ComputationsChain nextStep = (input, context) -> { - L locCtx = context.localContext(); - IgniteSupplier>> ks = keysSupplier.apply(input, locCtx); - - Ignite ignite = context.ignite(); - UUID trainingUUID = context.localContext().trainingUUID(); - String cacheName = context.cache().getName(); - ClusterGroup grp = ignite.cluster().forDataNodes(cacheName); - - // Apply first argument locally because it is common for all nodes. - IgniteSupplier extractor = Functions.curry(workerCtxExtractor).apply(input).apply(locCtx); - - return ignite.compute(grp).execute(new GroupTrainerKeysProcessorTask<>(trainingUUID, extractor, worker, ks, reducer, cacheName, ignite), null); - }; - return then(nextStep); - } - - /** - * Add a distributed step specified by {@link DistributedEntryProcessingStep}. - * - * @param step Distributed step. - * @param Type of output of distributed step. - * @param Type of context of distributed step. - * @return Combination of this chain and distributed step specified by input. - */ - default ComputationsChain thenDistributedForEntries( - DistributedEntryProcessingStep step) { - return thenDistributedForEntries(step::remoteContextSupplier, step.worker(), step::keys, step.reducer()); - } - - /** - * Add a distributed step specified by {@link DistributedKeyProcessingStep}. - * - * @param step Distributed step. - * @param Type of output of distributed step. - * @param Type of context of distributed step. - * @return Combination of this chain and distributed step specified by input. - */ - default ComputationsChain thenDistributedForKeys( - DistributedKeyProcessingStep step) { - return thenDistributedForKeys(step::remoteContextSupplier, step.worker(), step::keys, step.reducer()); - } - - /** - * Version of 'thenDistributedForKeys' where worker does not depend on context. - * - * @param worker Worker. - * @param kf Function providing supplier - * @param reducer Function from chain input and local context to supplier of keys for worker. - * @param Type of worker output. - * @return Combination of this chain and distributed step specified by given parameters. - */ - default ComputationsChain thenDistributedForKeys( - IgniteFunction, ResultAndUpdates> worker, - IgniteBiFunction>>> kf, - IgniteFunction, O1> reducer) { - - return thenDistributedForKeys((o, lc) -> () -> o, (context) -> worker.apply(context.key()), kf, reducer); - } - - /** - * Combine this computation chain with other computation chain in the following way: - * 1. perform this calculations chain and get result r. - * 2. while 'cond(r)' is true, r = otherChain(r, context) - * 3. return r. - * - * @param cond Condition checking if 'while' loop should continue. - * @param otherChain Chain to be combined with this chain. - * @return Combination of this chain and otherChain. - */ - default ComputationsChain thenWhile(IgniteBiPredicate cond, - ComputationsChain otherChain) { - ComputationsChain me = this; - return (input, context) -> { - O res = me.process(input, context); - - while (cond.apply(res, context.localContext())) - res = otherChain.process(res, context); - - return res; - }; - } - - /** - * Combine this chain with other: feed this chain as input to other, pass same context as second argument to both chains - * process method. - * - * @param next Next chain. - * @param Type of next chain output. - * @return Combined chain. - */ - default ComputationsChain then(ComputationsChain next) { - ComputationsChain me = this; - return (input, context) -> { - O myRes = me.process(input, context); - return next.process(myRes, context); - }; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedEntryProcessingStep.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedEntryProcessingStep.java deleted file mode 100644 index 8fd126497e11f..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedEntryProcessingStep.java +++ /dev/null @@ -1,34 +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.trainers.group.chain; - -import java.io.Serializable; - -/** - * {@link DistributedStep} specialized to {@link EntryAndContext}. - * - * @param Local context. - * @param Type of keys of cache used for group training. - * @param Type of values of cache used for group training. - * @param Context used by worker. - * @param Type of input to this step. - * @param Type of output of this step. - */ -public interface DistributedEntryProcessingStep extends - DistributedStep, L, K, C, I, O> { -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedKeyProcessingStep.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedKeyProcessingStep.java deleted file mode 100644 index fb8d867f3ef9b..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedKeyProcessingStep.java +++ /dev/null @@ -1,33 +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.trainers.group.chain; - -import java.io.Serializable; - -/** - * {@link DistributedStep} specialized to {@link KeyAndContext}. - * - * @param Local context. - * @param Type of keys of cache used for group training. - * @param Context used by worker. - * @param Type of input to this step. - * @param Type of output of this step. - */ -public interface DistributedKeyProcessingStep extends - DistributedStep, L, K, C, I, O> { -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedStep.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedStep.java deleted file mode 100644 index 804a886298a99..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedStep.java +++ /dev/null @@ -1,70 +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.trainers.group.chain; - -import java.io.Serializable; -import java.util.List; -import java.util.stream.Stream; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.functions.IgniteSupplier; -import org.apache.ignite.ml.trainers.group.GroupTrainerCacheKey; -import org.apache.ignite.ml.trainers.group.ResultAndUpdates; - -/** - * Class encapsulating logic of distributed step in {@link ComputationsChain}. - * - * @param Type of elements to be processed by worker. - * @param Local context. - * @param Type of keys of cache used for group training. - * @param Context used by worker. - * @param Type of input to this step. - * @param Type of output of this step. - */ -public interface DistributedStep { - /** - * Create supplier of context used by worker. - * - * @param input Input. - * @param locCtx Local context. - * @return Context used by worker. - */ - IgniteSupplier remoteContextSupplier(I input, L locCtx); - - /** - * Get function applied to each cache element specified by keys. - * - * @return Function applied to each cache entry specified by keys.. - */ - IgniteFunction> worker(); - - /** - * Get supplier of keys for worker. - * - * @param input Input to this step. - * @param locCtx Local context. - * @return Keys for worker. - */ - IgniteSupplier>> keys(I input, L locCtx); - - /** - * Get function used to reduce results returned by worker. - * - * @return Function used to reduce results returned by worker.. - */ - IgniteFunction, O> reducer(); -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/EntryAndContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/EntryAndContext.java deleted file mode 100644 index 59c3b34f63107..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/EntryAndContext.java +++ /dev/null @@ -1,70 +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.trainers.group.chain; - -import java.util.Map; -import org.apache.ignite.ml.trainers.group.GroupTrainerCacheKey; - -/** - * Entry of cache used for group training and context. - * This class is used as input for workers of distributed steps of {@link ComputationsChain}. - * - * @param Type of cache keys used for training. - * @param Type of cache values used for training. - * @param Type of context. - */ -public class EntryAndContext { - /** - * Entry of cache used for training. - */ - private Map.Entry, V> entry; - - /** - * Context. - */ - private C ctx; - - /** - * Construct instance of this class. - * - * @param entry Entry of cache used for training. - * @param ctx Context. - */ - public EntryAndContext(Map.Entry, V> entry, C ctx) { - this.entry = entry; - this.ctx = ctx; - } - - /** - * Get entry of cache used for training. - * - * @return Entry of cache used for training. - */ - public Map.Entry, V> entry() { - return entry; - } - - /** - * Get context. - * - * @return Context. - */ - public C context() { - return ctx; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/HasTrainingUUID.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/HasTrainingUUID.java deleted file mode 100644 index d855adf52302e..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/HasTrainingUUID.java +++ /dev/null @@ -1,32 +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.trainers.group.chain; - -import java.util.UUID; - -/** - * Interface for classes which contain UUID of training. - */ -public interface HasTrainingUUID { - /** - * Get training UUID. - * - * @return Training UUID. - */ - UUID trainingUUID(); -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/KeyAndContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/KeyAndContext.java deleted file mode 100644 index ba36e0e542fd6..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/KeyAndContext.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.ml.trainers.group.chain; - -import org.apache.ignite.ml.trainers.group.GroupTrainerCacheKey; - -/** - * Class containing key and remote context (see explanation of remote context in {@link ComputationsChain}). - * - * @param Cache key type. - * @param Remote context. - */ -public class KeyAndContext { - /** - * Key of group trainer. - */ - private GroupTrainerCacheKey key; - - /** - * Remote context. - */ - private C ctx; - - /** - * Construct instance of this class. - * - * @param key Cache key. - * @param ctx Remote context. - */ - public KeyAndContext(GroupTrainerCacheKey key, C ctx) { - this.key = key; - this.ctx = ctx; - } - - /** - * Get group trainer cache key. - * - * @return Group trainer cache key. - */ - public GroupTrainerCacheKey key() { - return key; - } - - /** - * Get remote context. - * - * @return Remote context. - */ - public C context() { - return ctx; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/package-info.java deleted file mode 100644 index 46dcc6b9e84c5..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/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. - */ - -/** - * - * Contains classes related to computations chain. - */ -package org.apache.ignite.ml.trainers.group.chain; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/package-info.java deleted file mode 100644 index 9b7f7cd9c96ac..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/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. - */ - -/** - * - * Contains group trainers. - */ -package org.apache.ignite.ml.trainers.group; \ No newline at end of file diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java index 9900f854be9f4..0c3408e1697fd 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java @@ -23,11 +23,9 @@ import org.apache.ignite.ml.knn.KNNTestSuite; import org.apache.ignite.ml.math.MathImplMainTestSuite; import org.apache.ignite.ml.nn.MLPTestSuite; -import org.apache.ignite.ml.optimization.OptimizationTestSuite; import org.apache.ignite.ml.preprocessing.PreprocessingTestSuite; import org.apache.ignite.ml.regressions.RegressionsTestSuite; import org.apache.ignite.ml.svm.SVMTestSuite; -import org.apache.ignite.ml.trainers.group.TrainersGroupTestSuite; import org.apache.ignite.ml.tree.DecisionTreeTestSuite; import org.junit.runner.RunWith; import org.junit.runners.Suite; @@ -45,8 +43,6 @@ KNNTestSuite.class, LocalModelsTest.class, MLPTestSuite.class, - TrainersGroupTestSuite.class, - OptimizationTestSuite.class, DatasetTestSuite.class, PreprocessingTestSuite.class, GAGridTestSuite.class diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeapTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeapTest.java index ec9fdaaca5498..bdd1eeae8fd65 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeapTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeapTest.java @@ -22,7 +22,7 @@ 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.isolve.LinSysPartitionDataBuilderOnHeap; +import org.apache.ignite.ml.dataset.primitive.builder.data.SimpleLabeledDatasetDataBuilder; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -64,9 +64,9 @@ public void testSolveLinearSystem() { LSQROnHeap lsqr = new LSQROnHeap<>( datasetBuilder, - new LinSysPartitionDataBuilderOnHeap<>( + new SimpleLabeledDatasetDataBuilder<>( (k, v) -> Arrays.copyOf(v, v.length - 1), - (k, v) -> v[3] + (k, v) -> new double[]{v[3]} ) ); @@ -87,9 +87,9 @@ public void testSolveLinearSystemWithX0() { LSQROnHeap lsqr = new LSQROnHeap<>( datasetBuilder, - new LinSysPartitionDataBuilderOnHeap<>( + new SimpleLabeledDatasetDataBuilder<>( (k, v) -> Arrays.copyOf(v, v.length - 1), - (k, v) -> v[3] + (k, v) -> new double[]{v[3]} ) ); @@ -118,9 +118,9 @@ public void testSolveLeastSquares() throws Exception { try (LSQROnHeap lsqr = new LSQROnHeap<>( datasetBuilder, - new LinSysPartitionDataBuilderOnHeap<>( + new SimpleLabeledDatasetDataBuilder<>( (k, v) -> Arrays.copyOf(v, v.length - 1), - (k, v) -> v[4] + (k, v) -> new double[]{v[4]} ) )) { LSQRResult res = lsqr.solve(0, 1e-12, 1e-12, 1e8, -1, false, null); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerIntegrationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerIntegrationTest.java index 038b880f5cebc..654ebe0030e22 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerIntegrationTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerIntegrationTest.java @@ -31,7 +31,6 @@ import org.apache.ignite.ml.nn.architecture.MLPArchitecture; import org.apache.ignite.ml.optimization.LossFunctions; import org.apache.ignite.ml.optimization.updatecalculators.*; -import org.apache.ignite.ml.trainers.group.UpdatesStrategy; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import java.io.Serializable; 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 c53f6f104d84e..db1488139ff70 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 @@ -24,7 +24,6 @@ import org.apache.ignite.ml.nn.architecture.MLPArchitecture; import org.apache.ignite.ml.optimization.LossFunctions; import org.apache.ignite.ml.optimization.updatecalculators.*; -import org.apache.ignite.ml.trainers.group.UpdatesStrategy; import org.junit.Before; import org.junit.Test; import org.junit.experimental.runners.Enclosed; diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistIntegrationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistIntegrationTest.java index a64af9b5cc545..3b65a28b49329 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistIntegrationTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistIntegrationTest.java @@ -32,7 +32,7 @@ import org.apache.ignite.ml.optimization.LossFunctions; import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate; import org.apache.ignite.ml.optimization.updatecalculators.RPropUpdateCalculator; -import org.apache.ignite.ml.trainers.group.UpdatesStrategy; +import org.apache.ignite.ml.nn.UpdatesStrategy; import org.apache.ignite.ml.util.MnistUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistTest.java index d966484d548c6..406331234722f 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistTest.java @@ -27,7 +27,7 @@ import org.apache.ignite.ml.optimization.LossFunctions; import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate; import org.apache.ignite.ml.optimization.updatecalculators.RPropUpdateCalculator; -import org.apache.ignite.ml.trainers.group.UpdatesStrategy; +import org.apache.ignite.ml.nn.UpdatesStrategy; import org.apache.ignite.ml.util.MnistUtils; import org.junit.Test; diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/optimization/GradientDescentTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/optimization/GradientDescentTest.java deleted file mode 100644 index f6f4775be78eb..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/optimization/GradientDescentTest.java +++ /dev/null @@ -1,64 +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.optimization; - -import org.apache.ignite.ml.TestUtils; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; -import org.junit.Test; - -/** - * Tests for {@link GradientDescent}. - */ -public class GradientDescentTest { - /** */ - private static final double PRECISION = 1e-6; - - /** - * Test gradient descent optimization on function y = x^2 with gradient function 2 * x. - */ - @Test - public void testOptimize() { - GradientDescent gradientDescent = new GradientDescent( - (inputs, groundTruth, point) -> point.times(2), - new SimpleUpdater(0.01) - ); - - Vector res = gradientDescent.optimize(new DenseLocalOnHeapMatrix(new double[1][1]), - new DenseLocalOnHeapVector(new double[]{ 2.0 })); - - TestUtils.assertEquals(0, res.get(0), PRECISION); - } - - /** - * Test gradient descent optimization on function y = (x - 2)^2 with gradient function 2 * (x - 2). - */ - @Test - public void testOptimizeWithOffset() { - GradientDescent gradientDescent = new GradientDescent( - (inputs, groundTruth, point) -> point.minus(new DenseLocalOnHeapVector(new double[]{ 2.0 })).times(2.0), - new SimpleUpdater(0.01) - ); - - Vector res = gradientDescent.optimize(new DenseLocalOnHeapMatrix(new double[1][1]), - new DenseLocalOnHeapVector(new double[]{ 2.0 })); - - TestUtils.assertEquals(2, res.get(0), PRECISION); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/optimization/OptimizationTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/optimization/OptimizationTestSuite.java deleted file mode 100644 index 0ae6e4c70bbe4..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/optimization/OptimizationTestSuite.java +++ /dev/null @@ -1,33 +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.optimization; - -import org.apache.ignite.ml.optimization.util.SparseDistributedMatrixMapReducerTest; -import org.junit.runner.RunWith; -import org.junit.runners.Suite; - -/** - * Test suite for group trainer tests. - */ -@RunWith(Suite.class) -@Suite.SuiteClasses({ - GradientDescentTest.class, - SparseDistributedMatrixMapReducerTest.class -}) -public class OptimizationTestSuite { -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/optimization/util/SparseDistributedMatrixMapReducerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/optimization/util/SparseDistributedMatrixMapReducerTest.java deleted file mode 100644 index 9017c431808fb..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/optimization/util/SparseDistributedMatrixMapReducerTest.java +++ /dev/null @@ -1,135 +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.optimization.util; - -import org.apache.ignite.Ignite; -import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; - -/** - * Tests for {@link SparseDistributedMatrixMapReducer}. - */ -public class SparseDistributedMatrixMapReducerTest extends GridCommonAbstractTest { - /** Number of nodes in grid */ - private static final int NODE_COUNT = 2; - - /** */ - 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()); - } - - /** - * Tests that matrix 100x100 filled by "1.0" and distributed across nodes successfully processed (calculate sum of - * all elements) via {@link SparseDistributedMatrixMapReducer}. - */ - public void testMapReduce() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - SparseDistributedMatrix distributedMatrix = new SparseDistributedMatrix(100, 100); - for (int i = 0; i < 100; i++) - for (int j = 0; j < 100; j++) - distributedMatrix.set(i, j, 1); - SparseDistributedMatrixMapReducer mapReducer = new SparseDistributedMatrixMapReducer(distributedMatrix); - double total = mapReducer.mapReduce( - (matrix, args) -> { - double partialSum = 0.0; - for (int i = 0; i < matrix.rowSize(); i++) - for (int j = 0; j < matrix.columnSize(); j++) - partialSum += matrix.get(i, j); - return partialSum; - }, - sums -> { - double totalSum = 0; - for (Double partialSum : sums) - if (partialSum != null) - totalSum += partialSum; - return totalSum; - }, 0.0); - assertEquals(100.0 * 100.0, total, 1e-18); - } - - /** - * Tests that matrix 100x100 filled by "1.0" and distributed across nodes successfully processed via - * {@link SparseDistributedMatrixMapReducer} even when mapping function returns {@code null}. - */ - public void testMapReduceWithNullValues() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - SparseDistributedMatrix distributedMatrix = new SparseDistributedMatrix(100, 100); - for (int i = 0; i < 100; i++) - for (int j = 0; j < 100; j++) - distributedMatrix.set(i, j, 1); - SparseDistributedMatrixMapReducer mapReducer = new SparseDistributedMatrixMapReducer(distributedMatrix); - double total = mapReducer.mapReduce( - (matrix, args) -> null, - sums -> { - double totalSum = 0; - for (Double partialSum : sums) - if (partialSum != null) - totalSum += partialSum; - return totalSum; - }, 0.0); - assertEquals(0, total, 1e-18); - } - - /** - * Tests that matrix 1x100 filled by "1.0" and distributed across nodes successfully processed (calculate sum of - * all elements) via {@link SparseDistributedMatrixMapReducer} even when not all nodes contains data. - */ - public void testMapReduceWithOneEmptyNode() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - SparseDistributedMatrix distributedMatrix = new SparseDistributedMatrix(1, 100); - for (int j = 0; j < 100; j++) - distributedMatrix.set(0, j, 1); - SparseDistributedMatrixMapReducer mapReducer = new SparseDistributedMatrixMapReducer(distributedMatrix); - double total = mapReducer.mapReduce( - (matrix, args) -> { - double partialSum = 0.0; - for (int i = 0; i < matrix.rowSize(); i++) - for (int j = 0; j < matrix.columnSize(); j++) - partialSum += matrix.get(i, j); - return partialSum; - }, - sums -> { - double totalSum = 0; - for (Double partialSum : sums) - if (partialSum != null) - totalSum += partialSum; - return totalSum; - }, 0.0); - assertEquals(100.0, total, 1e-18); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java index b3c9368194366..5005ef25f3e48 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java @@ -27,9 +27,6 @@ @RunWith(Suite.class) @Suite.SuiteClasses({ LinearRegressionModelTest.class, - LocalLinearRegressionQRTrainerTest.class, - DistributedLinearRegressionQRTrainerTest.class, - BlockDistributedLinearRegressionQRTrainerTest.class, LinearRegressionLSQRTrainerTest.class, LinearRegressionSGDTrainerTest.class }) diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/ArtificialRegressionDatasets.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/ArtificialRegressionDatasets.java deleted file mode 100644 index ed6bf36417410..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/ArtificialRegressionDatasets.java +++ /dev/null @@ -1,404 +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.regressions.linear; - -/** - * Artificial regression datasets to be used in regression trainers tests. These datasets were generated by scikit-learn - * tools, {@code sklearn.datasets.make_regression} procedure. - */ -public class ArtificialRegressionDatasets { - /** - * Artificial dataset with 10 observations described by 1 feature. - */ - public static final TestDataset regression10x1 = new TestDataset(new double[][] { - {1.97657990214, 0.197725444973}, - {-5.0835948878, -0.279921224228}, - {-5.09032600779, -0.352291245969}, - {9.67660993007, 0.755464872441}, - {4.95927629958, 0.451981771462}, - {29.2635107429, 2.2277440173}, - {-18.3122588459, -1.25363275369}, - {-3.61729307199, -0.273362913982}, - {-7.19042139249, -0.473846634967}, - {3.68008403347, 0.353883097536} - }, new double[] {13.554054703}, -0.808655936776); - - /** - * Artificial dataset with 10 observations described by 5 features. - */ - public static final TestDataset regression10x5 = new TestDataset(new double[][] { - {118.635647237, 0.687593385888, -1.18956185502, -0.305420702986, 1.98794097418, -0.776629036361}, - {-18.2808432286, -0.165921853684, -0.156162539573, 1.56284391134, -0.198876782109, -0.0921618505605}, - {22.6110523992, 0.0268106268606, 0.702141470035, -0.41503615392, -1.09726502337, 1.30830482813}, - {209.820435262, 0.379809113402, -0.192097238579, -1.27460497119, 2.48052002019, -0.574430888865}, - {-253.750024054, -1.48044570917, -0.331747484523, 0.387993627712, 0.372583756237, -2.27404065923}, - {-24.6467766166, -0.66991474156, 0.269042238935, -0.271412703096, -0.561166818525, 1.37067541854}, - {-311.903650717, 0.268274438122, -1.10491275353, -1.06738703543, -2.24387799735, -0.207431467989}, - {74.2055323536, -0.329489531894, -0.493350762533, -0.644851462227, 0.661220945573, 1.65950140864}, - {57.0312289904, -1.07266578457, 0.80375035572, -0.45207210139, 1.69314420969, -1.10526080856}, - {12.149399645, 1.46504629281, -1.05843246079, 0.266225365277, -0.0113100353869, -0.983495425471} - }, new double[] {99.8393653561, 82.4948224094, 20.2087724072, 97.3306384162, 55.7502297387}, 3.98444039189); - - /** - * Artificial dataset with 100 observations described by 5 features. - */ - public static final TestDataset regression100x5 = new TestDataset(new double[][] { - {-44.2310642946, -0.0331360137605, -0.5290800706, -0.634340342338, -0.428433927151, 0.830582347183}, - {76.2539139721, -0.216200869652, 0.513212019048, -0.693404511747, 0.132995973133, 1.28470259833}, - {293.369799914, 2.90735870802, 0.457740818846, -0.490470696097, -0.442343455187, 0.584038258781}, - {124.258807314, 1.64158129148, 0.0616936820145, 1.24082841519, -1.20126518593, -0.542298907742}, - {13.6610807249, -1.10834821778, 0.545508208111, 1.81361288715, -0.786543112444, 0.250772626496}, - {101.924582305, -0.433526394969, 0.257594734335, 1.22333193911, 0.76626554927, -0.0400734567005}, - {25.5963186303, -0.202003301507, 0.717101151637, -0.486881225605, 1.15215024807, -0.921615554612}, - {75.7959681263, -0.604173187402, 0.0364386836472, 1.67544714536, 0.394743148877, 0.0237966550759}, - {-97.539357166, -0.774517689169, -0.0966902473883, -0.152250704254, -0.325472625458, 0.0720711851256}, - {0.394748999236, -0.559303402754, -0.0493339259273, -1.10840277768, -0.0800969523557, 1.80939282066}, - {-62.0138166431, 0.062614716778, -0.844143618016, 0.55269949861, -2.32580899335, 1.58020577369}, - {584.427692931, 2.13184767906, 1.22222461994, 1.71894070494, 2.69512281718, 0.294123497874}, - {-59.8323709765, 1.00006112818, -1.54481230765, -0.781282316493, 0.0255925284853, -0.0821173744608}, - {101.565711925, -0.38699836725, 1.06934591441, -0.260429311097, 1.02628949564, 0.0431473245174}, - {-141.592607814, 0.993279116267, -0.371768203378, -0.851483217286, -1.96241293548, -0.612279404296}, - {34.8038723379, -0.0182719243972, 0.306367604506, -0.650526589206, 1.30693112283, -0.587465952557}, - {-16.9554534069, -0.703006786668, -0.770718401931, 0.748423272307, 0.502544067819, 0.346625621533}, - {-76.2896177709, -0.16440174812, -1.77431555198, 0.195326723837, 2.01240994405, -1.19559207119}, - {-3.23827624818, -0.674138419631, -1.62238580284, 2.02235607862, 0.679194838679, 0.150203732584}, - {-21.962456854, -0.766271014206, 0.958599712131, -0.313045794728, 0.232655576106, -0.360950549871}, - {349.583669646, 1.75976166947, 1.47271612346, 0.0346005603489, 0.474907228495, 0.61379496381}, - {-418.397356757, -1.83395936566, -0.911702678716, -0.532478094882, -2.03835348133, -0.423005552518}, - {55.0298153952, -0.0301384716096, -0.0137929430966, -0.348583692759, 0.986486580719, 0.154436524434}, - {127.150063206, 1.92682560465, -0.434844790414, 0.1082898967, -0.00723338222402, -0.513199251824}, - {89.6172507626, 1.02463790902, 0.744369837717, 1.250323683, -1.58252612128, -0.588242778808}, - {92.5124829355, -0.403298547743, 0.0422774545428, -0.175000467434, 1.61110066857, 0.422330077287}, - {-303.040366788, 0.611569308879, -1.21926246291, -2.49250330276, -0.789166929605, -1.30166501196}, - {-17.4020602839, 1.72337202371, -1.83540537288, 0.731588761841, -0.338642535062, -1.11053518125}, - {114.918701324, 0.437385758628, 0.975885170381, 0.439444038872, 1.51666514156, -1.93095020264}, - {-8.43548064928, -0.799507968686, -0.00842968328782, -0.154994093964, 1.09169753491, -0.0114818657732}, - {109.209286025, 2.56472965015, -2.07047248035, -0.46764001177, 0.845267147375, -0.236767841427}, - {61.5259982971, -0.379391870148, -0.131017762354, -0.220275015864, 1.82097825699, -0.0568354876403}, - {-71.3872099588, 0.642138455414, -1.00242489879, 0.536780074488, 0.350977275771, -1.8204862883}, - {-21.2768078629, -0.454268998895, 0.0992324274219, 0.0363496803224, 0.281940751723, -0.198435570828}, - {-8.07838891387, -0.331642089041, -0.494067341253, 0.386035842816, -0.738221128298, 1.18236299649}, - {30.4818041751, 0.099206096537, 0.150688905006, 0.332932621949, 0.194845631964, -0.446717875795}, - {237.209150991, 1.12560447042, 0.448488431264, -0.724623711259, 0.401868257097, 1.67129001163}, - {185.172816475, 0.36594142556, -0.0796476435741, 0.473836257, 1.30890722633, 0.592415068693}, - {19.8830237044, 1.52497319332, 0.466906090264, -0.716635613964, -1.19532276745, -0.697663531684}, - {209.396793626, 0.368478789658, 0.699162303982, 1.96702434462, -0.815379139879, 0.863369634396}, - {-215.100514168, -1.83902416164, -1.14966820385, -1.01044860587, 1.76881340629, -0.32165916241}, - {-33.4687353426, -0.0451102002703, 0.642212950033, 0.580822065219, -1.02341504063, -0.781229325942}, - {150.251474823, 0.220170650298, 0.224858901011, 0.541299425328, 1.15151550963, 0.0329044069571}, - {92.2160506097, 1.86450932451, -0.991150940533, -1.49137866968, 1.02113774105, 0.0544762857136}, - {41.2138467595, -0.778892265105, 0.714957464344, 1.79833618993, -0.335322825621, -0.397548301803}, - {13.151262759, 0.301745607362, 0.129778280739, 0.260094818273, -0.10587841585, -0.599330307629}, - {-367.864703951, -1.68695981263, -0.611957677512, -0.0362971579679, -1.2169760515, -1.43224375134}, - {-57.218869838, 0.428806849751, 0.654302177028, -1.31651788496, 0.363857431276, -1.49953703016}, - {53.0877462955, -0.411907760185, -0.192634094071, -0.275879375023, 0.603562526571, 1.16508196734}, - {-8.11860742896, 1.00263982158, -0.157031169267, -1.11795623393, 0.35711440521, -0.851124640982}, - {-49.1878248403, -0.0253797866589, -0.574767070714, 0.200339045636, -0.0107042446803, -0.351288977927}, - {-73.8835407053, -2.07980276724, 1.12235566491, -0.917150593536, 0.741384768556, 0.56229424235}, - {143.163604045, 0.33627769945, 1.07948757447, 0.894869929963, 1.18688316974, -1.54722487849}, - {92.7045830908, 0.944091525689, 0.693296229491, 0.700097596814, -1.23666276942, -0.203890113084}, - {79.1878852355, -0.221973023853, -0.566066329011, 1.57683748648, 0.52854717911, 0.147924782476}, - {30.6547392801, -1.03466213359, 0.606784904328, -0.298096511956, 0.83332987683, 0.636339018254}, - {-329.128386019, -1.41363866598, -1.34966434823, -0.989010564149, 0.46889477248, -1.20493210784}, - {121.190205512, 0.0393914245697, 1.98392444232, -0.65310705226, -0.385899987099, 0.444982471471}, - {-97.0333075649, 0.264325871992, -0.43074811924, -1.14737761316, -0.453134140655, -0.038507405311}, - {158.273624516, 0.302255432981, -0.292046617818, 1.0704087606, 0.815965268115, 0.470631083546}, - {8.24795061818, -1.15155524496, 1.29538707184, -0.4650881541, 0.805123486308, -0.134706887329}, - {87.1140049059, -0.103540823781, -0.192259440773, 1.79648860085, -1.07525447993, 1.06985127941}, - {-25.1300772481, -0.97140742052, 0.033393948794, -0.698311192672, 0.74417168942, 0.752776770225}, - {-285.477057638, -0.480612406803, -1.46081500036, -1.92518386336, -0.426454066275, -0.0539099489597}, - {-65.1269988498, -1.22733468764, 0.121538452336, 0.752958777557, -0.40643211762, 0.257674949803}, - {-17.1813504942, 0.823753836891, 0.445142465255, 0.185644700144, -1.99733367514, -0.247899323048}, - {-46.7543447303, 0.183482778928, -0.934858705943, -1.21961947396, 0.460921844744, 0.571388077177}, - {-1.7536190499, -0.107517908181, 0.0334282610968, -0.556676121428, -0.485957577159, 0.943570398164}, - {-42.8460452689, 0.944999215632, 0.00530052154909, -0.348526283976, -1.724125354, -0.122649339813}, - {62.6291497267, 0.249619894002, 1.3139125969, -1.5644227783, 0.117605482783, 0.304844650662}, - {97.4552176343, 1.59332799639, -1.17868305562, 1.02998378902, -0.31959491258, -0.183038322076}, - {-6.19358885758, 0.437951016253, 0.373339269494, -0.204072768495, 0.477969349931, -1.52176449389}, - {34.0350630099, 0.839319087287, -0.610157662489, 1.73881448393, -1.89200107709, 0.204946415522}, - {54.9790822536, -0.191792583114, 0.989791127554, -0.502154080064, 0.469939512389, -0.102304071079}, - {58.8272402843, 0.0769623906454, 0.501297284297, -0.410054999243, 0.595712387781, -0.0968329050729}, - {95.3620983209, 0.0661481959314, 0.0935137309086, 1.11823292347, -0.612960777903, 0.767865072757}, - {62.4278196648, 0.78350610065, -1.09977017652, 0.526824784479, 1.41310104196, -0.887902707319}, - {57.6298676729, 0.60084172954, -0.785932027202, 0.0271301584637, -0.134109499719, 0.877256170191}, - {5.14112905382, -0.738359365006, 1.40242539359, -0.852833010305, -0.68365080837, 0.88561193696}, - {11.6057244034, -0.958911227571, 1.15715937023, 1.20108425431, 0.882980929338, -1.77404120156}, - {-265.758185272, -1.2092434823, -0.0550151798639, 0.00703735243613, -1.01767244359, -1.40616581707}, - {180.625928828, -0.139091127126, 0.243250756129, 2.17509702585, -0.541735827898, 1.2109459934}, - {-183.604103216, -0.324555097769, -1.71317286749, 1.03645005723, 0.497569347608, -1.96688185911}, - {9.93237328848, 0.825483591345, 0.910287997312, -1.64938108528, 0.98964075968, -1.65748940528}, - {-88.6846949813, -0.0759295112746, -0.593311990101, -0.578711915019, 0.256298822361, -0.429322890198}, - {175.367391479, 0.9361754906, -0.0172852897292, 1.04078658833, 0.919566407184, -0.554923019093}, - {-175.538247146, -1.43498590417, 0.37233438556, -0.897205352198, -0.339309952316, -0.0321624527843}, - {-126.331680318, 0.160446617623, 0.816642363249, -1.39863371652, 0.199747744327, -2.13493607457}, - {116.677107593, 1.19300905847, -0.404409346893, 0.646338976096, -0.534204093869, 0.36692724765}, - {-181.675962893, -1.57613169533, -0.41549571451, -0.956673746013, 0.35723782515, 0.318317395128}, - {-55.1457877823, 0.63723030991, -0.324480386466, 0.296028333894, -1.68117515658, -0.131945601375}, - {25.2534791013, 0.594818219911, -0.0247380403547, -0.101492246071, -0.0745619242015, -0.370837128867}, - {63.6006283756, -1.53493473818, 0.946464097439, 0.637741397831, 0.938866921166, 0.54405291856}, - {-69.6245547661, 0.328482934094, -0.776881060846, -0.285133098443, -1.06107824512, 0.49952182341}, - {233.425957233, 3.10582399189, -0.0854710508706, 0.455873479133, -0.0974589364949, -1.18914783551}, - {-86.5564290626, -0.819839276484, 0.584745927593, -0.544737106102, -1.21927675581, 0.758502626434}, - {425.357285631, 1.70712253847, 1.19892647853, 1.60619661301, 0.36832665241, 0.880791322709}, - {111.797225426, 0.558940594145, -0.746492420236, 1.90172101792, 0.853590062366, -0.867970723941}, - {-253.616801014, -0.426513440051, 0.0388582291888, -1.18576061365, -2.70895868242, 0.26982210287}, - {-394.801501024, -1.65087241498, 0.735525201393, -2.02413077052, -0.96492749037, -1.89014065613} - }, new double[] {93.3843533037, 72.3610889215, 57.5295295915, 63.7287541653, 65.2263084024}, 6.85683020686); - - /** - * Artificial dataset with 100 observations described by 10 features. - */ - public static final TestDataset regression100x10 = new TestDataset(new double[][] { - {69.5794204114, -0.684238565877, 0.175665643732, 0.882115894035, 0.612844187624, - -0.685301720572, -0.8266500007, -0.0383407025118, 1.7105205222, 0.457436379836, -0.291563926494}, - {80.1390102826, -1.80708821811, 0.811271788195, 0.30248512861, 0.910658009566, - -1.61869762501, -0.148325085362, -0.0714164596509, 0.671646742271, 2.15160094956, -0.0495754979721}, - {-156.975447515, 0.170702943934, -0.973403372054, -0.093974528453, 1.54577255871, - -0.0969022857972, -1.10639617368, 1.51752480948, -2.86016865032, 1.24063030602, -0.521785751026}, - {-158.134931891, 0.0890071395055, -0.0811824442353, -0.737354274843, -1.7575255492, - 0.265777246641, 0.0745347238144, -0.457603542683, -1.37034043839, 1.86011799875, 0.651214189491}, - {-131.465820263, 0.0767565260375, 0.651724194978, 0.142113799753, 0.244367469855, - -0.334395162837, -0.069092305876, -0.691806779713, -1.28386786177, -1.43647491141, 0.00721053414234}, - {-125.468890054, 0.43361925912, -0.800231440065, -0.576001094593, 0.0783664516431, - -1.33613252233, -0.968385062126, -1.22077801286, 0.193456109638, -3.09372314386, 0.817979620215}, - {-44.1113403874, -0.595796803171, 1.29482131972, -0.784513985654, 0.364702038003, - -3.2452492093, -0.451605560847, 0.988546607514, 0.492096628873, -0.343018842342, -0.519231306954}, - {61.2269707872, -0.0289059337716, -1.00409238976, 0.329908621635, 1.41965097539, - 0.0395065997587, -0.477939549336, 0.842336765911, -0.808790019648, 1.70241718768, -0.117194118865}, - {301.434286126, 0.430005308515, 1.01290089725, -0.228221561554, 0.463405921629, - -0.602413489517, 1.13832440088, 0.930949226185, -0.196440161506, 1.46304624346, 1.23831509056}, - {-270.454814681, -1.43805412632, -0.256309572507, -0.358047601174, 0.265151660237, - 1.07087986377, -1.93784654681, -0.854440691754, 0.665691996289, -1.87508012738, -0.387092423365}, - {-97.6198688184, -1.67658167161, -0.170246709551, -2.26863722189, 0.280289356338, - -0.690038347855, -1.69282684019, 0.978606053022, 1.28237852256, -1.2941998486, 0.766405365374}, - {-29.5630902399, -1.75615633921, 0.633927486329, -1.24117311555, -0.15884687004, - 0.31296863712, -1.29513272039, 0.344090683606, 1.19598425093, -1.96195019104, 1.81415061059}, - {-130.896377427, 0.577719366939, -0.087267771748, -0.060088767013, 0.469803880788, - -1.03078212088, -1.41547398887, 1.38980586981, -0.37118000595, -1.81689513712, -0.3099432567}, - {79.6300698059, 1.23408625633, 1.06464588017, 1.23403332691, -1.10993859098, - 0.874825200577, 0.589337796957, -1.10266185141, 0.842960469618, -0.89231962021, 0.284074900504}, - {-154.712112815, -1.64474237898, -0.328581696933, 0.38834343178, 0.02682160335, - -0.251167527796, -0.199330632103, -0.0405837345525, -0.908200250794, -1.3283756975, 0.540894408264}, - {233.447381562, 0.395156450609, 0.156412599781, 0.126453148554, 2.40829068933, - 1.01623530754, -0.0856520211145, -0.874970377099, 0.280617145254, -0.307070438514, 0.4599616054}, - {209.012380432, -0.848646647675, 0.558383548084, -0.259628264419, 1.1624126549, - -0.0755949979572, -0.373930759448, 0.985903312667, 0.435839508011, -0.760916312668, 1.89847574116}, - {-39.8987262091, 0.176656582642, 0.508538223618, 0.995038391204, -2.08809409812, - 0.743926580134, 0.246007971514, -0.458288599906, -0.579976479473, 0.0591577146017, 1.64321662761}, - {222.078510236, -0.24031989218, -0.168104260522, -0.727838425954, 0.557181757624, - -0.164906646307, 2.01559331734, 0.897263594222, 0.0921535309562, 0.351910490325, -0.018228500121}, - {-250.916272061, -2.71504637339, 0.498966191294, -3.16410707344, -0.842488891776, - 1.27425275951, 0.0141733666756, 0.695942743199, 0.0917995810179, -0.501447196978, -0.355738068451}, - {134.07259088, 0.0845637591619, 0.237410106679, -0.291458113729, 1.39418566986, - -1.18813057956, -0.683117067763, -0.518910379335, 1.35998426879, -1.28404562245, 0.489131754943}, - {104.988440209, 0.00770925058526, 0.47113239214, -0.606231247854, 0.310679840217, - 0.146297599928, 0.732013998647, -0.284544010865, 0.402622530153, -0.0217367745613, 0.0742970687987}, - {155.558071031, 1.11171654653, 0.726629222799, -0.195820863177, 0.801333855535, - 0.744034755544, 1.11377275513, -0.75673532139, -0.114117607244, -0.158966474923, -0.29701120385}, - {90.7600194013, -0.104364079622, -0.0165109945217, 0.933002972987, -1.80652594466, - -1.34760892883, -0.304511906801, 0.0584734540581, 1.5332169392, 0.478835797824, 1.71534051065}, - {-313.910553214, 0.149908925551, 0.232806828559, -0.0708920471592, -0.0649553559745, - 0.377753357707, -0.957292311668, 0.545360522582, -1.37905464371, -0.940702110994, -1.53620430047}, - {-80.9380113754, 0.135586606896, 0.95759558815, -1.36879020479, 0.735413996144, - 0.637984100201, -1.79563152885, 1.55025691631, 0.634702068786, -0.203690334141, -0.83954824721}, - {-244.336816695, -0.179127343947, -2.12396005014, -0.431179356484, -0.860562153749, - -1.10270688639, -0.986886012982, -0.945091656162, -0.445428453767, 1.32269756209, -0.223712672168}, - {123.069612745, 0.703857129626, 0.291605144784, 1.40233051946, 0.278603787802, - -0.693567967466, -0.15587953395, 2.10213915684, 0.130663329174, -0.393184478882, 0.0874812844555}, - {-148.274944223, 1.66294967732, 0.0830002694123, 0.32492930502, 1.11864359687, - -0.381901627785, -1.06367037132, -0.392583620174, -1.16283326187, 0.104931461025, -1.64719611405}, - {-82.0018788235, 0.497118817453, 0.731125358012, -0.00976413646786, -0.0178930713492, - -0.814978582886, 0.0602834712523, -0.661940479055, -0.957902899386, -1.34489251111, 0.22166518707}, - {-35.742996986, 0.0661349516701, -0.204314495629, 1.17101314753, -2.53846825562, - -0.560282479298, -0.393442894828, 0.988953809491, -0.911281277704, 0.86862242698, 2.59576940486}, - {-109.588885664, -0.0793151346628, -0.408962434518, -0.598817776528, 0.0277205469561, - 0.116291018958, 0.0280416838086, -0.72544170676, -0.669302814774, 0.0751898759816, -0.311002356179}, - {57.8285173441, 0.53753903532, 0.676340503752, -2.10608342721, 0.477714987751, - 0.465695114442, 0.245966562421, -1.05230350808, -0.309794163113, -1.12067331828, 1.07841453304}, - {204.660622582, -0.717565166685, 0.295179660279, -0.377579912697, 1.88425526905, - 0.251875238436, -0.900214103232, -1.02877401105, 0.291693915093, 1.24889067987, 1.78506220081}, - {350.949109103, 2.82276814452, -0.429358342127, 1.12140362367, 1.18120725208, - -1.63913834939, 1.61441562446, -0.364003766916, -0.258752942225, -0.808124680189, 0.556463488303}, - {170.960252153, 0.147245922081, 0.3257117575, 0.211749283649, -0.0150701808404, - -0.888523132148, 0.777862088798, 0.296729270892, -0.332927550718, 0.888968144245, 1.20913118467}, - {112.192270383, 0.129846138824, -0.934371449036, -0.595825303214, 1.74749214629, - -0.0500069421443, -0.161976298602, -2.54100791613, 1.99632530735, -0.0691582773758, -0.863939367415}, - {-56.7847711121, 0.0950532853751, -0.467349228201, -0.26457152362, -0.422134692317, - -0.0734763062127, 0.90128235602, -1.68470856275, -0.0699692697335, -0.463335845504, -0.301754321169}, - {-37.9223252258, -1.40835827778, 0.566142056244, -3.22393318933, 0.228823495106, - -1.8480727782, 0.129468321643, -1.77392686536, 0.0112549619662, 0.146433267822, 1.29379901303}, - {-59.7303066136, 0.835675535576, -0.552173157548, 1.90730898966, -0.520145317195, - 1.55174485912, -1.37531768692, -0.408165743742, 0.0939675842223, 0.318004128812, 0.324378038446}, - {-0.916090786983, 0.425763794043, -0.295541268984, -0.066619586336, 2.03494974978, - -0.197109278058, -0.823307883209, 0.895531446352, -0.276435938737, -1.54580056755, -0.820051830246}, - {-20.3601082842, 0.56420556369, 0.741234589387, -0.565853617392, -0.311399905686, - 2.24066463251, -0.071704904286, -1.22796531596, 0.186020404046, -0.786874824874, 0.23140277151}, - {-22.9342855182, -0.0682789648279, -1.30680909143, 0.0486490588348, 0.890275695028, - -0.257961411112, -0.381531755985, 1.56251482581, -2.11808219232, 0.741828675202, 0.696388901165}, - {-157.251026807, -2.3120966502, 0.183734662375, 1.02192264962, 0.591272941061, - -0.0132855098339, -1.02016546348, 1.19642432892, 0.867653154846, -1.37600041722, -1.08542822792}, - {-68.6110752055, -1.2429968179, -0.950064269349, -0.332379873336, 0.25793632341, - 0.145780713577, -0.512109283074, -0.477887632032, 0.448960776324, -0.190215737958, 0.219578347563}, - {-56.1204152481, -0.811729480846, -0.647410362207, 0.934547463984, -0.390943346216, - -0.409981308474, 0.0923465893049, 1.9281242912, -0.624713581674, -0.0599353282306, -0.0188591746808}, - {348.530651658, 2.51721790231, 0.7560998114, -2.69620396681, 0.5174276585, - 0.403570816695, 0.901648571306, 0.269313230294, 1.07811463589, 0.986649559679, 0.514710327657}, - {-105.719065924, 0.679016972998, 0.341319363316, -0.515209647377, 0.800000866847, - -0.795474442628, -0.866849274801, -1.32927961486, 0.17679343917, -1.93744422464, -0.476447619273}, - {-197.389429553, -1.98585668879, -0.962610549884, -2.48860863254, -0.545990524642, - -0.13005685654, -1.23413782366, 1.17443427507, 1.4785554038, -0.193717671824, -0.466403609229}, - {-23.9625285402, -0.392164367603, 1.07583388583, -0.412686712477, -0.89339030785, - -0.774862334739, -0.186491999529, -0.300162444329, 0.177377235999, 0.134038296039, 0.957945226616}, - {-91.145725943, -0.154640540119, 0.732911957939, -0.206326119636, -0.569816760116, - 0.249393336416, -1.02762332953, 0.25096708081, 0.386927162941, -0.346382299592, 0.243099162109}, - {-80.7295722208, -1.72670707303, 0.138139045677, 0.0648055728598, 0.186182854422, - 1.07226527747, -1.26133459043, 0.213883744163, 1.47115466163, -1.54791582859, 0.170924664865}, - {-317.060323531, -0.349785690206, -0.740759426066, -0.407970845617, -0.689282767277, - -1.25608665316, -0.772546119412, -2.02925712813, 0.132949072522, -0.191465137244, -1.29079690284}, - {-252.491508279, -1.24643122869, 1.55335609203, 0.356613424877, 0.817434495353, - -1.74503747683, -0.818046363088, -1.58284235058, 0.357919389759, -1.18942962791, -1.91728745247}, - {-66.8121363157, -0.584246455697, -0.104254351782, 1.17911687508, -0.29288167882, - 0.891836132692, 0.232853863255, 0.423294355343, -0.669493690103, -1.15783890498, 0.188213983735}, - {140.681464689, 1.33156046873, -1.8847915949, -0.666528837988, -0.513356191443, - 0.281290031669, -1.07815005006, 1.22384196227, 1.39093631269, 0.527644817197, 1.21595221509}, - {-174.22326767, 0.475428766034, 0.856847216768, -0.734282773151, -0.923514989791, - 0.917510828772, 0.674878068543, 0.0644776431114, -0.607796192908, 0.867740011912, -1.97799769281}, - {74.3899799579, 0.00915743526294, 0.553578683413, 1.66930486354, 0.15562803404, - 1.8455840688, -0.371704942927, 1.11228894843, -0.37464389118, -0.48789151589, 0.79553866342}, - {70.1167175897, 0.154877045187, 1.47803572976, -0.0355743163524, -2.47914644675, - 0.672384381837, 1.63160379529, 1.81874583854, 1.22797339421, -0.0131258061634, -0.390265963676}, - {-11.0364788877, 0.173049156249, -1.78140521797, -1.29982707214, -0.48025663179, - -0.469112922302, -1.98718063269, 0.585086542043, 0.264611327837, 1.48855512579, 2.00672263496}, - {-112.711292736, -1.59239636827, -0.600613018822, -0.0209667499746, -1.81872893331, - -0.739893084955, 0.140261888569, -0.498107678308, 2.53664045504, -0.536385019089, -0.608755809378}, - {-198.064468217, 0.737175509877, -2.01835515547, -2.18045950065, 0.428584922529, - -1.01848835019, -0.470645361539, -0.00703630153547, -2.2341302754, 1.51483167022, -0.410184418418}, - {70.2747963991, 1.49474111532, -0.19517712503, 0.7392852909, -0.326060871666, - -0.566710349675, 0.14053094122, -0.562830341306, 0.22931613446, -0.0344439061448, 0.175150510551}, - {207.909021337, 0.839887009159, 0.268826583246, -0.313047158862, 1.12009996015, - 0.214209976971, -0.396147338251, 2.16039704403, 0.699141312749, 0.756192350992, -0.145368196901}, - {169.428609429, -1.13702350819, 1.23964530597, -0.864443556622, -0.885630795949, - -0.523872327352, 0.467159824748, 0.476596383923, 0.4343735578, 1.4075417896, 2.22939328991}, - {-176.909833405, 0.0875512760866, -0.455542269288, 0.539742307764, -0.762003092788, - 0.41829123457, -0.818116139644, -2.01761645956, 0.557395073218, 1.5823271814, -1.0168826293}, - {-27.734298611, -0.841257541979, 0.348961259301, 1.36935991472, -0.0694528057586, - -1.27303784913, 0.152155656569, 1.9279466651, 0.9589415766, -1.76634370106, -1.08831026428}, - {-55.8416853588, 0.927711536927, 0.157856746063, -0.295628714893, 0.0296602829783, - 1.75198587897, -0.38285446366, -0.253287154535, -1.64032395229, -0.842089054965, 1.00493779183}, - {56.0899797005, 0.326117761734, -1.93514762146, 1.0229172721, 0.125568968732, - 2.37760000658, -0.498532972011, -0.733375842271, -0.757445726993, -0.49515057432, 2.01559891524}, - {-176.220234909, 1.571129843, -0.867707605929, -0.709690799512, -1.51535538937, - 1.27424225477, -0.109513704468, -1.46822183, 0.281077088939, -1.97084024232, -0.322309524179}, - {37.7155152941, 0.363383774219, -0.0240881298641, -1.60692745228, -1.26961656439, - -0.41299134216, 1.2890099968, -1.34101694629, -0.455387485256, -0.14055003482, 1.5407059956}, - {-102.163416997, -2.05927378316, -0.470182865756, -0.875528863204, 0.0361720859253, - -1.03713912263, 0.417362606334, 0.707587625276, -0.0591627772581, -2.58905252006, 0.516573345216}, - {-206.47095321, 0.270030584651, 1.85544202116, -0.144189208964, -0.696400687327, - 0.0226388634283, -0.490952489106, -1.69209527849, 0.00973614309272, -0.484105876992, -0.991474668217}, - {201.50637416, 0.513659215697, -0.335630132208, -0.140006500483, 0.149679720127, - -1.89526167503, -0.0614973894156, 0.0813221153552, 0.630952530848, 2.40201011339, 0.997708264073}, - {-72.0667371571, 0.0841570292899, -0.216125859013, -1.77155215764, 2.15081767322, - 0.00953341785443, -1.0826077946, -0.791135571106, -0.989393577892, -0.791485083644, -0.063560999686}, - {-162.903837815, -0.273764637097, 0.282387854873, -1.39881596931, 0.554941097854, - -0.88790718926, -0.693189960902, 0.398762630571, -1.61878562893, -0.345976341096, 0.138298909959}, - {-34.3291926715, -0.499883755911, -0.847296893019, -0.323673126437, 0.531205373462, - -0.0204345595983, 0.284954510306, 0.565031773028, -0.272049818708, -0.130369799738, -0.617572026201}, - {76.1272883187, -0.908810282403, -1.04139421904, 0.890678872055, 1.32990256154, - -0.0150445428835, 0.593918101047, 0.356897732999, 0.824651162423, -1.54544256217, -0.795703905296}, - {171.833705285, -0.0425219657568, -0.884042952325, 1.91202504537, 0.381908223898, - -0.205693527739, 1.53656598237, 0.534880398015, 0.291950716831, -1.1258051056, -0.0612803476297}, - {-235.445792009, 0.261252102941, -0.170931758001, 1.67878144235, 0.0278283741792, - -1.23194408479, -0.190931886594, 1.0000157972, -2.18792142659, -0.230654984288, -1.36626493512}, - {348.968834231, 1.35713154434, 0.950377770072, 0.0700577471848, 0.96907140156, - 2.00890422081, 0.0896405239806, 0.614309607351, 1.07723409067, 2.58506968136, 0.202889806148}, - {-61.0128039201, 0.465438505031, -1.31448530533, 0.374781933416, -0.0118298606041, - -0.477338357738, -0.587656108109, 1.66449545077, 0.435836048385, -0.287027953004, -1.06613472784}, - {-50.687090469, 0.382331825989, -0.597140322197, 1.1276065465, -1.35593777887, - 1.14949964423, -0.858742432885, -0.563211485633, -0.57167161928, 0.0294891749132, 1.9571639493}, - {-186.653649045, -0.00981380006029, 1.0371088941, -1.25319048981, -0.694043021068, - 1.7280802541, -0.191210409232, -0.866039238001, -0.0791927416078, -0.232228656558, -0.93723545053}, - {34.5395591744, 0.680943971029, -0.075875481801, -0.144408300848, -0.869070791528, - 0.496870904214, 1.0940401388, -0.510489750436, -0.47562728601, 0.951406841944, 0.12983846382}, - {-23.7618645627, 0.527032820313, -0.58295129357, -0.3894567306, -0.0547905472556, - -1.86103603537, 0.0506988360667, 1.02778539291, -0.0613720063422, 0.411280841442, -0.665810811374}, - {116.007776415, 0.441750249008, 0.549342185228, 0.731558201455, -0.903624700864, - -2.13208328824, 0.381223328983, 0.283479210749, 1.17705098922, -2.38800904207, 1.32108350152}, - {-148.479593311, -0.814604260049, -0.821204361946, -1.08768677334, -0.0659445766599, - 0.583741297405, 0.669345853296, -0.0935352010726, -0.254906787938, -0.394599725657, -1.26305927257}, - {244.865845084, 0.776784257443, 0.267205388558, 2.37746488031, -0.379275360853, - -0.157454754411, -0.359580726073, 0.886887721861, 1.53707627973, 0.634390546684, 0.984864824122}, - {-81.9954096721, 0.594841146008, -1.22273253129, 0.532466794358, 1.69864239257, - -0.12293671327, -2.06645974171, 0.611808231703, -1.32291985291, 0.722066660478, -0.0021343848511}, - {-245.715046329, -1.77850303496, -0.176518810079, 1.20463434525, -0.597826204963, - -1.45842350123, -0.765730251727, -2.17764204443, 0.12996635702, -0.705509516482, 0.170639846082}, - {123.011946043, -0.909707162714, 0.92357208515, 0.373251929121, 1.24629576577, - 0.0662688299998, -0.372240547929, -0.739353735168, 0.323495756066, 0.954154005738, 0.69606859977}, - {-70.4564963177, 0.650682297051, 0.378131376232, 1.37860253614, -0.924042783872, - 0.802851073842, -0.450299927542, 0.235646185302, -0.148779896161, 1.01308126122, -0.48206889502}, - {21.5288687935, 0.290876355386, 0.0765702960599, 0.905225489744, 0.252841861521, - 1.26729272819, 0.315397441908, -2.00317261368, -0.250990653758, 0.425615332405, 0.0875320802483}, - {231.370169905, 0.535138021352, -1.07151617232, 0.824383756287, 1.84428896701, - -0.890892034494, 0.0480296332924, -0.59251208055, 0.267564961845, -0.230698441998, 0.857077278291}, - {38.8318274023, 2.63547217711, -0.585553060394, 0.430550920323, -0.532619160993, - 1.25335488136, -1.65265278435, 0.0433880112291, -0.166143379872, 0.534066441314, 1.18929937797}, - {116.362219013, -0.275949982433, 0.468069787645, -0.879814121059, 0.862799331322, - 1.18464846725, 0.747084253268, 1.39202500691, -1.23374181275, 0.0949815110503, 0.696546907194}, - {260.540154731, 1.13798788241, -0.0991903174656, 0.1241636043, -0.201415073037, - 1.57683389508, 1.81535629587, 1.07873616646, -0.355800782882, 2.18333193195, 0.0711071144615}, - {-165.835194521, -2.76613178307, 0.805314338858, 0.81526046683, -0.710489036197, - -1.20189542317, -0.692110074722, -0.117239516622, 1.0431459458, -0.111898596299, -0.0775811519297}, - {-341.189958588, 0.668555635008, -1.0940034941, -0.497881262778, -0.603682823779, - -0.396875163796, -0.849144848521, 0.403936807183, -1.82076277475, -0.137500972546, -1.22769896568} - }, new double[] {45.8685095528, 11.9400336005, 16.3984976652, 79.9069814034, 5.65486853464, - 83.6427296424, 27.4571268153, 73.5881193584, 27.1465364511, 79.4095449062}, -5.14077007134); - - /** */ - public static class TestDataset { - - /** */ - private final double[][] data; - - /** */ - private final double[] expWeights; - - /** */ - private final double expIntercept; - - /** */ - TestDataset(double[][] data, double[] expWeights, double expIntercept) { - this.data = data; - this.expWeights = expWeights; - this.expIntercept = expIntercept; - } - - /** */ - public double[][] getData() { - return data; - } - - /** */ - public double[] getExpWeights() { - return expWeights; - } - - /** */ - public double getExpIntercept() { - return expIntercept; - } - } -} \ No newline at end of file diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/BlockDistributedLinearRegressionQRTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/BlockDistributedLinearRegressionQRTrainerTest.java deleted file mode 100644 index 0c09d75dd8314..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/BlockDistributedLinearRegressionQRTrainerTest.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.ml.regressions.linear; - -import org.apache.ignite.ml.math.impls.matrix.SparseBlockDistributedMatrix; -import org.apache.ignite.ml.math.impls.vector.SparseBlockDistributedVector; - -/** - * Tests for {@link LinearRegressionQRTrainer} on {@link SparseBlockDistributedMatrix}. - */ -public class BlockDistributedLinearRegressionQRTrainerTest extends GridAwareAbstractLinearRegressionTrainerTest { - /** */ - public BlockDistributedLinearRegressionQRTrainerTest() { - super( - new LinearRegressionQRTrainer(), - SparseBlockDistributedMatrix::new, - SparseBlockDistributedVector::new, - 1e-6 - ); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/DistributedLinearRegressionQRTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/DistributedLinearRegressionQRTrainerTest.java deleted file mode 100644 index 2a506d9410c86..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/DistributedLinearRegressionQRTrainerTest.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.ml.regressions.linear; - -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.ml.math.impls.vector.SparseDistributedVector; - -/** - * Tests for {@link LinearRegressionQRTrainer} on {@link SparseDistributedMatrix}. - */ -public class DistributedLinearRegressionQRTrainerTest extends GridAwareAbstractLinearRegressionTrainerTest { - /** */ - public DistributedLinearRegressionQRTrainerTest() { - super( - new LinearRegressionQRTrainer(), - SparseDistributedMatrix::new, - SparseDistributedVector::new, - 1e-6 - ); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GenericLinearRegressionTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GenericLinearRegressionTrainerTest.java deleted file mode 100644 index a55623ca15200..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GenericLinearRegressionTrainerTest.java +++ /dev/null @@ -1,206 +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.regressions.linear; - -import java.util.Scanner; -import org.apache.ignite.ml.TestUtils; -import org.apache.ignite.ml.Trainer; -import org.apache.ignite.ml.math.Matrix; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.junit.Test; - -/** - * Base class for all linear regression trainers. - */ -public class GenericLinearRegressionTrainerTest { - /** */ - private final Trainer trainer; - - /** */ - private final IgniteFunction matrixCreator; - - /** */ - private final IgniteFunction vectorCreator; - - /** */ - private final double precision; - - /** */ - public GenericLinearRegressionTrainerTest( - Trainer trainer, - IgniteFunction matrixCreator, - IgniteFunction vectorCreator, - double precision) { - this.trainer = trainer; - this.matrixCreator = matrixCreator; - this.vectorCreator = vectorCreator; - this.precision = precision; - } - - /** - * Test trainer on regression model y = 2 * x. - */ - @Test - public void testTrainWithoutIntercept() { - Matrix data = matrixCreator.apply(new double[][] { - {2.0, 1.0}, - {4.0, 2.0} - }); - - LinearRegressionModel mdl = trainer.train(data); - - TestUtils.assertEquals(4, mdl.apply(vectorCreator.apply(new double[] {2})), precision); - TestUtils.assertEquals(6, mdl.apply(vectorCreator.apply(new double[] {3})), precision); - TestUtils.assertEquals(8, mdl.apply(vectorCreator.apply(new double[] {4})), precision); - } - - /** - * Test trainer on regression model y = -1 * x + 1. - */ - @Test - public void testTrainWithIntercept() { - Matrix data = matrixCreator.apply(new double[][] { - {1.0, 0.0}, - {0.0, 1.0} - }); - - LinearRegressionModel mdl = trainer.train(data); - - TestUtils.assertEquals(0.5, mdl.apply(vectorCreator.apply(new double[] {0.5})), precision); - TestUtils.assertEquals(2, mdl.apply(vectorCreator.apply(new double[] {-1})), precision); - TestUtils.assertEquals(-1, mdl.apply(vectorCreator.apply(new double[] {2})), precision); - } - - /** - * Test trainer on diabetes dataset. - */ - @Test - public void testTrainOnDiabetesDataset() { - Matrix data = loadDataset("datasets/regression/diabetes.csv", 442, 10); - - LinearRegressionModel mdl = trainer.train(data); - - Vector expWeights = vectorCreator.apply(new double[] { - -10.01219782, -239.81908937, 519.83978679, 324.39042769, -792.18416163, - 476.74583782, 101.04457032, 177.06417623, 751.27932109, 67.62538639 - }); - - double expIntercept = 152.13348416; - - TestUtils.assertEquals("Wrong weights", expWeights, mdl.getWeights(), precision); - TestUtils.assertEquals("Wrong intercept", expIntercept, mdl.getIntercept(), precision); - } - - /** - * Test trainer on boston dataset. - */ - @Test - public void testTrainOnBostonDataset() { - Matrix data = loadDataset("datasets/regression/boston.csv", 506, 13); - - LinearRegressionModel mdl = trainer.train(data); - - Vector expWeights = vectorCreator.apply(new double[] { - -1.07170557e-01, 4.63952195e-02, 2.08602395e-02, 2.68856140e+00, -1.77957587e+01, 3.80475246e+00, - 7.51061703e-04, -1.47575880e+00, 3.05655038e-01, -1.23293463e-02, -9.53463555e-01, 9.39251272e-03, - -5.25466633e-01 - }); - - double expIntercept = 36.4911032804; - - TestUtils.assertEquals("Wrong weights", expWeights, mdl.getWeights(), precision); - TestUtils.assertEquals("Wrong intercept", expIntercept, mdl.getIntercept(), precision); - } - - /** - * Tests trainer on artificial dataset with 10 observations described by 1 feature. - */ - @Test - public void testTrainOnArtificialDataset10x1() { - ArtificialRegressionDatasets.TestDataset dataset = ArtificialRegressionDatasets.regression10x1; - - LinearRegressionModel mdl = trainer.train(matrixCreator.apply(dataset.getData())); - - TestUtils.assertEquals("Wrong weights", dataset.getExpWeights(), mdl.getWeights(), precision); - TestUtils.assertEquals("Wrong intercept", dataset.getExpIntercept(), mdl.getIntercept(), precision); - } - - /** - * Tests trainer on artificial dataset with 10 observations described by 5 features. - */ - @Test - public void testTrainOnArtificialDataset10x5() { - ArtificialRegressionDatasets.TestDataset dataset = ArtificialRegressionDatasets.regression10x5; - - LinearRegressionModel mdl = trainer.train(matrixCreator.apply(dataset.getData())); - - TestUtils.assertEquals("Wrong weights", dataset.getExpWeights(), mdl.getWeights(), precision); - TestUtils.assertEquals("Wrong intercept", dataset.getExpIntercept(), mdl.getIntercept(), precision); - } - - /** - * Tests trainer on artificial dataset with 100 observations described by 5 features. - */ - @Test - public void testTrainOnArtificialDataset100x5() { - ArtificialRegressionDatasets.TestDataset dataset = ArtificialRegressionDatasets.regression100x5; - - LinearRegressionModel mdl = trainer.train(matrixCreator.apply(dataset.getData())); - - TestUtils.assertEquals("Wrong weights", dataset.getExpWeights(), mdl.getWeights(), precision); - TestUtils.assertEquals("Wrong intercept", dataset.getExpIntercept(), mdl.getIntercept(), precision); - } - - /** - * Tests trainer on artificial dataset with 100 observations described by 10 features. - */ - @Test - public void testTrainOnArtificialDataset100x10() { - ArtificialRegressionDatasets.TestDataset dataset = ArtificialRegressionDatasets.regression100x10; - - LinearRegressionModel mdl = trainer.train(matrixCreator.apply(dataset.getData())); - - TestUtils.assertEquals("Wrong weights", dataset.getExpWeights(), mdl.getWeights(), precision); - TestUtils.assertEquals("Wrong intercept", dataset.getExpIntercept(), mdl.getIntercept(), precision); - } - - /** - * Loads dataset file and returns corresponding matrix. - * - * @param fileName Dataset file name - * @param nobs Number of observations - * @param nvars Number of features - * @return Data matrix - */ - private Matrix loadDataset(String fileName, int nobs, int nvars) { - double[][] matrix = new double[nobs][nvars + 1]; - Scanner scanner = new Scanner(this.getClass().getClassLoader().getResourceAsStream(fileName)); - int i = 0; - while (scanner.hasNextLine()) { - String row = scanner.nextLine(); - int j = 0; - for (String feature : row.split(",")) { - matrix[i][j] = Double.parseDouble(feature); - j++; - } - i++; - } - return matrixCreator.apply(matrix); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GridAwareAbstractLinearRegressionTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GridAwareAbstractLinearRegressionTrainerTest.java deleted file mode 100644 index 9b75bd41392a6..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GridAwareAbstractLinearRegressionTrainerTest.java +++ /dev/null @@ -1,127 +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.regressions.linear; - -import org.apache.ignite.Ignite; -import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.Trainer; -import org.apache.ignite.ml.math.Matrix; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import org.junit.Test; - -/** - * Grid aware abstract linear regression trainer test. - */ -public abstract class GridAwareAbstractLinearRegressionTrainerTest extends GridCommonAbstractTest { - /** Number of nodes in grid */ - private static final int NODE_COUNT = 3; - - /** - * Delegate actually performs tests. - */ - private final GenericLinearRegressionTrainerTest delegate; - - /** */ - private Ignite ignite; - - /** */ - public GridAwareAbstractLinearRegressionTrainerTest( - Trainer trainer, - IgniteFunction matrixCreator, - IgniteFunction vectorCreator, - double precision) { - delegate = new GenericLinearRegressionTrainerTest(trainer, matrixCreator, vectorCreator, precision); - } - - /** {@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 regression model y = 2 * x. - */ - @Test - public void testTrainWithoutIntercept() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - delegate.testTrainWithoutIntercept(); - } - - /** - * Test trainer on regression model y = -1 * x + 1. - */ - @Test - public void testTrainWithIntercept() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - delegate.testTrainWithIntercept(); - } - - /** - * Tests trainer on artificial dataset with 10 observations described by 1 feature. - */ - @Test - public void testTrainOnArtificialDataset10x1() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - delegate.testTrainOnArtificialDataset10x1(); - } - - /** - * Tests trainer on artificial dataset with 10 observations described by 5 features. - */ - @Test - public void testTrainOnArtificialDataset10x5() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - delegate.testTrainOnArtificialDataset10x5(); - } - - /** - * Tests trainer on artificial dataset with 100 observations described by 5 features. - */ - @Test - public void testTrainOnArtificialDataset100x5() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - delegate.testTrainOnArtificialDataset100x5(); - } - - /** - * Tests trainer on artificial dataset with 100 observations described by 10 features. - */ - @Test - public void testTrainOnArtificialDataset100x10() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - delegate.testTrainOnArtificialDataset100x10(); - } -} 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 fa8fac408b112..c62cca5e9ea90 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 @@ -19,7 +19,7 @@ import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate; import org.apache.ignite.ml.optimization.updatecalculators.RPropUpdateCalculator; -import org.apache.ignite.ml.trainers.group.UpdatesStrategy; +import org.apache.ignite.ml.nn.UpdatesStrategy; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LocalLinearRegressionQRTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LocalLinearRegressionQRTrainerTest.java deleted file mode 100644 index f37d71d0c8b9f..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LocalLinearRegressionQRTrainerTest.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.ml.regressions.linear; - -import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; - -/** - * Tests for {@link LinearRegressionQRTrainer} on {@link DenseLocalOnHeapMatrix}. - */ -public class LocalLinearRegressionQRTrainerTest extends GenericLinearRegressionTrainerTest { - /** */ - public LocalLinearRegressionQRTrainerTest() { - super( - new LinearRegressionQRTrainer(), - DenseLocalOnHeapMatrix::new, - DenseLocalOnHeapVector::new, - 1e-6 - ); - } -} \ No newline at end of file diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/DistributedWorkersChainTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/DistributedWorkersChainTest.java deleted file mode 100644 index 7ad59d168e099..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/DistributedWorkersChainTest.java +++ /dev/null @@ -1,189 +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.trainers.group; - -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.stream.Stream; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.Ignition; -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.trainers.group.chain.Chains; -import org.apache.ignite.ml.trainers.group.chain.ComputationsChain; -import org.apache.ignite.ml.trainers.group.chain.EntryAndContext; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import org.junit.Assert; - -/** */ -public class DistributedWorkersChainTest extends GridCommonAbstractTest { - /** Count of nodes. */ - private static final int NODE_COUNT = 3; - - /** Grid instance. */ - protected Ignite ignite; - - /** - * Default constructor. - */ - public DistributedWorkersChainTest() { - super(false); - } - - /** - * {@inheritDoc} - */ - @Override protected void beforeTest() throws Exception { - ignite = grid(NODE_COUNT); - TestGroupTrainingCache.getOrCreate(ignite).removeAll(); - TestGroupTrainingSecondCache.getOrCreate(ignite).removeAll(); - } - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - for (int i = 1; i <= NODE_COUNT; i++) - startGrid(i); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() throws Exception { - stopAllGrids(); - } - - /** */ - public void testId() { - ComputationsChain chain = Chains.create(); - - UUID trainingUUID = UUID.randomUUID(); - Integer res = chain.process(1, new GroupTrainingContext<>(new TestLocalContext(0, trainingUUID), TestGroupTrainingCache.getOrCreate(ignite), ignite)); - - Assert.assertEquals(1L, (long)res); - } - - /** */ - public void testSimpleLocal() { - ComputationsChain chain = Chains.create(); - - IgniteCache, Integer> cache = TestGroupTrainingCache.getOrCreate(ignite); - int init = 1; - int initLocCtxData = 0; - UUID trainingUUID = UUID.randomUUID(); - TestLocalContext locCtx = new TestLocalContext(initLocCtxData, trainingUUID); - - Integer res = chain. - thenLocally((prev, lc) -> prev + 1). - process(init, new GroupTrainingContext<>(locCtx, cache, ignite)); - - Assert.assertEquals(init + 1, (long)res); - Assert.assertEquals(initLocCtxData, locCtx.data()); - } - - /** */ - public void testChainLocal() { - ComputationsChain chain = Chains.create(); - - IgniteCache, Integer> cache = TestGroupTrainingCache.getOrCreate(ignite); - int init = 1; - int initLocCtxData = 0; - UUID trainingUUID = UUID.randomUUID(); - TestLocalContext locCtx = new TestLocalContext(initLocCtxData, trainingUUID); - - Integer res = chain. - thenLocally((prev, lc) -> prev + 1). - thenLocally((prev, lc) -> prev * 5). - process(init, new GroupTrainingContext<>(locCtx, cache, ignite)); - - Assert.assertEquals((init + 1) * 5, (long)res); - Assert.assertEquals(initLocCtxData, locCtx.data()); - } - - /** */ - public void testChangeLocalContext() { - ComputationsChain chain = Chains.create(); - IgniteCache, Integer> cache = TestGroupTrainingCache.getOrCreate(ignite); - int init = 1; - int newData = 10; - UUID trainingUUID = UUID.randomUUID(); - TestLocalContext locCtx = new TestLocalContext(0, trainingUUID); - - Integer res = chain. - thenLocally((prev, lc) -> { lc.setData(newData); return prev;}). - process(init, new GroupTrainingContext<>(locCtx, cache, ignite)); - - Assert.assertEquals(newData, locCtx.data()); - Assert.assertEquals(init, res.intValue()); - } - - /** */ - public void testDistributed() { - ComputationsChain chain = Chains.create(); - IgniteCache, Integer> cache = TestGroupTrainingCache.getOrCreate(ignite); - int init = 1; - UUID trainingUUID = UUID.randomUUID(); - TestLocalContext locCtx = new TestLocalContext(0, trainingUUID); - - Map, Integer> m = new HashMap<>(); - m.put(new GroupTrainerCacheKey<>(0L, 1.0, trainingUUID), 1); - m.put(new GroupTrainerCacheKey<>(1L, 2.0, trainingUUID), 2); - m.put(new GroupTrainerCacheKey<>(2L, 3.0, trainingUUID), 3); - m.put(new GroupTrainerCacheKey<>(3L, 4.0, trainingUUID), 4); - - Stream> keys = m.keySet().stream(); - - cache.putAll(m); - - IgniteBiFunction>>> function = (o, l) -> () -> keys; - IgniteFunction, Integer> max = ints -> ints.stream().mapToInt(x -> x).max().orElse(Integer.MIN_VALUE); - - Integer res = chain. - thenDistributedForEntries((integer, context) -> () -> null, this::readAndIncrement, function, max). - process(init, new GroupTrainingContext<>(locCtx, cache, ignite)); - - int localMax = m.values().stream().max(Comparator.comparingInt(i -> i)).orElse(Integer.MIN_VALUE); - - assertEquals((long)localMax, (long)res); - - for (GroupTrainerCacheKey key : m.keySet()) - m.compute(key, (k, v) -> v + 1); - - assertMapEqualsCache(m, cache); - } - - /** */ - private ResultAndUpdates readAndIncrement(EntryAndContext ec) { - Integer val = ec.entry().getValue(); - - ResultAndUpdates res = ResultAndUpdates.of(val); - res.updateCache(TestGroupTrainingCache.getOrCreate(Ignition.localIgnite()), ec.entry().getKey(), val + 1); - - return res; - } - - /** */ - private void assertMapEqualsCache(Map m, IgniteCache cache) { - assertEquals(m.size(), cache.size()); - - for (K k : m.keySet()) - assertEquals(m.get(k), cache.get(k)); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/GroupTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/GroupTrainerTest.java deleted file mode 100644 index 5bb9a4706b7b7..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/GroupTrainerTest.java +++ /dev/null @@ -1,90 +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.trainers.group; - -import java.util.HashMap; -import java.util.Map; -import org.apache.ignite.Ignite; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; - -/** - * Test of {@link GroupTrainer}. - */ -public class GroupTrainerTest extends GridCommonAbstractTest { - /** Count of nodes. */ - private static final int NODE_COUNT = 3; - - /** Grid instance. */ - private Ignite ignite; - - /** - * Default constructor. - */ - public GroupTrainerTest() { - super(false); - } - - /** - * {@inheritDoc} - */ - @Override protected void beforeTest() throws Exception { - ignite = grid(NODE_COUNT); - TestGroupTrainingCache.getOrCreate(ignite).removeAll(); - TestGroupTrainingSecondCache.getOrCreate(ignite).removeAll(); - } - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - for (int i = 1; i <= NODE_COUNT; i++) - startGrid(i); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() throws Exception { - stopAllGrids(); - } - - /** */ - public void testGroupTrainer() { - TestGroupTrainer trainer = new TestGroupTrainer(ignite); - - int limit = 5; - int eachNumCnt = 3; - int iterCnt = 2; - - ConstModel mdl = trainer.train(new SimpleGroupTrainerInput(limit, eachNumCnt, iterCnt)); - int locRes = computeLocally(limit, eachNumCnt, iterCnt); - assertEquals(locRes, (int)mdl.apply(10)); - } - - /** */ - private int computeLocally(int limit, int eachNumCnt, int iterCnt) { - Map, Integer> m = new HashMap<>(); - - for (int i = 0; i < limit; i++) { - for (int j = 0; j < eachNumCnt; j++) - m.put(new GroupTrainerCacheKey<>(i, (double)j, null), i); - } - - for (int i = 0; i < iterCnt; i++) - for (GroupTrainerCacheKey key : m.keySet()) - m.compute(key, (key1, integer) -> integer * integer); - - return m.values().stream().filter(x -> x % 2 == 0).mapToInt(i -> i).sum(); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/SimpleGroupTrainerInput.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/SimpleGroupTrainerInput.java deleted file mode 100644 index db1adc74b81cf..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/SimpleGroupTrainerInput.java +++ /dev/null @@ -1,63 +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.trainers.group; - -import java.util.UUID; -import java.util.stream.IntStream; -import java.util.stream.Stream; -import org.apache.ignite.ml.math.functions.IgniteSupplier; - -public class SimpleGroupTrainerInput implements GroupTrainerInput { - /** */ - private int limit; - - /** */ - private int eachNumberCount; - - /** */ - private int iterCnt; - - /** */ - public SimpleGroupTrainerInput(int limit, int eachNumCnt, int iterCnt) { - this.limit = limit; - this.eachNumberCount = eachNumCnt; - this.iterCnt = iterCnt; - } - - /** {@inheritDoc} */ - @Override public IgniteSupplier>> initialKeys(UUID trainingUUID) { - int lim = limit; - UUID uuid = trainingUUID; - return () -> IntStream.range(0, lim).mapToObj(i -> new GroupTrainerCacheKey<>(i, 0.0, uuid)); - } - - /** */ - public int limit() { - return limit; - } - - /** */ - public int iterCnt() { - return iterCnt; - } - - /** */ - public int eachNumberCount() { - return eachNumberCount; - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainer.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainer.java deleted file mode 100644 index 0a49fe0fd7b74..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainer.java +++ /dev/null @@ -1,144 +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.trainers.group; - -import java.util.List; -import java.util.UUID; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.Ignition; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.functions.IgniteSupplier; -import org.apache.ignite.ml.trainers.group.chain.Chains; -import org.apache.ignite.ml.trainers.group.chain.ComputationsChain; -import org.apache.ignite.ml.trainers.group.chain.EntryAndContext; - -/** - * Test group trainer. - */ -public class TestGroupTrainer extends GroupTrainer, SimpleGroupTrainerInput, Void> { - /** - * Construct instance of this class with given parameters. - * - * @param ignite Ignite instance. - */ - public TestGroupTrainer(Ignite ignite) { - super(TestGroupTrainingCache.getOrCreate(ignite), ignite); - } - - /** {@inheritDoc} */ - @Override protected TestGroupTrainerLocalContext initialLocalContext(SimpleGroupTrainerInput data, - UUID trainingUUID) { - return new TestGroupTrainerLocalContext(data.iterCnt(), data.eachNumberCount(), data.limit(), trainingUUID); - } - - /** {@inheritDoc} */ - @Override protected IgniteFunction, ResultAndUpdates> distributedInitializer( - SimpleGroupTrainerInput data) { - return key -> { - long i = key.nodeLocalEntityIndex(); - UUID trainingUUID = key.trainingUUID(); - IgniteCache, Integer> cache - = TestGroupTrainingCache.getOrCreate(Ignition.localIgnite()); - - long sum = i * data.eachNumberCount(); - - ResultAndUpdates res = ResultAndUpdates.of((int)sum); - - for (int j = 0; j < data.eachNumberCount(); j++) - res.updateCache(cache, new GroupTrainerCacheKey<>(i, (double)j, trainingUUID), (int)i); - - return res; - }; - } - - /** {@inheritDoc} */ - @Override protected IgniteFunction, Integer> reduceDistributedInitData() { - return id -> id.stream().mapToInt(x -> x).sum(); - } - - /** {@inheritDoc} */ - @Override protected Double locallyProcessInitData(Integer data, TestGroupTrainerLocalContext locCtx) { - return data.doubleValue(); - } - - /** {@inheritDoc} */ - @Override protected ComputationsChain trainingLoopStep() { - // TODO:IGNITE-7405 here we should explicitly create variable because we cannot infer context type, think about it. - ComputationsChain chain = Chains. - create(new TestTrainingLoopStep()); - return chain. - thenLocally((aDouble, context) -> { - context.incCnt(); - return aDouble; - }); - } - - /** {@inheritDoc} */ - @Override protected boolean shouldContinue(Double data, TestGroupTrainerLocalContext locCtx) { - return locCtx.cnt() < locCtx.maxCnt(); - } - - /** {@inheritDoc} */ - @Override protected IgniteSupplier extractContextForFinalResultCreation(Double data, - TestGroupTrainerLocalContext locCtx) { - // No context is needed. - return () -> null; - } - - /** {@inheritDoc} */ - @Override protected IgniteSupplier>> finalResultKeys(Double data, - TestGroupTrainerLocalContext locCtx) { - int limit = locCtx.limit(); - int cnt = locCtx.eachNumberCnt(); - UUID uuid = locCtx.trainingUUID(); - - return () -> TestGroupTrainingCache.allKeys(limit, cnt, uuid); - } - - /** {@inheritDoc} */ - @Override protected IgniteFunction, - ResultAndUpdates> finalResultsExtractor() { - return entryAndCtx -> { - Integer val = entryAndCtx.entry().getValue(); - return ResultAndUpdates.of(val % 2 == 0 ? val : 0); - }; - } - - /** {@inheritDoc} */ - @Override protected IgniteFunction, Integer> finalResultsReducer() { - return id -> id.stream().mapToInt(x -> x).sum(); - } - - /** {@inheritDoc} */ - @Override protected ConstModel mapFinalResult(Integer res, TestGroupTrainerLocalContext locCtx) { - return new ConstModel<>(res); - } - - /** {@inheritDoc} */ - @Override protected void cleanup(TestGroupTrainerLocalContext locCtx) { - Stream> toRemote = TestGroupTrainingCache.allKeys(locCtx.limit(), - locCtx.eachNumberCnt(), locCtx.trainingUUID()); - - TestGroupTrainingCache.getOrCreate(ignite).removeAll(toRemote.collect(Collectors.toSet())); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainerLocalContext.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainerLocalContext.java deleted file mode 100644 index e1a533b16761f..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainerLocalContext.java +++ /dev/null @@ -1,85 +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.trainers.group; - -import java.util.UUID; -import org.apache.ignite.ml.trainers.group.chain.HasTrainingUUID; - -/** */ -public class TestGroupTrainerLocalContext implements HasTrainingUUID { - /** */ - private int cnt = 0; - - /** */ - private int maxCnt; - - /** */ - private int eachNumberCnt; - - /** */ - private int limit; - - /** */ - private UUID trainingUUID; - - /** */ - public TestGroupTrainerLocalContext(int maxCnt, int eachNumberCnt, int limit, UUID trainingUUID) { - this.maxCnt = maxCnt; - this.eachNumberCnt = eachNumberCnt; - this.limit = limit; - this.trainingUUID = trainingUUID; - this.cnt = 0; - } - - /** */ - public int cnt() { - return cnt; - } - - /** */ - public void setCnt(int cnt) { - this.cnt = cnt; - } - - /** */ - public TestGroupTrainerLocalContext incCnt() { - this.cnt++; - - return this; - } - - /** */ - public int maxCnt() { - return maxCnt; - } - - /** */ - public int eachNumberCnt() { - return eachNumberCnt; - } - - /** */ - public int limit() { - return limit; - } - - /** {@inheritDoc} */ - @Override public UUID trainingUUID() { - return trainingUUID; - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainingCache.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainingCache.java deleted file mode 100644 index afee674528c69..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainingCache.java +++ /dev/null @@ -1,70 +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.trainers.group; - -import java.util.Arrays; -import java.util.UUID; -import java.util.stream.Stream; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.cache.CacheAtomicityMode; -import org.apache.ignite.cache.CacheMode; -import org.apache.ignite.cache.CacheWriteSynchronizationMode; -import org.apache.ignite.configuration.CacheConfiguration; - -/** */ -public class TestGroupTrainingCache { - /** */ - public static String CACHE_NAME = "TEST_GROUP_TRAINING_CACHE"; - - /** */ - public static IgniteCache, Integer> getOrCreate(Ignite ignite) { - CacheConfiguration, Integer> cfg = new CacheConfiguration<>(); - - // Write to primary. - cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC); - - // Atomic transactions only. - cfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); - - // No copying of values. - cfg.setCopyOnRead(false); - - // Cache is partitioned. - cfg.setCacheMode(CacheMode.PARTITIONED); - - cfg.setBackups(0); - - cfg.setOnheapCacheEnabled(true); - - cfg.setName(CACHE_NAME); - - return ignite.getOrCreateCache(cfg); - } - - /** */ - public static Stream> allKeys(int limit, int eachNumberCnt, UUID trainingUUID) { - GroupTrainerCacheKey[] a =new GroupTrainerCacheKey[limit * eachNumberCnt]; - - for (int num = 0; num < limit; num++) - for (int i = 0; i < eachNumberCnt; i++) - a[num * eachNumberCnt + i] = new GroupTrainerCacheKey<>(num, (double)i, trainingUUID); - - return Arrays.stream(a); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainingSecondCache.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainingSecondCache.java deleted file mode 100644 index e16ed7c74ada2..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainingSecondCache.java +++ /dev/null @@ -1,56 +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.trainers.group; - -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.cache.CacheAtomicityMode; -import org.apache.ignite.cache.CacheMode; -import org.apache.ignite.cache.CacheWriteSynchronizationMode; -import org.apache.ignite.configuration.CacheConfiguration; - -/** */ -public class TestGroupTrainingSecondCache { - /** */ - public static String CACHE_NAME = "TEST_GROUP_TRAINING_SECOND_CACHE"; - - /** */ - public static IgniteCache, Integer> getOrCreate(Ignite ignite) { - CacheConfiguration, Integer> cfg = new CacheConfiguration<>(); - - // Write to primary. - cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC); - - // Atomic transactions only. - cfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); - - // No copying of values. - cfg.setCopyOnRead(false); - - // Cache is partitioned. - cfg.setCacheMode(CacheMode.PARTITIONED); - - cfg.setBackups(0); - - cfg.setOnheapCacheEnabled(true); - - cfg.setName(CACHE_NAME); - - return ignite.getOrCreateCache(cfg); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestLocalContext.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestLocalContext.java deleted file mode 100644 index 3f0237fc1c9fa..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestLocalContext.java +++ /dev/null @@ -1,51 +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.trainers.group; - -import java.util.UUID; -import org.apache.ignite.ml.trainers.group.chain.HasTrainingUUID; - -/** */ -public class TestLocalContext implements HasTrainingUUID { - /** */ - private final UUID trainingUUID; - - /** */ - private int data; - - /** */ - public TestLocalContext(int data, UUID trainingUUID) { - this.data = data; - this.trainingUUID = trainingUUID; - } - - /** */ - public int data() { - return data; - } - - /** */ - public void setData(int data) { - this.data = data; - } - - /** {@inheritDoc} */ - @Override public UUID trainingUUID() { - return trainingUUID; - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestTrainingLoopStep.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestTrainingLoopStep.java deleted file mode 100644 index caf92f6b53ce8..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestTrainingLoopStep.java +++ /dev/null @@ -1,65 +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.trainers.group; - -import java.util.List; -import java.util.UUID; -import java.util.stream.Stream; -import org.apache.ignite.Ignition; -import org.apache.ignite.ml.math.functions.IgniteFunction; -import org.apache.ignite.ml.math.functions.IgniteSupplier; -import org.apache.ignite.ml.trainers.group.chain.DistributedEntryProcessingStep; -import org.apache.ignite.ml.trainers.group.chain.EntryAndContext; - -/** */ -public class TestTrainingLoopStep implements DistributedEntryProcessingStep { - /** {@inheritDoc} */ - @Override public IgniteSupplier remoteContextSupplier(Double input, TestGroupTrainerLocalContext locCtx) { - // No context is needed. - return () -> null; - } - - /** {@inheritDoc} */ - @Override public IgniteFunction, ResultAndUpdates> worker() { - return entryAndContext -> { - Integer oldVal = entryAndContext.entry().getValue(); - double v = oldVal * oldVal; - ResultAndUpdates res = ResultAndUpdates.of(v); - res.updateCache(TestGroupTrainingCache.getOrCreate(Ignition.localIgnite()), - entryAndContext.entry().getKey(), (int)v); - return res; - }; - } - - /** {@inheritDoc} */ - @Override public IgniteSupplier>> keys(Double input, - TestGroupTrainerLocalContext locCtx) { - // Copying here because otherwise locCtx will be serialized with supplier returned in result. - int limit = locCtx.limit(); - int cnt = locCtx.eachNumberCnt(); - UUID uuid = locCtx.trainingUUID(); - - return () -> TestGroupTrainingCache.allKeys(limit, cnt, uuid); - } - - /** {@inheritDoc} */ - @Override public IgniteFunction, Double> reducer() { - return doubles -> doubles.stream().mapToDouble(x -> x).sum(); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TrainersGroupTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TrainersGroupTestSuite.java deleted file mode 100644 index 0ec5afb2ce559..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TrainersGroupTestSuite.java +++ /dev/null @@ -1,32 +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.trainers.group; - -import org.junit.runner.RunWith; -import org.junit.runners.Suite; - -/** - * Test suite for group trainer tests. - */ -@RunWith(Suite.class) -@Suite.SuiteClasses({ - DistributedWorkersChainTest.class, - GroupTrainerTest.class -}) -public class TrainersGroupTestSuite { -} diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/regression/IgniteOLSMultipleLinearRegressionBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/regression/IgniteOLSMultipleLinearRegressionBenchmark.java deleted file mode 100644 index 89b54716261f8..0000000000000 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/regression/IgniteOLSMultipleLinearRegressionBenchmark.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.yardstick.ml.regression; - -import java.util.Map; -import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; -import org.apache.ignite.ml.regressions.linear.LinearRegressionModel; -import org.apache.ignite.ml.regressions.linear.LinearRegressionQRTrainer; -import org.apache.ignite.yardstick.IgniteAbstractBenchmark; - -/** - * Ignite benchmark that performs ML Grid operations. - */ -@SuppressWarnings("unused") -public class IgniteOLSMultipleLinearRegressionBenchmark extends IgniteAbstractBenchmark { - /** {@inheritDoc} */ - @Override public boolean test(Map ctx) throws Exception { - runLongly(); - - return true; - } - - /** - * Based on OLSMultipleLinearRegressionTest#testLongly. - */ - private void runLongly() { - // Y values are first, then independent vars - // Each row is one observation - double[][] data = new double[][] { - { 60323, 83.0, 234289, 2356, 1590, 107608, 1947 }, - { 61122, 88.5, 259426, 2325, 1456, 108632, 1948 }, - { 60171, 88.2, 258054, 3682, 1616, 109773, 1949 }, - { 61187, 89.5, 284599, 3351, 1650, 110929, 1950 }, - { 63221, 96.2, 328975, 2099, 3099, 112075, 1951 }, - { 63639, 98.1, 346999, 1932, 3594, 113270, 1952 }, - { 64989, 99.0, 365385, 1870, 3547, 115094, 1953 }, - { 63761, 100.0, 363112, 3578, 3350, 116219, 1954 }, - { 66019, 101.2, 397469, 2904, 3048, 117388, 1955 }, - { 67857, 104.6, 419180, 2822, 2857, 118734, 1956 }, - { 68169, 108.4, 442769, 2936, 2798, 120445, 1957 }, - { 66513, 110.8, 444546, 4681, 2637, 121950, 1958 }, - { 68655, 112.6, 482704, 3813, 2552, 123366, 1959 }, - { 69564, 114.2, 502601, 3931, 2514, 125368, 1960 }, - { 69331, 115.7, 518173, 4806, 2572, 127852, 1961 }, - { 70551, 116.9, 554894, 4007, 2827, 130081, 1962 } - }; - - final int nobs = 16; - final int nvars = 6; - - LinearRegressionQRTrainer trainer = new LinearRegressionQRTrainer(); - LinearRegressionModel model = trainer.train(new DenseLocalOnHeapMatrix(data)); - } -} diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/regression/package-info.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/regression/package-info.java deleted file mode 100644 index 0a5dc1a0b6bf1..0000000000000 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/regression/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. - */ - -/** - * - * ML Grid regression benchmarks. - */ -package org.apache.ignite.yardstick.ml.regression; \ No newline at end of file From 8acee1da7fc8475e7d2ffb25b179055d93906034 Mon Sep 17 00:00:00 2001 From: YuriBabak Date: Fri, 13 Apr 2018 20:27:15 +0300 Subject: [PATCH 0052/1463] IGNITE-8232: ML package cleanup for 2.5 release this closes #3823 (cherry picked from commit c6ab036) --- .../org/apache/ignite/ml/math/functions/IgniteBiFunction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/functions/IgniteBiFunction.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/functions/IgniteBiFunction.java index 45fd035710e79..560be4ba35d7b 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/math/functions/IgniteBiFunction.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/functions/IgniteBiFunction.java @@ -27,7 +27,7 @@ * @see java.util.function.BiFunction */ public interface IgniteBiFunction extends BiFunction, Serializable { - /** {@inheritDoc} */ + /** */ default IgniteBiFunction andThen(IgniteFunction after) { Objects.requireNonNull(after); return (T t, U u) -> after.apply(apply(t, u)); From c51f2465113d0933336a36d0f173051f68c91be7 Mon Sep 17 00:00:00 2001 From: Ilya Kasnacheev Date: Fri, 13 Apr 2018 14:50:11 -0700 Subject: [PATCH 0053/1463] IGNITE-2766 - Opportunistically reopen cache after client reconnect - Fixes #3417 Signed-off-by: Valentin Kulichenko --- .../cache/GatewayProtectedCacheProxy.java | 676 +++++++----------- .../processors/cache/GridCacheGateway.java | 7 + .../cache/IgniteCacheProxyImpl.java | 31 +- .../ignite/spi/discovery/tcp/ClientImpl.java | 6 +- .../IgniteCacheQueryCacheDestroySelfTest.java | 4 + ...lientReconnectAfterClusterRestartTest.java | 33 +- 6 files changed, 316 insertions(+), 441 deletions(-) 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 27fc39563e36a..2e8120ba7bde2 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 @@ -36,6 +36,7 @@ import javax.cache.processor.EntryProcessor; import javax.cache.processor.EntryProcessorResult; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.CacheEntry; import org.apache.ignite.cache.CacheEntryProcessor; import org.apache.ignite.cache.CacheMetrics; @@ -48,6 +49,9 @@ import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cluster.ClusterGroup; import org.apache.ignite.internal.AsyncSupportAdapter; +import org.apache.ignite.internal.GridKernalState; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.future.IgniteFutureImpl; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteClosure; @@ -138,15 +142,13 @@ public void setCacheManager(org.apache.ignite.cache.CacheManager cacheMgr) { /** {@inheritDoc} */ @Override public GatewayProtectedCacheProxy withExpiryPolicy(ExpiryPolicy plc) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return new GatewayProtectedCacheProxy<>(delegate, opCtx.withExpiryPolicy(plc), lock); } finally { - onLeave(gate, prev); + onLeave(opGate); } } @@ -157,9 +159,7 @@ public void setCacheManager(org.apache.ignite.cache.CacheManager cacheMgr) { /** {@inheritDoc} */ @Override public GatewayProtectedCacheProxy skipStore() { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { boolean skip = opCtx.skipStore(); @@ -170,15 +170,13 @@ public void setCacheManager(org.apache.ignite.cache.CacheManager cacheMgr) { return new GatewayProtectedCacheProxy<>(delegate, opCtx.setSkipStore(true), lock); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public GatewayProtectedCacheProxy withNoRetries() { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { boolean noRetries = opCtx.noRetries(); @@ -189,15 +187,13 @@ public void setCacheManager(org.apache.ignite.cache.CacheManager cacheMgr) { return new GatewayProtectedCacheProxy<>(delegate, opCtx.setNoRetries(true), lock); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public GatewayProtectedCacheProxy withPartitionRecover() { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { boolean recovery = opCtx.recovery(); @@ -208,7 +204,7 @@ public void setCacheManager(org.apache.ignite.cache.CacheManager cacheMgr) { return new GatewayProtectedCacheProxy<>(delegate, opCtx.setRecovery(true), lock); } finally { - onLeave(gate, prev); + onLeave(opGate); } } @@ -219,23 +215,19 @@ public void setCacheManager(org.apache.ignite.cache.CacheManager cacheMgr) { /** {@inheritDoc} */ @Override public GatewayProtectedCacheProxy keepBinary() { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return new GatewayProtectedCacheProxy<>((IgniteCacheProxy) delegate, opCtx.keepBinary(), lock); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public GatewayProtectedCacheProxy withDataCenterId(byte dataCenterId) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { Byte prevDataCenterId = opCtx.dataCenterId(); @@ -246,91 +238,79 @@ public void setCacheManager(org.apache.ignite.cache.CacheManager cacheMgr) { return new GatewayProtectedCacheProxy<>(delegate, opCtx.setDataCenterId(dataCenterId), lock); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public void loadCache(@Nullable IgniteBiPredicate p, @Nullable Object... args) throws CacheException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { delegate.loadCache(p, args); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture loadCacheAsync(@Nullable IgniteBiPredicate p, @Nullable Object... args) throws CacheException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.loadCacheAsync(p, args); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public void localLoadCache(@Nullable IgniteBiPredicate p, @Nullable Object... args) throws CacheException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { delegate.localLoadCache(p, args); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture localLoadCacheAsync(@Nullable IgniteBiPredicate p, @Nullable Object... args) throws CacheException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.localLoadCacheAsync(p, args); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public V getAndPutIfAbsent(K key, V val) throws CacheException, TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.getAndPutIfAbsent(key, val); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture getAndPutIfAbsentAsync(K key, V val) throws CacheException, TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.getAndPutIfAbsentAsync(key, val); } finally { - onLeave(gate, prev); + onLeave(opGate); } } @@ -346,1093 +326,937 @@ public void setCacheManager(org.apache.ignite.cache.CacheManager cacheMgr) { /** {@inheritDoc} */ @Override public boolean isLocalLocked(K key, boolean byCurrThread) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.isLocalLocked(key, byCurrThread); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public QueryCursor query(Query qry) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.query(qry); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public FieldsQueryCursor> query(SqlFieldsQuery qry) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.query(qry); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public List>> queryMultipleStatements(SqlFieldsQuery qry) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.queryMultipleStatements(qry); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public QueryCursor query(Query qry, IgniteClosure transformer) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.query(qry, transformer); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public Iterable> localEntries(CachePeekMode... peekModes) throws CacheException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.localEntries(peekModes); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public QueryMetrics queryMetrics() { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.queryMetrics(); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public void resetQueryMetrics() { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { delegate.resetQueryMetrics(); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public Collection queryDetailMetrics() { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.queryDetailMetrics(); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public void resetQueryDetailMetrics() { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { delegate.resetQueryDetailMetrics(); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public void localEvict(Collection keys) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { delegate.localEvict(keys); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public V localPeek(K key, CachePeekMode... peekModes) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.localPeek(key, peekModes); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public int size(CachePeekMode... peekModes) throws CacheException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.size(peekModes); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture sizeAsync(CachePeekMode... peekModes) throws CacheException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.sizeAsync(peekModes); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public long sizeLong(CachePeekMode... peekModes) throws CacheException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.sizeLong(peekModes); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture sizeLongAsync(CachePeekMode... peekModes) throws CacheException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.sizeLongAsync(peekModes); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public long sizeLong(int partition, CachePeekMode... peekModes) throws CacheException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.sizeLong(partition, peekModes); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture sizeLongAsync(int partition, CachePeekMode... peekModes) throws CacheException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.sizeLongAsync(partition, peekModes); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public int localSize(CachePeekMode... peekModes) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.localSize(peekModes); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public long localSizeLong(CachePeekMode... peekModes) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.localSizeLong(peekModes); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public long localSizeLong(int partition, CachePeekMode... peekModes) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.localSizeLong(partition, peekModes); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public Map> invokeAll(Map> map, Object... args) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.invokeAll(map, args); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture>> invokeAllAsync(Map> map, Object... args) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.invokeAllAsync(map, args); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public V get(K key) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.get(key); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture getAsync(K key) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.getAsync(key); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public CacheEntry getEntry(K key) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.getEntry(key); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture> getEntryAsync(K key) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.getEntryAsync(key); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public Map getAll(Set keys) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.getAll(keys); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture> getAllAsync(Set keys) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.getAllAsync(keys); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public Collection> getEntries(Set keys) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.getEntries(keys); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture>> getEntriesAsync(Set keys) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.getEntriesAsync(keys); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public Map getAllOutTx(Set keys) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.getAllOutTx(keys); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture> getAllOutTxAsync(Set keys) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.getAllOutTxAsync(keys); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public boolean containsKey(K key) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.containsKey(key); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public void loadAll(Set keys, boolean replaceExisting, CompletionListener completionListener) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { delegate.loadAll(keys, replaceExisting, completionListener); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture containsKeyAsync(K key) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.containsKeyAsync(key); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public boolean containsKeys(Set keys) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.containsKeys(keys); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture containsKeysAsync(Set keys) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.containsKeysAsync(keys); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public void put(K key, V val) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { delegate.put(key, val); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture putAsync(K key, V val) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.putAsync(key, val); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public V getAndPut(K key, V val) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.getAndPut(key, val); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture getAndPutAsync(K key, V val) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.getAndPutAsync(key, val); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public void putAll(Map map) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { delegate.putAll(map); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture putAllAsync(Map map) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.putAllAsync(map); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public boolean putIfAbsent(K key, V val) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.putIfAbsent(key, val); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture putIfAbsentAsync(K key, V val) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.putIfAbsentAsync(key, val); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public boolean remove(K key) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.remove(key); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture removeAsync(K key) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.removeAsync(key); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public boolean remove(K key, V oldVal) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.remove(key, oldVal); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture removeAsync(K key, V oldVal) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.removeAsync(key, oldVal); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public V getAndRemove(K key) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.getAndRemove(key); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture getAndRemoveAsync(K key) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.getAndRemoveAsync(key); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public boolean replace(K key, V oldVal, V newVal) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.replace(key, oldVal, newVal); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture replaceAsync(K key, V oldVal, V newVal) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.replaceAsync(key, oldVal, newVal); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public boolean replace(K key, V val) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.replace(key, val); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture replaceAsync(K key, V val) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.replaceAsync(key, val); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public V getAndReplace(K key, V val) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.getAndReplace(key, val); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture getAndReplaceAsync(K key, V val) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.getAndReplaceAsync(key, val); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public void removeAll(Set keys) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { delegate.removeAll(keys); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture removeAllAsync(Set keys) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.removeAllAsync(keys); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public void removeAll() { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { delegate.removeAll(); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture removeAllAsync() { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.removeAllAsync(); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public void clear() { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { delegate.clear(); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture clearAsync() { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.clearAsync(); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public void clear(K key) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { delegate.clear(key); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture clearAsync(K key) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.clearAsync(key); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public void clearAll(Set keys) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { delegate.clearAll(keys); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture clearAllAsync(Set keys) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.clearAllAsync(keys); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public void localClear(K key) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { delegate.localClear(key); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public void localClearAll(Set keys) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { delegate.localClearAll(keys); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public T invoke(K key, EntryProcessor entryProcessor, Object... arguments) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.invoke(key, entryProcessor, arguments); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture invokeAsync(K key, EntryProcessor entryProcessor, Object... arguments) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.invokeAsync(key, entryProcessor, arguments); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public T invoke(K key, CacheEntryProcessor entryProcessor, Object... arguments) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.invoke(key, entryProcessor, arguments); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture invokeAsync(K key, CacheEntryProcessor entryProcessor, Object... arguments) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.invokeAsync(key, entryProcessor, arguments); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public Map> invokeAll(Set keys, EntryProcessor entryProcessor, Object... args) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.invokeAll(keys, entryProcessor, args); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture>> invokeAllAsync(Set keys, EntryProcessor entryProcessor, Object... args) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.invokeAllAsync(keys, entryProcessor, args); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public Map> invokeAll(Set keys, CacheEntryProcessor entryProcessor, Object... args) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.invokeAll(keys, entryProcessor, args); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public IgniteFuture>> invokeAllAsync(Set keys, CacheEntryProcessor entryProcessor, Object... args) throws TransactionException { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.invokeAllAsync(keys, entryProcessor, args); } finally { - onLeave(gate, prev); + onLeave(opGate); } } @@ -1443,43 +1267,37 @@ public void setCacheManager(org.apache.ignite.cache.CacheManager cacheMgr) { /** {@inheritDoc} */ @Override public void registerCacheEntryListener(CacheEntryListenerConfiguration cacheEntryListenerConfiguration) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { delegate.registerCacheEntryListener(cacheEntryListenerConfiguration); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public void deregisterCacheEntryListener(CacheEntryListenerConfiguration cacheEntryListenerConfiguration) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { delegate.deregisterCacheEntryListener(cacheEntryListenerConfiguration); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public Iterator> iterator() { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.iterator(); } finally { - onLeave(gate, prev); + onLeave(opGate); } } @@ -1550,99 +1368,85 @@ public void setCacheManager(org.apache.ignite.cache.CacheManager cacheMgr) { /** {@inheritDoc} */ @Override public CacheMetrics metrics() { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.metrics(); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public CacheMetrics metrics(ClusterGroup grp) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.metrics(grp); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public CacheMetrics localMetrics() { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.localMetrics(); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public CacheMetricsMXBean mxBean() { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.mxBean(); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public CacheMetricsMXBean localMxBean() { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.localMxBean(); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public Collection lostPartitions() { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { return delegate.lostPartitions(); } finally { - onLeave(gate, prev); + onLeave(opGate); } } /** {@inheritDoc} */ @Override public void enableStatistics(boolean enabled) { - GridCacheGateway gate = gate(); - - CacheOperationContext prev = onEnter(gate, opCtx); + CacheOperationGate opGate = onEnter(); try { delegate.enableStatistics(enabled); } finally { - onLeave(gate, prev); + onLeave(opGate); } } @@ -1662,26 +1466,49 @@ public void setCacheManager(org.apache.ignite.cache.CacheManager cacheMgr) { * * @param gate Cache gateway. */ - private void checkProxyIsValid(@Nullable GridCacheGateway gate) { + private GridCacheGateway checkProxyIsValid(@Nullable GridCacheGateway gate, boolean tryRestart) { if (isProxyClosed()) throw new IllegalStateException("Cache has been closed: " + context().name()); - if (delegate instanceof IgniteCacheProxyImpl) + boolean isCacheProxy = delegate instanceof IgniteCacheProxyImpl; + + if (isCacheProxy) ((IgniteCacheProxyImpl) delegate).checkRestart(); if (gate == null) throw new IllegalStateException("Gateway is unavailable. Probably cache has been destroyed, but proxy is not closed."); + + if (isCacheProxy && tryRestart && gate.isStopped() && + context().kernalContext().gateway().getState() == GridKernalState.STARTED) { + IgniteCacheProxyImpl proxyImpl = (IgniteCacheProxyImpl) delegate; + + try { + IgniteInternalCache cache = context().kernalContext().cache().publicJCache(context().name()).internalProxy(); + + GridFutureAdapter fut = proxyImpl.opportunisticRestart(); + + if (fut == null) + proxyImpl.onRestarted(cache.context(), cache.context().cache()); + else + new IgniteFutureImpl<>(fut).get(); + + return gate(); + } catch (IgniteCheckedException ice) { + // Opportunity didn't work out. + } + } + + return gate; } /** - * @param gate Cache gateway. - * @param opCtx Cache operation context to guard. * @return Previous projection set on this thread. */ - private CacheOperationContext onEnter(@Nullable GridCacheGateway gate, CacheOperationContext opCtx) { - checkProxyIsValid(gate); + private CacheOperationGate onEnter() { + GridCacheGateway gate = checkProxyIsValid(gate(), true); - return lock ? gate.enter(opCtx) : gate.enterNoLock(opCtx); + return new CacheOperationGate(gate, + lock ? gate.enter(opCtx) : gate.enterNoLock(opCtx)); } /** @@ -1690,7 +1517,7 @@ private CacheOperationContext onEnter(@Nullable GridCacheGateway gate, Cac */ private boolean onEnterIfNoStop(@Nullable GridCacheGateway gate) { try { - checkProxyIsValid(gate); + checkProxyIsValid(gate, false); } catch (Exception e) { return false; @@ -1700,14 +1527,13 @@ private boolean onEnterIfNoStop(@Nullable GridCacheGateway gate) { } /** - * @param gate Cache gateway. - * @param opCtx Operation context to guard. + * @param opGate Operation context to guard. */ - private void onLeave(GridCacheGateway gate, CacheOperationContext opCtx) { + private void onLeave(CacheOperationGate opGate) { if (lock) - gate.leave(opCtx); + opGate.gate.leave(opGate.prev); else - gate.leaveNoLock(opCtx); + opGate.gate.leaveNoLock(opGate.prev); } /** @@ -1774,4 +1600,28 @@ private void onLeave(GridCacheGateway gate) { @Override public int hashCode() { return delegate.hashCode(); } + + /** + * Holder for gate being entered and operation context to restore. + */ + private class CacheOperationGate { + /** + * Gate being entered in this operation. + */ + public final GridCacheGateway gate; + + /** + * Operation context to restore after current operation completes. + */ + public final CacheOperationContext prev; + + /** + * @param gate Gate being entered in this operation. + * @param prev Operation context to restore after current operation completes. + */ + public CacheOperationGate(GridCacheGateway gate, CacheOperationContext prev) { + this.gate = gate; + this.prev = prev; + } + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java index b9a4b257b529f..658ca2a8de039 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java @@ -253,6 +253,13 @@ private void onEnter() { ctx.deploy().onEnter(); } + /** + * + */ + public boolean isStopped() { + return !checkState(false, false); + } + /** * */ 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 be4b0dbffaf5a..68e5b850aafed 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 @@ -1824,8 +1824,10 @@ private void setFuture(IgniteInternalFuture fut) { * Throws {@code IgniteCacheRestartingException} if proxy is restarting. */ public void checkRestart() { - if (isRestarting()) - throw new IgniteCacheRestartingException(new IgniteFutureImpl<>(restartFut.get()), "Cache is restarting: " + + GridFutureAdapter currentFut = this.restartFut.get(); + + if (currentFut != null) + throw new IgniteCacheRestartingException(new IgniteFutureImpl<>(currentFut), "Cache is restarting: " + context().name()); } @@ -1833,13 +1835,13 @@ public void checkRestart() { * @return True if proxy is restarting, false in other case. */ public boolean isRestarting() { - return restartFut != null && restartFut.get() != null; + return restartFut.get() != null; } /** * Restarts this cache proxy. */ - public void restart() { + public boolean restart() { GridFutureAdapter restartFut = new GridFutureAdapter<>(); final GridFutureAdapter curFut = this.restartFut.get(); @@ -1855,6 +1857,27 @@ public void restart() { curFut.onDone(); } }); + + return changed; + } + + /** + * If proxy is already being restarted, returns future to wait on, else restarts this cache proxy. + * + * @return Future to wait on, or null. + */ + public GridFutureAdapter opportunisticRestart() { + GridFutureAdapter restartFut = new GridFutureAdapter<>(); + + while (true) { + if (this.restartFut.compareAndSet(null, restartFut)) + return null; + + GridFutureAdapter curFut = this.restartFut.get(); + + if (curFut != null) + return curFut; + } } /** diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index 16c5d3a4829b2..b22a3970f2710 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -1616,8 +1616,6 @@ else if (msg == SPI_RECONNECT) { onDisconnected(); - notifyDiscovery(EVT_CLIENT_NODE_DISCONNECTED, topVer, locNode, allVisibleNodes()); - UUID newId = UUID.randomUUID(); U.quietAndWarn(log, "Local node will try to reconnect to cluster with new id due " + @@ -1716,8 +1714,6 @@ else if (msg == SPI_RECONNECT_FAILED) { } onDisconnected(); - - notifyDiscovery(EVT_CLIENT_NODE_DISCONNECTED, topVer, locNode, allVisibleNodes()); } UUID newId = UUID.randomUUID(); @@ -1820,6 +1816,8 @@ private void onDisconnected() { delayDiscoData.clear(); + notifyDiscovery(EVT_CLIENT_NODE_DISCONNECTED, topVer, locNode, allVisibleNodes()); + IgniteClientDisconnectedCheckedException err = new IgniteClientDisconnectedCheckedException(null, "Failed to ping node, " + "client node disconnected."); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IgniteCacheQueryCacheDestroySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IgniteCacheQueryCacheDestroySelfTest.java index dea491c50be06..d0d392b50b604 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IgniteCacheQueryCacheDestroySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IgniteCacheQueryCacheDestroySelfTest.java @@ -48,6 +48,10 @@ public class IgniteCacheQueryCacheDestroySelfTest extends GridCommonAbstractTest /** */ public static final int GRID_CNT = 3; + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + /** * The main test code. */ diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ClientReconnectAfterClusterRestartTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ClientReconnectAfterClusterRestartTest.java index 392cdc771bd6b..505d373ad0386 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ClientReconnectAfterClusterRestartTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ClientReconnectAfterClusterRestartTest.java @@ -17,8 +17,10 @@ package org.apache.ignite.internal.processors.cache; +import javax.cache.CacheException; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteClientDisconnectedException; import org.apache.ignite.IgniteDataStreamer; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryObjectBuilder; @@ -31,9 +33,8 @@ import org.apache.ignite.events.Event; import org.apache.ignite.events.EventType; import org.apache.ignite.internal.binary.BinaryMarshaller; -import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgnitePredicate; -import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.jetbrains.annotations.NotNull; @@ -119,6 +120,8 @@ public void testReconnectClient() throws Exception { checkTopology(2); + IgniteCache cache = client.getOrCreateCache(CACHE_PARAMS).withKeepBinary(); + client.events().localListen(new IgnitePredicate() { @Override public boolean apply(Event event) { @@ -161,27 +164,17 @@ public void testReconnectClient() throws Exception { startGrid(0); - assert GridTestUtils.waitForCondition(new GridAbsPredicate() { - @Override public boolean apply() { - try { - checkTopology(2); - - return true; - } catch (Exception ex) { - return false; - } - } - }, 30_000); + try { + assertNull(cache.get(1L)); + } catch (CacheException ce) { + IgniteClientDisconnectedException icde = (IgniteClientDisconnectedException)ce.getCause(); - info("Pre-insert"); + icde.reconnectFuture().get(); - streamer = client.dataStreamer("PPRB_PARAMS"); - streamer.allowOverwrite(true); - streamer.keepBinary(true); - streamer.perNodeBufferSize(10000); - streamer.perNodeParallelOperations(100); + assertNull(cache.get(1L)); + } - IgniteCache cache = client.getOrCreateCache(CACHE_PARAMS).withKeepBinary(); + info("Pre-insert"); builder = client.binary().builder("PARAMS"); builder.setField("ID", 2L); From c43049d4fc154e532497320e178d5acb468c15d9 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Mon, 16 Apr 2018 11:28:39 +0300 Subject: [PATCH 0054/1463] IGNITE-8129: MTCGA: setup default SSL context in JdbcthinConnectionSSLTest (because sometimes default SSL context may be setup by build system). This closes #3795. --- .../jdbc/thin/JdbcThinConnectionSSLTest.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSSLTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSSLTest.java index cc71f51772400..355a198c56672 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSSLTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSSLTest.java @@ -164,7 +164,14 @@ public void testConnectionUseIgniteFactory() throws Exception { * @throws Exception If failed. */ public void testDefaultContext() throws Exception { + // Store exists default SSL context to restore after test. + final SSLContext dfltSslCtx = SSLContext.getDefault(); + + // Setup default context + SSLContext.setDefault(getTestSslContextFactory().create()); + setSslCtxFactoryToCli = true; + // Factory return default SSL context sslCtxFactory = new Factory() { @Override public SSLContext create() { @@ -177,23 +184,16 @@ public void testDefaultContext() throws Exception { } }; - System.setProperty("javax.net.ssl.keyStore", CLI_KEY_STORE_PATH); - System.setProperty("javax.net.ssl.keyStorePassword", "123456"); - System.setProperty("javax.net.ssl.trustStore", TRUST_KEY_STORE_PATH); - System.setProperty("javax.net.ssl.trustStorePassword", "123456"); - startGrids(1); try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1/?sslMode=require")) { checkConnection(conn); } finally { - System.getProperties().remove("javax.net.ssl.keyStore"); - System.getProperties().remove("javax.net.ssl.keyStorePassword"); - System.getProperties().remove("javax.net.ssl.trustStore"); - System.getProperties().remove("javax.net.ssl.trustStorePassword"); - stopAllGrids(); + + // Restore SSL context. + SSLContext.setDefault(dfltSslCtx); } } From b3f252666785fddf41ef07846e034d55ae0dda71 Mon Sep 17 00:00:00 2001 From: Alexey Kukushkin Date: Mon, 16 Apr 2018 11:47:19 +0300 Subject: [PATCH 0055/1463] IGNITE-8097: Java thin client: throw handshake exception eagerly on connect phase in case of failure. This closes #3822. --- .../internal/client/thin/ReliableChannel.java | 2 + .../apache/ignite/client/FunctionalTest.java | 37 +++++++++++++++---- .../apache/ignite/client/SecurityTest.java | 22 +++++------ 3 files changed, 42 insertions(+), 19 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ReliableChannel.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ReliableChannel.java index 392b8f87ab63c..dac4320bb68a2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ReliableChannel.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ReliableChannel.java @@ -81,6 +81,8 @@ final class ReliableChannel implements AutoCloseable { primary = addrs.get(new Random().nextInt(addrs.size())); // we already verified there is at least one address + ch = chFactory.apply(new ClientChannelConfiguration(clientCfg).setAddress(primary)).get(); + for (InetSocketAddress a : addrs) if (a != primary) this.backups.add(a); 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 d69ac4d16ed03..b49f7e3a186f8 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 @@ -39,21 +39,16 @@ import org.apache.ignite.cache.PartitionLossPolicy; import org.apache.ignite.cache.QueryEntity; import org.apache.ignite.cache.QueryIndex; -import org.apache.ignite.client.ClientCache; -import org.apache.ignite.client.ClientCacheConfiguration; -import org.apache.ignite.client.Comparers; -import org.apache.ignite.client.Config; -import org.apache.ignite.client.IgniteClient; import org.apache.ignite.configuration.ClientConfiguration; -import org.apache.ignite.client.LocalIgniteCluster; -import org.apache.ignite.client.Person; import org.junit.Test; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; /** * Thin client functional tests. @@ -380,6 +375,34 @@ public void testRemoveReplace() throws Exception { } } + /** + * Test client fails on start if server is unavailable + */ + @Test + public void testClientFailsOnStart() { + ClientConnectionException expEx = null; + + try (IgniteClient ignored = Ignition.startClient(getClientConfiguration())) { + // No-op. + } + catch (ClientConnectionException connEx) { + expEx = connEx; + } + catch (Exception ex) { + fail(String.format( + "%s expected but %s was received: %s", + ClientConnectionException.class.getName(), + ex.getClass().getName(), + ex + )); + } + + assertNotNull( + String.format("%s expected but no exception was received", ClientConnectionException.class.getName()), + expEx + ); + } + /** */ private static ClientConfiguration getClientConfiguration() { return new ClientConfiguration() 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 dc57f0ca524d6..e2b11db2f66e0 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 @@ -127,25 +127,23 @@ public void testEncryption() throws Exception { /** Test valid user authentication. */ @Test - public void testInvalidUserAuthentication() throws Exception { + public void testInvalidUserAuthentication() { + Exception authError = null; + try (Ignite ignored = igniteWithAuthentication(); IgniteClient client = Ignition.startClient(new ClientConfiguration().setAddresses(Config.SERVER) .setUserName("JOE") .setUserPassword("password") ) ) { - Exception authError = null; - - try { - client.getOrCreateCache("testAuthentication"); - } - catch (Exception e) { - authError = e; - } - - assertNotNull("Authentication with invalid credentials succeeded", authError); - assertTrue("Invalid type of authentication error", authError instanceof ClientAuthenticationException); + client.getOrCreateCache("testAuthentication"); + } + catch (Exception e) { + authError = e; } + + assertNotNull("Authentication with invalid credentials succeeded", authError); + assertTrue("Invalid type of authentication error", authError instanceof ClientAuthenticationException); } /** Test valid user authentication. */ From 7173b0c48dab05be0b8e7825ae527824fcdbbc83 Mon Sep 17 00:00:00 2001 From: zaleslaw Date: Mon, 16 Apr 2018 20:20:49 +0300 Subject: [PATCH 0056/1463] IGNITE-8169: [ML] Adopt KMeans to the new Partitioned Dataset and cleanup old code this closes #3817 (cherry picked from commit 9e21cec) --- .../DatasetWithObviousStructure.java | 105 ---- .../ml/clustering/FuzzyCMeansExample.java | 134 ----- .../clustering/FuzzyCMeansLocalExample.java | 95 ---- .../KMeansClusterizationExample.java | 226 ++++++++ .../KMeansDistributedClustererExample.java | 97 ---- .../KMeansLocalClustererExample.java | 106 ---- .../ignite/ml/FuzzyCMeansModelFormat.java | 76 --- .../clustering/BaseFuzzyCMeansClusterer.java | 90 --- .../ml/clustering/BaseKMeansClusterer.java | 96 ---- .../FuzzyCMeansDistributedClusterer.java | 512 ------------------ .../clustering/FuzzyCMeansLocalClusterer.java | 254 --------- .../ml/clustering/FuzzyCMeansModel.java | 88 --- .../KMeansDistributedClusterer.java | 306 ----------- .../ml/clustering/KMeansLocalClusterer.java | 177 ------ .../ml/clustering/WeightedClusterer.java | 38 -- .../ml/clustering/{ => kmeans}/Clusterer.java | 2 +- .../{ => kmeans}/ClusterizationModel.java | 4 +- .../clustering/{ => kmeans}/KMeansModel.java | 31 +- .../kmeans}/KMeansModelFormat.java | 4 +- .../ml/clustering/kmeans/KMeansTrainer.java | 320 +++++++++++ .../ml/clustering/kmeans/package-info.java} | 21 +- .../preprocessing/LabellingMachine.java | 41 -- .../structures/preprocessing/Normalizer.java | 80 --- .../org/apache/ignite/ml/LocalModelsTest.java | 26 +- .../ml/clustering/ClusteringTestSuite.java | 7 +- .../FuzzyCMeansDistributedClustererTest.java | 180 ------ .../FuzzyCMeansLocalClustererTest.java | 202 ------- ...eansDistributedClustererTestMultiNode.java | 138 ----- ...ansDistributedClustererTestSingleNode.java | 198 ------- .../clustering/KMeansLocalClustererTest.java | 46 -- .../ignite/ml/clustering/KMeansModelTest.java | 63 +++ .../ml/clustering/KMeansTrainerTest.java | 73 +++ 32 files changed, 728 insertions(+), 3108 deletions(-) delete mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/clustering/DatasetWithObviousStructure.java delete mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/clustering/FuzzyCMeansExample.java delete mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/clustering/FuzzyCMeansLocalExample.java create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/clustering/KMeansClusterizationExample.java delete mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/clustering/KMeansDistributedClustererExample.java delete mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/clustering/KMeansLocalClustererExample.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/FuzzyCMeansModelFormat.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/clustering/BaseFuzzyCMeansClusterer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/clustering/BaseKMeansClusterer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/clustering/FuzzyCMeansDistributedClusterer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/clustering/FuzzyCMeansLocalClusterer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/clustering/FuzzyCMeansModel.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/clustering/KMeansDistributedClusterer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/clustering/KMeansLocalClusterer.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/clustering/WeightedClusterer.java rename modules/ml/src/main/java/org/apache/ignite/ml/clustering/{ => kmeans}/Clusterer.java (95%) rename modules/ml/src/main/java/org/apache/ignite/ml/clustering/{ => kmeans}/ClusterizationModel.java (92%) rename modules/ml/src/main/java/org/apache/ignite/ml/clustering/{ => kmeans}/KMeansModel.java (77%) rename modules/ml/src/main/java/org/apache/ignite/ml/{ => clustering/kmeans}/KMeansModelFormat.java (94%) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansTrainer.java rename modules/ml/src/{test/java/org/apache/ignite/ml/clustering/KMeansUtil.java => main/java/org/apache/ignite/ml/clustering/kmeans/package-info.java} (62%) delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/LabellingMachine.java delete mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/Normalizer.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/clustering/FuzzyCMeansDistributedClustererTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/clustering/FuzzyCMeansLocalClustererTest.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansDistributedClustererTestMultiNode.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansDistributedClustererTestSingleNode.java delete mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansLocalClustererTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansModelTest.java create mode 100644 modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansTrainerTest.java diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/clustering/DatasetWithObviousStructure.java b/examples/src/main/java/org/apache/ignite/examples/ml/clustering/DatasetWithObviousStructure.java deleted file mode 100644 index 5cd0e099c1a13..0000000000000 --- a/examples/src/main/java/org/apache/ignite/examples/ml/clustering/DatasetWithObviousStructure.java +++ /dev/null @@ -1,105 +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.clustering; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.ignite.ml.math.Matrix; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.VectorUtils; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; - -/** - * See KMeansDistributedClustererTestSingleNode#testClusterizationOnDatasetWithObviousStructure. - */ -class DatasetWithObviousStructure { - /** */ - private final Random rnd = new Random(123456L); - - /** Let centers be in the vertices of square. */ - private final Map centers = new HashMap<>(); - - /** Square side length. */ - private final int squareSideLen; - - /** */ - DatasetWithObviousStructure(int squareSideLen) { - this.squareSideLen = squareSideLen; - centers.put(100, new DenseLocalOnHeapVector(new double[] {0.0, 0.0})); - centers.put(900, new DenseLocalOnHeapVector(new double[] {squareSideLen, 0.0})); - centers.put(3000, new DenseLocalOnHeapVector(new double[] {0.0, squareSideLen})); - centers.put(6000, new DenseLocalOnHeapVector(new double[] {squareSideLen, squareSideLen})); - } - - /** */ - List generate(Matrix points) { - int ptsCnt = points.rowSize(); - - // Mass centers of dataset. - List massCenters = new ArrayList<>(); - - int centersCnt = centers.size(); - - List permutation = IntStream.range(0, ptsCnt).boxed().collect(Collectors.toList()); - Collections.shuffle(permutation, rnd); - - Vector[] mc = new Vector[centersCnt]; - Arrays.fill(mc, VectorUtils.zeroes(2)); - - int totalCnt = 0; - - int centIdx = 0; - massCenters.clear(); - - for (Integer count : centers.keySet()) { - for (int i = 0; i < count; i++) { - Vector pnt = getPoint(count); - - mc[centIdx] = mc[centIdx].plus(pnt); - - points.assignRow(permutation.get(totalCnt), pnt); - - totalCnt++; - } - massCenters.add(mc[centIdx].times(1 / (double)count)); - centIdx++; - } - - return massCenters; - } - - /** */ - Map centers() { - return centers; - } - - /** */ - private Vector getPoint(Integer cnt) { - Vector pnt = new DenseLocalOnHeapVector(2).assign(centers.get(cnt)); - // Perturbate point on random value. - pnt.map(val -> val + rnd.nextDouble() * squareSideLen / 100); - return pnt; - } -} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/clustering/FuzzyCMeansExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/clustering/FuzzyCMeansExample.java deleted file mode 100644 index 23aeed7abc2f7..0000000000000 --- a/examples/src/main/java/org/apache/ignite/examples/ml/clustering/FuzzyCMeansExample.java +++ /dev/null @@ -1,134 +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.clustering; - -import org.apache.ignite.Ignite; -import org.apache.ignite.Ignition; -import org.apache.ignite.examples.ExampleNodeStartup; -import org.apache.ignite.ml.clustering.BaseFuzzyCMeansClusterer; -import org.apache.ignite.ml.clustering.FuzzyCMeansDistributedClusterer; -import org.apache.ignite.ml.clustering.FuzzyCMeansModel; -import org.apache.ignite.ml.math.StorageConstants; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.distances.DistanceMeasure; -import org.apache.ignite.ml.math.distances.EuclideanDistance; -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.thread.IgniteThread; - -/** - *

    - * This example shows how to use {@link FuzzyCMeansDistributedClusterer}.

    - *

    - * Remote nodes should always be started with special configuration file which - * enables P2P class loading: {@code 'ignite.{sh|bat} examples/config/example-ignite.xml'}.

    - *

    - * Alternatively you can run {@link ExampleNodeStartup} in another JVM which will start node - * with {@code examples/config/example-ignite.xml} configuration.

    - */ -public final class FuzzyCMeansExample { - /** - * Executes example. - * - * @param args Command line arguments, none required. - */ - public static void main(String[] args) throws InterruptedException { - System.out.println(">>> Fuzzy C-Means usage example started."); - - // Start ignite grid. - try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { - System.out.println(">>> Ignite grid started."); - - // Start new Ignite thread. - IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), - FuzzyCMeansExample.class.getSimpleName(), - () -> { - // Distance measure that computes distance between two points. - DistanceMeasure distanceMeasure = new EuclideanDistance(); - - // "Fuzziness" - specific constant that is used in membership calculation (1.0+-eps ~ K-Means). - double exponentialWeight = 2.0; - - // Condition that indicated when algorithm must stop. - // In this example algorithm stops if memberships have changed insignificantly. - BaseFuzzyCMeansClusterer.StopCondition stopCond = - BaseFuzzyCMeansClusterer.StopCondition.STABLE_MEMBERSHIPS; - - // Maximum difference between new and old membership values with which algorithm will continue to work. - double maxDelta = 0.01; - - // The maximum number of FCM iterations. - int maxIterations = 50; - - // Value that is used to initialize random numbers generator. You can choose it randomly. - Long seed = null; - - // Number of steps of primary centers selection (more steps more candidates). - int initializationSteps = 2; - - // Number of K-Means iteration that is used to choose required number of primary centers from candidates. - int kMeansMaxIterations = 50; - - // Create new distributed clusterer with parameters described above. - System.out.println(">>> Create new Distributed Fuzzy C-Means clusterer."); - FuzzyCMeansDistributedClusterer clusterer = new FuzzyCMeansDistributedClusterer( - distanceMeasure, exponentialWeight, stopCond, maxDelta, maxIterations, - seed, initializationSteps, kMeansMaxIterations); - - // Create sample data. - double[][] points = new double[][] { - {-10, -10}, {-9, -11}, {-10, -9}, {-11, -9}, - {10, 10}, {9, 11}, {10, 9}, {11, 9}, - {-10, 10}, {-9, 11}, {-10, 9}, {-11, 9}, - {10, -10}, {9, -11}, {10, -9}, {11, -9}}; - - // Initialize matrix of data points. Each row contains one point. - int rows = points.length; - int cols = points[0].length; - - System.out.println(">>> Create the matrix that contains sample points."); - SparseDistributedMatrix pntMatrix = new SparseDistributedMatrix(rows, cols, - StorageConstants.ROW_STORAGE_MODE, StorageConstants.RANDOM_ACCESS_MODE); - - // Store points into matrix. - pntMatrix.assign(points); - - // Call clusterization method with some number of centers. - // It returns model that can predict results for new points. - System.out.println(">>> Perform clusterization."); - int numCenters = 4; - FuzzyCMeansModel mdl = clusterer.cluster(pntMatrix, numCenters); - - // You can also get centers of clusters that is computed by Fuzzy C-Means algorithm. - Vector[] centers = mdl.centers(); - - String res = ">>> Results:\n" - + ">>> 1st center: " + centers[0].get(0) + " " + centers[0].get(1) + "\n" - + ">>> 2nd center: " + centers[1].get(0) + " " + centers[1].get(1) + "\n" - + ">>> 3rd center: " + centers[2].get(0) + " " + centers[2].get(1) + "\n" - + ">>> 4th center: " + centers[3].get(0) + " " + centers[3].get(1) + "\n"; - - System.out.println(res); - - pntMatrix.destroy(); - }); - - igniteThread.start(); - igniteThread.join(); - } - } -} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/clustering/FuzzyCMeansLocalExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/clustering/FuzzyCMeansLocalExample.java deleted file mode 100644 index 5c1753ad37ffb..0000000000000 --- a/examples/src/main/java/org/apache/ignite/examples/ml/clustering/FuzzyCMeansLocalExample.java +++ /dev/null @@ -1,95 +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.clustering; - -import org.apache.ignite.ml.clustering.BaseFuzzyCMeansClusterer; -import org.apache.ignite.ml.clustering.FuzzyCMeansLocalClusterer; -import org.apache.ignite.ml.clustering.FuzzyCMeansModel; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.distances.DistanceMeasure; -import org.apache.ignite.ml.math.distances.EuclideanDistance; -import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; - -/** - * This example shows how to use {@link FuzzyCMeansLocalClusterer}. - */ -public final class FuzzyCMeansLocalExample { - /** - * Executes example. - * - * @param args Command line arguments, none required. - */ - public static void main(String[] args) { - System.out.println(">>> Local Fuzzy C-Means usage example started."); - - // Distance measure that computes distance between two points. - DistanceMeasure distanceMeasure = new EuclideanDistance(); - - // "Fuzziness" - specific constant that is used in membership calculation (1.0+-eps ~ K-Means). - double exponentialWeight = 2.0; - - // Condition that indicated when algorithm must stop. - // In this example algorithm stops if memberships have changed insignificantly. - BaseFuzzyCMeansClusterer.StopCondition stopCond = - BaseFuzzyCMeansClusterer.StopCondition.STABLE_MEMBERSHIPS; - - // Maximum difference between new and old membership values with which algorithm will continue to work. - double maxDelta = 0.01; - - // The maximum number of FCM iterations. - int maxIterations = 50; - - // Value that is used to initialize random numbers generator. You can choose it randomly. - Long seed = null; - - // Create new distributed clusterer with parameters described above. - System.out.println(">>> Create new Local Fuzzy C-Means clusterer."); - FuzzyCMeansLocalClusterer clusterer = new FuzzyCMeansLocalClusterer(distanceMeasure, - exponentialWeight, stopCond, - maxDelta, maxIterations, seed); - - // Create sample data. - double[][] points = new double[][] { - {-10, -10}, {-9, -11}, {-10, -9}, {-11, -9}, - {10, 10}, {9, 11}, {10, 9}, {11, 9}, - {-10, 10}, {-9, 11}, {-10, 9}, {-11, 9}, - {10, -10}, {9, -11}, {10, -9}, {11, -9}}; - - // Initialize matrix of data points. Each row contains one point. - System.out.println(">>> Create the matrix that contains sample points."); - // Store points into matrix. - DenseLocalOnHeapMatrix pntMatrix = new DenseLocalOnHeapMatrix(points); - - // Call clusterization method with some number of centers. - // It returns model that can predict results for new points. - System.out.println(">>> Perform clusterization."); - int numCenters = 4; - FuzzyCMeansModel mdl = clusterer.cluster(pntMatrix, numCenters); - - // You can also get centers of clusters that is computed by Fuzzy C-Means algorithm. - Vector[] centers = mdl.centers(); - - String res = ">>> Results:\n" - + ">>> 1st center: " + centers[0].get(0) + " " + centers[0].get(1) + "\n" - + ">>> 2nd center: " + centers[1].get(0) + " " + centers[1].get(1) + "\n" - + ">>> 3rd center: " + centers[2].get(0) + " " + centers[2].get(1) + "\n" - + ">>> 4th center: " + centers[3].get(0) + " " + centers[3].get(1) + "\n"; - - System.out.println(res); - } -} 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 new file mode 100644 index 0000000000000..8825ebbd4a913 --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/clustering/KMeansClusterizationExample.java @@ -0,0 +1,226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.clustering; + +import java.util.Arrays; +import java.util.UUID; +import javax.cache.Cache; +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.cache.query.QueryCursor; +import org.apache.ignite.cache.query.ScanQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder; +import org.apache.ignite.ml.knn.classification.KNNClassificationTrainer; +import org.apache.ignite.ml.math.Tracer; +import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; +import org.apache.ignite.ml.clustering.kmeans.KMeansModel; +import org.apache.ignite.ml.clustering.kmeans.KMeansTrainer; +import org.apache.ignite.thread.IgniteThread; + +/** + * Run kNN multi-class classification trainer over distributed dataset. + * + * @see KNNClassificationTrainer + */ +public class KMeansClusterizationExample { + /** Run example. */ + public static void main(String[] args) throws InterruptedException { + System.out.println(); + System.out.println(">>> KMeans clustering algorithm over cached dataset usage example started."); + // Start ignite grid. + try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { + System.out.println(">>> Ignite grid started."); + + IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), + KMeansClusterizationExample.class.getSimpleName(), () -> { + IgniteCache dataCache = getTestCache(ignite); + + KMeansTrainer trainer = new KMeansTrainer() + .withSeed(7867L); + + KMeansModel mdl = trainer.fit( + new CacheBasedDatasetBuilder<>(ignite, dataCache), + (k, v) -> Arrays.copyOfRange(v, 1, v.length), + (k, v) -> v[0] + ); + + System.out.println(">>> KMeans centroids"); + Tracer.showAscii(mdl.centers()[0]); + Tracer.showAscii(mdl.centers()[1]); + System.out.println(">>>"); + + System.out.println(">>> -----------------------------------"); + System.out.println(">>> | Predicted cluster\t| Real Label\t|"); + System.out.println(">>> -----------------------------------"); + + int amountOfErrors = 0; + int totalAmount = 0; + + try (QueryCursor> observations = dataCache.query(new ScanQuery<>())) { + for (Cache.Entry observation : observations) { + double[] val = observation.getValue(); + double[] inputs = Arrays.copyOfRange(val, 1, val.length); + double groundTruth = val[0]; + + double prediction = mdl.apply(new DenseLocalOnHeapVector(inputs)); + + totalAmount++; + if (groundTruth != prediction) + amountOfErrors++; + + System.out.printf(">>> | %.4f\t\t\t| %.4f\t\t|\n", prediction, groundTruth); + } + + System.out.println(">>> ---------------------------------"); + + System.out.println("\n>>> Absolute amount of errors " + amountOfErrors); + System.out.println("\n>>> Accuracy " + (1 - amountOfErrors / (double)totalAmount)); + } + }); + + igniteThread.start(); + igniteThread.join(); + } + } + + /** + * Fills cache with data and returns it. + * + * @param ignite Ignite instance. + * @return Filled Ignite Cache. + */ + private static IgniteCache getTestCache(Ignite ignite) { + CacheConfiguration cacheConfiguration = new CacheConfiguration<>(); + cacheConfiguration.setName("TEST_" + UUID.randomUUID()); + 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; + } + + /** The Iris dataset. */ + private static final double[][] data = { + {0, 5.1, 3.5, 1.4, 0.2}, + {0, 4.9, 3, 1.4, 0.2}, + {0, 4.7, 3.2, 1.3, 0.2}, + {0, 4.6, 3.1, 1.5, 0.2}, + {0, 5, 3.6, 1.4, 0.2}, + {0, 5.4, 3.9, 1.7, 0.4}, + {0, 4.6, 3.4, 1.4, 0.3}, + {0, 5, 3.4, 1.5, 0.2}, + {0, 4.4, 2.9, 1.4, 0.2}, + {0, 4.9, 3.1, 1.5, 0.1}, + {0, 5.4, 3.7, 1.5, 0.2}, + {0, 4.8, 3.4, 1.6, 0.2}, + {0, 4.8, 3, 1.4, 0.1}, + {0, 4.3, 3, 1.1, 0.1}, + {0, 5.8, 4, 1.2, 0.2}, + {0, 5.7, 4.4, 1.5, 0.4}, + {0, 5.4, 3.9, 1.3, 0.4}, + {0, 5.1, 3.5, 1.4, 0.3}, + {0, 5.7, 3.8, 1.7, 0.3}, + {0, 5.1, 3.8, 1.5, 0.3}, + {0, 5.4, 3.4, 1.7, 0.2}, + {0, 5.1, 3.7, 1.5, 0.4}, + {0, 4.6, 3.6, 1, 0.2}, + {0, 5.1, 3.3, 1.7, 0.5}, + {0, 4.8, 3.4, 1.9, 0.2}, + {0, 5, 3, 1.6, 0.2}, + {0, 5, 3.4, 1.6, 0.4}, + {0, 5.2, 3.5, 1.5, 0.2}, + {0, 5.2, 3.4, 1.4, 0.2}, + {0, 4.7, 3.2, 1.6, 0.2}, + {0, 4.8, 3.1, 1.6, 0.2}, + {0, 5.4, 3.4, 1.5, 0.4}, + {0, 5.2, 4.1, 1.5, 0.1}, + {0, 5.5, 4.2, 1.4, 0.2}, + {0, 4.9, 3.1, 1.5, 0.1}, + {0, 5, 3.2, 1.2, 0.2}, + {0, 5.5, 3.5, 1.3, 0.2}, + {0, 4.9, 3.1, 1.5, 0.1}, + {0, 4.4, 3, 1.3, 0.2}, + {0, 5.1, 3.4, 1.5, 0.2}, + {0, 5, 3.5, 1.3, 0.3}, + {0, 4.5, 2.3, 1.3, 0.3}, + {0, 4.4, 3.2, 1.3, 0.2}, + {0, 5, 3.5, 1.6, 0.6}, + {0, 5.1, 3.8, 1.9, 0.4}, + {0, 4.8, 3, 1.4, 0.3}, + {0, 5.1, 3.8, 1.6, 0.2}, + {0, 4.6, 3.2, 1.4, 0.2}, + {0, 5.3, 3.7, 1.5, 0.2}, + {0, 5, 3.3, 1.4, 0.2}, + {1, 7, 3.2, 4.7, 1.4}, + {1, 6.4, 3.2, 4.5, 1.5}, + {1, 6.9, 3.1, 4.9, 1.5}, + {1, 5.5, 2.3, 4, 1.3}, + {1, 6.5, 2.8, 4.6, 1.5}, + {1, 5.7, 2.8, 4.5, 1.3}, + {1, 6.3, 3.3, 4.7, 1.6}, + {1, 4.9, 2.4, 3.3, 1}, + {1, 6.6, 2.9, 4.6, 1.3}, + {1, 5.2, 2.7, 3.9, 1.4}, + {1, 5, 2, 3.5, 1}, + {1, 5.9, 3, 4.2, 1.5}, + {1, 6, 2.2, 4, 1}, + {1, 6.1, 2.9, 4.7, 1.4}, + {1, 5.6, 2.9, 3.6, 1.3}, + {1, 6.7, 3.1, 4.4, 1.4}, + {1, 5.6, 3, 4.5, 1.5}, + {1, 5.8, 2.7, 4.1, 1}, + {1, 6.2, 2.2, 4.5, 1.5}, + {1, 5.6, 2.5, 3.9, 1.1}, + {1, 5.9, 3.2, 4.8, 1.8}, + {1, 6.1, 2.8, 4, 1.3}, + {1, 6.3, 2.5, 4.9, 1.5}, + {1, 6.1, 2.8, 4.7, 1.2}, + {1, 6.4, 2.9, 4.3, 1.3}, + {1, 6.6, 3, 4.4, 1.4}, + {1, 6.8, 2.8, 4.8, 1.4}, + {1, 6.7, 3, 5, 1.7}, + {1, 6, 2.9, 4.5, 1.5}, + {1, 5.7, 2.6, 3.5, 1}, + {1, 5.5, 2.4, 3.8, 1.1}, + {1, 5.5, 2.4, 3.7, 1}, + {1, 5.8, 2.7, 3.9, 1.2}, + {1, 6, 2.7, 5.1, 1.6}, + {1, 5.4, 3, 4.5, 1.5}, + {1, 6, 3.4, 4.5, 1.6}, + {1, 6.7, 3.1, 4.7, 1.5}, + {1, 6.3, 2.3, 4.4, 1.3}, + {1, 5.6, 3, 4.1, 1.3}, + {1, 5.5, 2.5, 4, 1.3}, + {1, 5.5, 2.6, 4.4, 1.2}, + {1, 6.1, 3, 4.6, 1.4}, + {1, 5.8, 2.6, 4, 1.2}, + {1, 5, 2.3, 3.3, 1}, + {1, 5.6, 2.7, 4.2, 1.3}, + {1, 5.7, 3, 4.2, 1.2}, + {1, 5.7, 2.9, 4.2, 1.3}, + {1, 6.2, 2.9, 4.3, 1.3}, + {1, 5.1, 2.5, 3, 1.1}, + {1, 5.7, 2.8, 4.1, 1.3}, + }; +} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/clustering/KMeansDistributedClustererExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/clustering/KMeansDistributedClustererExample.java deleted file mode 100644 index f8709e6da45b6..0000000000000 --- a/examples/src/main/java/org/apache/ignite/examples/ml/clustering/KMeansDistributedClustererExample.java +++ /dev/null @@ -1,97 +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.clustering; - -import java.util.Arrays; -import java.util.List; -import org.apache.ignite.Ignite; -import org.apache.ignite.Ignition; -import org.apache.ignite.examples.ExampleNodeStartup; -import org.apache.ignite.examples.ml.math.matrix.SparseDistributedMatrixExample; -import org.apache.ignite.ml.clustering.KMeansDistributedClusterer; -import org.apache.ignite.ml.math.StorageConstants; -import org.apache.ignite.ml.math.Tracer; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.distances.EuclideanDistance; -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.thread.IgniteThread; - -/** - *

    - * Example of using {@link KMeansDistributedClusterer}.

    - *

    - * Note that in this example we cannot guarantee order in which nodes return results of intermediate - * computations and therefore algorithm can return different results.

    - *

    - * Remote nodes should always be started with special configuration file which - * enables P2P class loading: {@code 'ignite.{sh|bat} examples/config/example-ignite.xml'}.

    - *

    - * Alternatively you can run {@link ExampleNodeStartup} in another JVM which will start node - * with {@code examples/config/example-ignite.xml} configuration.

    - */ -public class KMeansDistributedClustererExample { - /** - * Executes example. - * - * @param args Command line arguments, none required. - */ - public static void main(String[] args) throws InterruptedException { - // IMPL NOTE based on KMeansDistributedClustererTestSingleNode#testClusterizationOnDatasetWithObviousStructure - System.out.println(">>> K-means distributed clusterer example started."); - - // Start ignite grid. - try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) { - System.out.println(">>> Ignite grid started."); - - // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread - // because we create ignite cache internally. - IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), - SparseDistributedMatrixExample.class.getSimpleName(), () -> { - - int ptsCnt = 10000; - - SparseDistributedMatrix points = new SparseDistributedMatrix(ptsCnt, 2, - StorageConstants.ROW_STORAGE_MODE, StorageConstants.RANDOM_ACCESS_MODE); - - DatasetWithObviousStructure dataset = new DatasetWithObviousStructure(10000); - - List massCenters = dataset.generate(points); - - EuclideanDistance dist = new EuclideanDistance(); - - KMeansDistributedClusterer clusterer = new KMeansDistributedClusterer(dist, 3, 100, 1L); - - Vector[] resCenters = clusterer.cluster(points, 4).centers(); - - System.out.println("Mass centers:"); - massCenters.forEach(Tracer::showAscii); - - System.out.println("Cluster centers:"); - Arrays.asList(resCenters).forEach(Tracer::showAscii); - - points.destroy(); - - System.out.println("\n>>> K-means distributed clusterer example completed."); - }); - - igniteThread.start(); - - igniteThread.join(); - } - } -} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/clustering/KMeansLocalClustererExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/clustering/KMeansLocalClustererExample.java deleted file mode 100644 index 28ca9d940d029..0000000000000 --- a/examples/src/main/java/org/apache/ignite/examples/ml/clustering/KMeansLocalClustererExample.java +++ /dev/null @@ -1,106 +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.clustering; - -import java.util.Arrays; -import java.util.Comparator; -import java.util.List; -import org.apache.ignite.ml.clustering.KMeansLocalClusterer; -import org.apache.ignite.ml.clustering.KMeansModel; -import org.apache.ignite.ml.math.Tracer; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.distances.DistanceMeasure; -import org.apache.ignite.ml.math.distances.EuclideanDistance; -import org.apache.ignite.ml.math.functions.Functions; -import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; - -/** - * Example of using {@link KMeansLocalClusterer}. - */ -public class KMeansLocalClustererExample { - /** - * Executes example. - * - * @param args Command line arguments, none required. - */ - public static void main(String[] args) { - // IMPL NOTE based on KMeansDistributedClustererTestSingleNode#testClusterizationOnDatasetWithObviousStructure - System.out.println(">>> K-means local clusterer example started."); - - int ptsCnt = 10000; - DenseLocalOnHeapMatrix points = new DenseLocalOnHeapMatrix(ptsCnt, 2); - - DatasetWithObviousStructure dataset = new DatasetWithObviousStructure(10000); - - List massCenters = dataset.generate(points); - - EuclideanDistance dist = new EuclideanDistance(); - OrderedNodesComparator comp = new OrderedNodesComparator( - dataset.centers().values().toArray(new Vector[] {}), dist); - - massCenters.sort(comp); - - KMeansLocalClusterer clusterer = new KMeansLocalClusterer(dist, 100, 1L); - - KMeansModel mdl = clusterer.cluster(points, 4); - Vector[] resCenters = mdl.centers(); - Arrays.sort(resCenters, comp); - - System.out.println("Mass centers:"); - massCenters.forEach(Tracer::showAscii); - - System.out.println("Cluster centers:"); - Arrays.asList(resCenters).forEach(Tracer::showAscii); - - System.out.println("\n>>> K-means local clusterer example completed."); - } - - /** */ - private static class OrderedNodesComparator implements Comparator { - /** */ - private final DistanceMeasure measure; - - /** */ - List orderedNodes; - - /** */ - OrderedNodesComparator(Vector[] orderedNodes, DistanceMeasure measure) { - this.orderedNodes = Arrays.asList(orderedNodes); - this.measure = measure; - } - - /** */ - private int findClosestNodeIndex(Vector v) { - return Functions.argmin(orderedNodes, v1 -> measure.compute(v1, v)).get1(); - } - - /** */ - @Override public int compare(Vector v1, Vector v2) { - int ind1 = findClosestNodeIndex(v1); - int ind2 = findClosestNodeIndex(v2); - - int signum = (int)Math.signum(ind1 - ind2); - - if (signum != 0) - return signum; - - return (int)Math.signum(orderedNodes.get(ind1).minus(v1).kNorm(2) - - orderedNodes.get(ind2).minus(v2).kNorm(2)); - } - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/FuzzyCMeansModelFormat.java b/modules/ml/src/main/java/org/apache/ignite/ml/FuzzyCMeansModelFormat.java deleted file mode 100644 index cc3d9b3cf4309..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/FuzzyCMeansModelFormat.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.ml; - -import java.io.Serializable; -import java.util.Arrays; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.distances.DistanceMeasure; - -/** Fuzzy C-Means model representation. */ -public class FuzzyCMeansModelFormat implements Serializable { - /** Centers of clusters. */ - private final Vector[] centers; - - /** Distance measure. */ - private final DistanceMeasure measure; - - /** - * Constructor that retains result of clusterization and distance measure. - * - * @param centers Centers found while clusterization. - * @param measure Distance measure. - */ - public FuzzyCMeansModelFormat(Vector[] centers, DistanceMeasure measure) { - this.centers = centers; - this.measure = measure; - } - - /** Distance measure used while clusterization. */ - public DistanceMeasure getMeasure() { - return measure; - } - - /** Get cluster centers. */ - public Vector[] getCenters() { - return centers; - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - int res = 1; - - res = res * 37 + measure.hashCode(); - res = res * 37 + Arrays.hashCode(centers); - - return res; - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object obj) { - if (this == obj) - return true; - - if (obj == null || getClass() != obj.getClass()) - return false; - - FuzzyCMeansModelFormat that = (FuzzyCMeansModelFormat) obj; - - return measure.equals(that.measure) && Arrays.deepEquals(centers, that.centers); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/BaseFuzzyCMeansClusterer.java b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/BaseFuzzyCMeansClusterer.java deleted file mode 100644 index 2b2febfcd2c75..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/BaseFuzzyCMeansClusterer.java +++ /dev/null @@ -1,90 +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.clustering; - -import org.apache.ignite.ml.math.Matrix; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.distances.DistanceMeasure; -import org.apache.ignite.ml.math.exceptions.ConvergenceException; -import org.apache.ignite.ml.math.exceptions.MathIllegalArgumentException; - -/** The abstract class that defines the basic interface of Fuzzy C-Means clusterers */ -public abstract class BaseFuzzyCMeansClusterer implements Clusterer { - /** Distance measure. */ - protected DistanceMeasure measure; - - /** Specific constant which is used in calculating of membership matrix. */ - protected double exponentialWeight; - - /** The maximum distance between old and new centers or the maximum difference between new and old membership matrix - * elements for which algorithm must stop. */ - protected double maxDelta; - - /** The flag that tells when algorithm should stop. */ - protected StopCondition stopCond; - - /** - * Constructor that stores some required parameters. - * - * @param measure Distance measure. - * @param exponentialWeight Specific constant which is used in calculating of membership matrix. - * @param stopCond Flag that tells when algorithm should stop. - * @param maxDelta The maximum distance between old and new centers or maximum difference between new and old - * membership matrix elements for which algorithm must stop. - */ - protected BaseFuzzyCMeansClusterer(DistanceMeasure measure, double exponentialWeight, StopCondition stopCond, - double maxDelta) { - this.measure = measure; - this.exponentialWeight = exponentialWeight; - this.stopCond = stopCond; - this.maxDelta = maxDelta; - } - - /** - * Perform a cluster analysis on the given set of points. - * - * @param points The set of points. - * @return A list of clusters. - * @throws MathIllegalArgumentException If points are null or the number of data points is not compatible with this - * clusterer. - * @throws ConvergenceException If the algorithm has not yet converged after the maximum number of iterations has - * been exceeded. - */ - public abstract FuzzyCMeansModel cluster(T points, int k); - - /** - * Calculates the distance between two vectors. * with the configured {@link DistanceMeasure}. - * - * @return The distance between two points. - */ - protected double distance(final Vector v1, final Vector v2) { - return measure.compute(v1, v2); - } - - /** Enumeration that contains different conditions under which algorithm must stop. */ - public enum StopCondition { - /** Algorithm stops if the maximum distance between new and old centers is less than {@link #maxDelta}. */ - STABLE_CENTERS, - - /** - * Algorithm stops if the maximum difference between elements of new and old membership matrix is less than - * {@link #maxDelta}. - */ - STABLE_MEMBERSHIPS - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/BaseKMeansClusterer.java b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/BaseKMeansClusterer.java deleted file mode 100644 index 521437c71b2f6..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/BaseKMeansClusterer.java +++ /dev/null @@ -1,96 +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.clustering; - -import java.util.List; -import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.ml.math.Matrix; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.distances.DistanceMeasure; -import org.apache.ignite.ml.math.exceptions.ConvergenceException; -import org.apache.ignite.ml.math.exceptions.MathIllegalArgumentException; - -/** - * This class is partly based on the corresponding class from Apache Common Math lib. - */ -public abstract class BaseKMeansClusterer implements Clusterer { - /** The distance measure to use. */ - private DistanceMeasure measure; - - /** - * Build a new clusterer with the given {@link DistanceMeasure}. - * - * @param measure the distance measure to use - */ - protected BaseKMeansClusterer(final DistanceMeasure measure) { - this.measure = measure; - } - - /** - * Perform a cluster analysis on the given set of points. - * - * @param points the set of points - * @return a {@link List} of clusters - * @throws MathIllegalArgumentException if points are null or the number of data points is not compatible with this - * clusterer - * @throws ConvergenceException if the algorithm has not yet converged after the maximum number of iterations has - * been exceeded - */ - public abstract KMeansModel cluster(T points, int k) - throws MathIllegalArgumentException, ConvergenceException; - - /** - * Returns the {@link DistanceMeasure} instance used by this clusterer. - * - * @return the distance measure - */ - public DistanceMeasure getDistanceMeasure() { - return measure; - } - - /** - * Calculates the distance between two vectors. - * with the configured {@link DistanceMeasure}. - * - * @return the distance between the two clusterables - */ - protected double distance(final Vector v1, final Vector v2) { - return measure.compute(v1, v2); - } - - /** - * Find the closest cluster center index and distance to it from a given point. - * - * @param centers Centers to look in. - * @param pnt Point. - */ - protected IgniteBiTuple findClosest(Vector[] centers, Vector pnt) { - double bestDistance = Double.POSITIVE_INFINITY; - int bestInd = 0; - - for (int i = 0; i < centers.length; i++) { - double dist = distance(centers[i], pnt); - if (dist < bestDistance) { - bestDistance = dist; - bestInd = i; - } - } - - return new IgniteBiTuple<>(bestInd, bestDistance); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/FuzzyCMeansDistributedClusterer.java b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/FuzzyCMeansDistributedClusterer.java deleted file mode 100644 index 8823c10676485..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/FuzzyCMeansDistributedClusterer.java +++ /dev/null @@ -1,512 +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.clustering; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.stream.Collectors; -import javax.cache.Cache; -import org.apache.ignite.internal.util.GridArgumentCheck; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.VectorUtils; -import org.apache.ignite.ml.math.distances.DistanceMeasure; -import org.apache.ignite.ml.math.distributed.CacheUtils; -import org.apache.ignite.ml.math.distributed.keys.impl.SparseMatrixKey; -import org.apache.ignite.ml.math.exceptions.ConvergenceException; -import org.apache.ignite.ml.math.exceptions.MathIllegalArgumentException; -import org.apache.ignite.ml.math.functions.Functions; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.math.functions.IgniteSupplier; -import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.ml.math.impls.storage.matrix.SparseDistributedMatrixStorage; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; -import org.apache.ignite.ml.math.util.MatrixUtil; - -/** This class implements distributed version of Fuzzy C-Means clusterization of equal-weighted points. */ -public class FuzzyCMeansDistributedClusterer extends BaseFuzzyCMeansClusterer { - /** Random numbers generator which is used in centers selection. */ - private Random rnd; - - /** The value that is used to initialize random numbers generator. */ - private long seed; - - /** The number of initialization steps each of which adds some number of candidates for being a center. */ - private int initSteps; - - /** The maximum number of iterations of K-Means algorithm which selects the required number of centers. */ - private int kMeansMaxIterations; - - /** The maximum number of FCM iterations. */ - private int cMeansMaxIterations; - - /** - * Constructor that retains all required parameters. - * - * @param measure Distance measure. - * @param exponentialWeight Specific constant which is used in calculating of membership matrix. - * @param stopCond Flag that tells when algorithm should stop. - * @param maxDelta The maximum distance between old and new centers or maximum difference between new and old - * membership matrix elements for which algorithm must stop. - * @param cMeansMaxIterations The maximum number of FCM iterations. - * @param seed Seed for random numbers generator. - * @param initSteps Number of steps of primary centers selection (the more steps, the more candidates). - * @param kMeansMaxIterations The maximum number of K-Means iteration in primary centers selection. - */ - public FuzzyCMeansDistributedClusterer(DistanceMeasure measure, double exponentialWeight, - StopCondition stopCond, double maxDelta, int cMeansMaxIterations, - Long seed, int initSteps, int kMeansMaxIterations) { - super(measure, exponentialWeight, stopCond, maxDelta); - - this.seed = seed != null ? seed : new Random().nextLong(); - this.initSteps = initSteps; - this.cMeansMaxIterations = cMeansMaxIterations; - this.kMeansMaxIterations = kMeansMaxIterations; - rnd = new Random(this.seed); - } - - /** {@inheritDoc} */ - @Override public FuzzyCMeansModel cluster(SparseDistributedMatrix points, int k) - throws MathIllegalArgumentException, ConvergenceException { - GridArgumentCheck.notNull(points, "points"); - - if (k < 2) - throw new MathIllegalArgumentException("The number of clusters is less than 2"); - - Vector[] centers = initializeCenters(points, k); - - MembershipsAndSums membershipsAndSums = null; - - int iteration = 0; - boolean finished = false; - while (!finished && iteration < cMeansMaxIterations) { - MembershipsAndSums newMembershipsAndSums = calculateMembership(points, centers); - Vector[] newCenters = calculateNewCenters(points, newMembershipsAndSums, k); - - if (stopCond == StopCondition.STABLE_CENTERS) - finished = isFinished(centers, newCenters); - else - finished = isFinished(membershipsAndSums, newMembershipsAndSums); - - centers = newCenters; - membershipsAndSums = newMembershipsAndSums; - - iteration++; - } - - if (iteration == cMeansMaxIterations) - throw new ConvergenceException("Fuzzy C-Means algorithm has not converged after " + - Integer.toString(iteration) + " iterations"); - - return new FuzzyCMeansModel(centers, measure); - } - - /** - * Choose k primary centers from source points. - * - * @param points Matrix with source points. - * @param k Number of centers. - * @return Array of primary centers. - */ - private Vector[] initializeCenters(SparseDistributedMatrix points, int k) { - int pointsNum = points.rowSize(); - - Vector firstCenter = points.viewRow(rnd.nextInt(pointsNum)); - - List centers = new ArrayList<>(); - List newCenters = new ArrayList<>(); - - centers.add(firstCenter); - newCenters.add(firstCenter); - - ConcurrentHashMap costs = new ConcurrentHashMap<>(); - - int step = 0; - UUID uuid = points.getUUID(); - String cacheName = ((SparseDistributedMatrixStorage) points.getStorage()).cacheName(); - - while(step < initSteps) { - ConcurrentHashMap newCosts = getNewCosts(cacheName, uuid, newCenters); - - for (Integer key : newCosts.keySet()) - costs.merge(key, newCosts.get(key), Math::min); - - double costsSum = costs.values().stream().mapToDouble(Double::valueOf).sum(); - - newCenters = getNewCenters(cacheName, uuid, costs, costsSum, k); - centers.addAll(newCenters); - - step++; - } - - return chooseKCenters(cacheName, uuid, centers, k); - } - - /** - * Calculate new distances from each point to the nearest center. - * - * @param cacheName Cache name of point matrix. - * @param uuid Uuid of point matrix. - * @param newCenters The list of centers that was added on previous step. - * @return Hash map with distances. - */ - private ConcurrentHashMap getNewCosts(String cacheName, UUID uuid, - List newCenters) { - return CacheUtils.distributedFold(cacheName, - (IgniteBiFunction>, - ConcurrentHashMap, - ConcurrentHashMap>)(vectorWithIndex, map) -> { - Vector vector = VectorUtils.fromMap(vectorWithIndex.getValue(), false); - - for (Vector center : newCenters) - map.merge(vectorWithIndex.getKey().index(), distance(vector, center), Functions.MIN); - - return map; - }, - key -> key.dataStructureId().equals(uuid), - (map1, map2) -> { - map1.putAll(map2); - return map1; - }, - ConcurrentHashMap::new); - } - - /** - * Choose some number of center candidates from source points according to their costs. - * - * @param cacheName Cache name of point matrix. - * @param uuid Uuid of point matrix. - * @param costs Hash map with costs (distances to nearest center). - * @param costsSum The sum of costs. - * @param k The estimated number of centers. - * @return The list of new candidates. - */ - private List getNewCenters(String cacheName, UUID uuid, - ConcurrentHashMap costs, double costsSum, int k) { - return CacheUtils.distributedFold(cacheName, - (IgniteBiFunction>, - List, - List>)(vectorWithIndex, centers) -> { - Integer idx = vectorWithIndex.getKey().index(); - Vector vector = VectorUtils.fromMap(vectorWithIndex.getValue(), false); - - double probability = (costs.get(idx) * 2.0 * k) / costsSum; - - if (rnd.nextDouble() < probability) - centers.add(vector); - - return centers; - }, - key -> key.dataStructureId().equals(uuid), - (list1, list2) -> { - list1.addAll(list2); - return list1; - }, - ArrayList::new); - } - - /** - * Weight candidates and use K-Means to choose required number of them. - * - * @param cacheName Cache name of the point matrix. - * @param uuid Uuid of the point matrix. - * @param centers The list of candidates. - * @param k The estimated number of centers. - * @return {@code k} centers. - */ - private Vector[] chooseKCenters(String cacheName, UUID uuid, List centers, int k) { - centers = centers.stream().distinct().collect(Collectors.toList()); - - ConcurrentHashMap weightsMap = weightCenters(cacheName, uuid, centers); - - List weights = new ArrayList<>(centers.size()); - - for (int i = 0; i < centers.size(); i++) - weights.add(i, Double.valueOf(weightsMap.getOrDefault(i, 0))); - - DenseLocalOnHeapMatrix centersMatrix = MatrixUtil.fromList(centers, true); - - KMeansLocalClusterer clusterer = new KMeansLocalClusterer(measure, kMeansMaxIterations, seed); - return clusterer.cluster(centersMatrix, k, weights).centers(); - } - - /** - * Weight each center with number of points for which this center is the nearest. - * - * @param cacheName Cache name of the point matrix. - * @param uuid Uuid of the point matrix. - * @param centers The list of centers. - * @return Hash map with weights. - */ - public ConcurrentHashMap weightCenters(String cacheName, UUID uuid, List centers) { - if (centers.size() == 0) - return new ConcurrentHashMap<>(); - - return CacheUtils.distributedFold(cacheName, - (IgniteBiFunction>, - ConcurrentHashMap, - ConcurrentHashMap>)(vectorWithIndex, counts) -> { - Vector vector = VectorUtils.fromMap(vectorWithIndex.getValue(), false); - - int nearest = 0; - double minDistance = distance(centers.get(nearest), vector); - - for (int i = 0; i < centers.size(); i++) { - double currDistance = distance(centers.get(i), vector); - if (currDistance < minDistance) { - minDistance = currDistance; - nearest = i; - } - } - - counts.compute(nearest, (index, value) -> value == null ? 1 : value + 1); - - return counts; - }, - key -> key.dataStructureId().equals(uuid), - (map1, map2) -> { - map1.putAll(map2); - return map1; - }, - ConcurrentHashMap::new); - } - - /** - * Calculate matrix of membership coefficients for each point and each center. - * - * @param points Matrix with source points. - * @param centers Array of current centers. - * @return Membership matrix and sums of membership coefficients for each center. - */ - private MembershipsAndSums calculateMembership(SparseDistributedMatrix points, Vector[] centers) { - String cacheName = ((SparseDistributedMatrixStorage) points.getStorage()).cacheName(); - UUID uuid = points.getUUID(); - double fuzzyMembershipCoefficient = 2 / (exponentialWeight - 1); - - MembershipsAndSumsSupplier supplier = new MembershipsAndSumsSupplier(centers.length); - - return CacheUtils.distributedFold(cacheName, - (IgniteBiFunction>, - MembershipsAndSums, - MembershipsAndSums>)(vectorWithIndex, membershipsAndSums) -> { - Integer idx = vectorWithIndex.getKey().index(); - Vector pnt = VectorUtils.fromMap(vectorWithIndex.getValue(), false); - Vector distances = new DenseLocalOnHeapVector(centers.length); - Vector pntMemberships = new DenseLocalOnHeapVector(centers.length); - - for (int i = 0; i < centers.length; i++) - distances.setX(i, distance(centers[i], pnt)); - - for (int i = 0; i < centers.length; i++) { - double invertedFuzzyWeight = 0.0; - - for (int j = 0; j < centers.length; j++) { - double val = Math.pow(distances.getX(i) / distances.getX(j), fuzzyMembershipCoefficient); - if (Double.isNaN(val)) - val = 1.0; - - invertedFuzzyWeight += val; - } - - double membership = Math.pow(1.0 / invertedFuzzyWeight, exponentialWeight); - pntMemberships.setX(i, membership); - } - - membershipsAndSums.memberships.put(idx, pntMemberships); - membershipsAndSums.membershipSums = membershipsAndSums.membershipSums.plus(pntMemberships); - - return membershipsAndSums; - }, - key -> key.dataStructureId().equals(uuid), - (mem1, mem2) -> { - mem1.merge(mem2); - return mem1; - }, - supplier); - } - - /** - * Calculate new centers according to membership matrix. - * - * @param points Matrix with source points. - * @param membershipsAndSums Membership matrix and sums of membership coefficient for each center. - * @param k The number of centers. - * @return Array of new centers. - */ - private Vector[] calculateNewCenters(SparseDistributedMatrix points, MembershipsAndSums membershipsAndSums, int k) { - String cacheName = ((SparseDistributedMatrixStorage) points.getStorage()).cacheName(); - UUID uuid = points.getUUID(); - - CentersArraySupplier supplier = new CentersArraySupplier(k, points.columnSize()); - - Vector[] centers = CacheUtils.distributedFold(cacheName, - (IgniteBiFunction>, - Vector[], - Vector[]>)(vectorWithIndex, centerSums) -> { - Integer idx = vectorWithIndex.getKey().index(); - Vector pnt = MatrixUtil.localCopyOf(VectorUtils.fromMap(vectorWithIndex.getValue(), false)); - Vector pntMemberships = membershipsAndSums.memberships.get(idx); - - for (int i = 0; i < k; i++) { - Vector weightedPnt = pnt.times(pntMemberships.getX(i)); - centerSums[i] = centerSums[i].plus(weightedPnt); - } - - return centerSums; - }, - key -> key.dataStructureId().equals(uuid), - (sums1, sums2) -> { - for (int i = 0; i < k; i++) - sums1[i] = sums1[i].plus(sums2[i]); - - return sums1; - }, - supplier); - - for (int i = 0; i < k; i++) - centers[i] = centers[i].divide(membershipsAndSums.membershipSums.getX(i)); - - return centers; - } - - /** - * Check if centers have moved insignificantly. - * - * @param centers Old centers. - * @param newCenters New centers. - * @return The result of comparison. - */ - private boolean isFinished(Vector[] centers, Vector[] newCenters) { - int numCenters = centers.length; - - for (int i = 0; i < numCenters; i++) - if (distance(centers[i], newCenters[i]) > maxDelta) - return false; - - return true; - } - - /** - * Check memberships difference. - * - * @param membershipsAndSums Old memberships. - * @param newMembershipsAndSums New memberships. - * @return The result of comparison. - */ - private boolean isFinished(MembershipsAndSums membershipsAndSums, MembershipsAndSums newMembershipsAndSums) { - if (membershipsAndSums == null) - return false; - - double currMaxDelta = 0.0; - for (Integer key : membershipsAndSums.memberships.keySet()) { - double distance = measure.compute(membershipsAndSums.memberships.get(key), - newMembershipsAndSums.memberships.get(key)); - if (distance > currMaxDelta) - currMaxDelta = distance; - } - - return currMaxDelta <= maxDelta; - } - - /** Service class used to optimize counting of membership sums. */ - private class MembershipsAndSums { - /** Membership matrix. */ - public ConcurrentHashMap memberships = new ConcurrentHashMap<>(); - - /** Membership sums. */ - public Vector membershipSums; - - /** - * Default constructor. - * - * @param k The number of centers. - */ - public MembershipsAndSums(int k) { - membershipSums = new DenseLocalOnHeapVector(k); - } - - /** - * Merge results of calculation for different parts of points. - * @param another Another part of memberships and sums. - */ - public void merge(MembershipsAndSums another) { - memberships.putAll(another.memberships); - membershipSums = membershipSums.plus(another.membershipSums); - } - } - - /** Service class that is used to create new {@link MembershipsAndSums} instances. */ - private class MembershipsAndSumsSupplier implements IgniteSupplier { - /** The number of centers */ - int k; - - /** - * Constructor that retains the number of centers. - * - * @param k The number of centers. - */ - public MembershipsAndSumsSupplier(int k) { - this.k = k; - } - - /** - * Create new instance of {@link MembershipsAndSums}. - * - * @return {@link MembershipsAndSums} object. - */ - @Override public MembershipsAndSums get() { - return new MembershipsAndSums(k); - } - } - - /** Service class that is used to create new arrays of vectors. */ - private class CentersArraySupplier implements IgniteSupplier { - /** The number of centers. */ - int k; - - /** The number of coordinates. */ - int dim; - - /** - * Constructor that retains all required parameters. - * - * @param k The number of centers. - * @param dim The number of coordinates. - */ - public CentersArraySupplier(int k, int dim) { - this.k = k; - this.dim = dim; - } - - /** - * Create new array of vectors. - * - * @return Array of vectors. - */ - @Override public Vector[] get() { - DenseLocalOnHeapVector[] centerSumsArr = new DenseLocalOnHeapVector[k]; - for (int i = 0; i < k; i++) - centerSumsArr[i] = new DenseLocalOnHeapVector(dim); - return centerSumsArr; - } - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/FuzzyCMeansLocalClusterer.java b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/FuzzyCMeansLocalClusterer.java deleted file mode 100644 index a1b6d3faa1134..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/FuzzyCMeansLocalClusterer.java +++ /dev/null @@ -1,254 +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.clustering; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Random; -import org.apache.ignite.internal.util.GridArgumentCheck; -import org.apache.ignite.ml.math.Matrix; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.distances.DistanceMeasure; -import org.apache.ignite.ml.math.exceptions.ConvergenceException; -import org.apache.ignite.ml.math.exceptions.MathIllegalArgumentException; -import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; - -/** Implements the local version of Fuzzy C-Means algorithm for weighted points. */ -public class FuzzyCMeansLocalClusterer extends BaseFuzzyCMeansClusterer implements - WeightedClusterer { - /** The maximum number of iterations. */ - private int maxIterations; - - /** The random numbers generator that is used to choose primary centers. */ - private Random rnd; - - /** - * Constructor that retains all required parameters. - * - * @param measure Distance measure. - * @param exponentialWeight Specific constant which is used in calculating of membership matrix. - * @param stopCond Flag that tells when algorithm should stop. - * @param maxDelta The maximum distance between old and new centers or maximum difference between new and old - * membership matrix elements for which algorithm must stop. - * @param maxIterations The maximum number of FCM iterations. - */ - public FuzzyCMeansLocalClusterer(DistanceMeasure measure, double exponentialWeight, StopCondition stopCond, - double maxDelta, int maxIterations, Long seed) { - super(measure, exponentialWeight, stopCond, maxDelta); - this.maxIterations = maxIterations; - rnd = seed != null ? new Random(seed) : new Random(); - } - - /** {@inheritDoc} */ - @Override public FuzzyCMeansModel cluster(DenseLocalOnHeapMatrix points, int k) { - List ones = new ArrayList<>(Collections.nCopies(points.rowSize(), 1.0)); - return cluster(points, k, ones); - } - - /** {@inheritDoc} */ - @Override public FuzzyCMeansModel cluster(DenseLocalOnHeapMatrix points, int k, List weights) - throws MathIllegalArgumentException, ConvergenceException { - GridArgumentCheck.notNull(points, "points"); - GridArgumentCheck.notNull(weights, "weights"); - - if (points.rowSize() != weights.size()) - throw new MathIllegalArgumentException("The number of points and the number of weights are not equal"); - - if (k < 2) - throw new MathIllegalArgumentException("The number of clusters is less than 2"); - - Matrix centers = new DenseLocalOnHeapMatrix(k, points.columnSize()); - Matrix distances = new DenseLocalOnHeapMatrix(k, points.rowSize()); - Matrix membership = new DenseLocalOnHeapMatrix(k, points.rowSize()); - Vector weightsVector = new DenseLocalOnHeapVector(weights.size()); - for (int i = 0; i < weights.size(); i++) - weightsVector.setX(i, weights.get(i)); - - initializeCenters(centers, points, k, weightsVector); - - int iteration = 0; - boolean finished = false; - while (iteration < maxIterations && !finished) { - calculateDistances(distances, points, centers); - Matrix newMembership = calculateMembership(distances, weightsVector); - Matrix newCenters = calculateNewCenters(points, newMembership); - - if (this.stopCond == StopCondition.STABLE_CENTERS) - finished = areCentersStable(centers, newCenters); - else - finished = areMembershipStable(membership, newMembership); - - centers = newCenters; - membership = newMembership; - iteration++; - } - - if (iteration == maxIterations) - throw new ConvergenceException("Fuzzy C-Means algorithm has not converged after " + - Integer.toString(iteration) + " iterations"); - - Vector[] centersArr = new Vector[k]; - for (int i = 0; i < k; i++) - centersArr[i] = centers.getRow(i); - - return new FuzzyCMeansModel(centersArr, measure); - } - - /** - * Choose {@code k} centers according to their weights. - * - * @param centers Output matrix containing primary centers. - * @param points Matrix of source points. - * @param k The number of centers. - * @param weights Vector of weights. - */ - private void initializeCenters(Matrix centers, Matrix points, int k, Vector weights) { - //int dimensions = points.columnSize(); - int numPoints = points.rowSize(); - - Vector firstCenter = points.viewRow(rnd.nextInt(numPoints)); - centers.setRow(0, firstCenter.getStorage().data()); - - Vector costs = points.foldRows(vector -> distance(vector, firstCenter)); - costs = costs.times(weights); - - double sum = costs.sum(); - - for (int i = 1; i < k; i++) { - double probe = rnd.nextDouble() * sum; - double cntr = 0; - int id = 0; - - for (int j = 0; j < numPoints; j++) { - cntr += costs.getX(j); - if (cntr >= probe) { - id = j; - break; - } - } - - centers.setRow(i, points.viewRow(id).getStorage().data()); - sum -= costs.get(id); - costs.set(id, 0.0); - } - } - - /** - * Calculate matrix of distances form each point to each center. - * - * @param distances Output matrix. - * @param points Matrix that contains source points. - * @param centers Matrix that contains centers. - */ - private void calculateDistances(Matrix distances, Matrix points, Matrix centers) { - int numPoints = points.rowSize(); - int numCenters = centers.rowSize(); - - for (int i = 0; i < numCenters; i++) - for (int j = 0; j < numPoints; j++) - distances.set(i, j, distance(centers.viewRow(i), points.viewRow(j))); - } - - /** - * Calculate membership matrix. - * - * @param distances Matrix of distances. - * @param weights Vector of weights. - * @ - */ - private Matrix calculateMembership(Matrix distances, Vector weights) { - Matrix newMembership = new DenseLocalOnHeapMatrix(distances.rowSize(), distances.columnSize()); - int numPoints = distances.columnSize(); - int numCenters = distances.rowSize(); - double fuzzyMembershipCoefficient = 2 / (exponentialWeight - 1); - - for (int i = 0; i < numCenters; i++) { - for (int j = 0; j < numPoints; j++) { - double invertedFuzzyWeight = 0.0; - - for (int k = 0; k < numCenters; k++) { - double val = Math.pow(distances.get(i, j) / distances.get(k, j), - fuzzyMembershipCoefficient); - if (Double.isNaN(val)) - val = 1.0; - - invertedFuzzyWeight += val; - } - - double weight = 1.0 / invertedFuzzyWeight * weights.getX(j); - newMembership.setX(i, j, Math.pow(weight, exponentialWeight)); - } - } - return newMembership; - } - - /** - * Calculate new centers using membership matrix. - * - * @param points Matrix of source points. - * @param membership Matrix that contains membership coefficients. - * @return Matrix that contains new centers. - */ - private Matrix calculateNewCenters(Matrix points, Matrix membership) { - Vector membershipSums = membership.foldRows(Vector::sum); - Matrix newCenters = membership.times(points); - - int numCenters = newCenters.rowSize(); - for (int i = 0; i < numCenters; i++) - newCenters.viewRow(i).divide(membershipSums.getX(i)); - - return newCenters; - } - - /** - * Check if centers have moved insignificantly. - * - * @param centers Old centers. - * @param newCenters New centers. - * @return The result of comparison. - */ - private boolean areCentersStable(Matrix centers, Matrix newCenters) { - int numCenters = centers.rowSize(); - for (int i = 0; i < numCenters; i++) - if (distance(centers.viewRow(i), newCenters.viewRow(i)) > maxDelta) - return false; - - return true; - } - - /** - * Check if membership matrix has changed insignificantly. - * - * @param membership Old membership matrix. - * @param newMembership New membership matrix. - * @return The result of comparison. - */ - private boolean areMembershipStable(Matrix membership, Matrix newMembership) { - int numCenters = membership.rowSize(); - int numPoints = membership.columnSize(); - - for (int i = 0; i < numCenters; i++) - for (int j = 0; j < numPoints; j++) - if (Math.abs(newMembership.getX(i, j) - membership.getX(i, j)) > maxDelta) - return false; - - return true; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/FuzzyCMeansModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/FuzzyCMeansModel.java deleted file mode 100644 index 70009cb91379b..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/FuzzyCMeansModel.java +++ /dev/null @@ -1,88 +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.clustering; - -import java.util.Arrays; -import org.apache.ignite.ml.Exportable; -import org.apache.ignite.ml.Exporter; -import org.apache.ignite.ml.FuzzyCMeansModelFormat; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.distances.DistanceMeasure; - -/** This class incapsulates result of clusterization. */ -public class FuzzyCMeansModel implements ClusterizationModel, Exportable { - /** Centers of clusters. */ - private Vector[] centers; - - /** Distance measure. */ - private DistanceMeasure measure; - - /** - * Constructor that creates FCM model by centers and measure. - * - * @param centers Array of centers. - * @param measure Distance measure. - */ - public FuzzyCMeansModel(Vector[] centers, DistanceMeasure measure) { - this.centers = Arrays.copyOf(centers, centers.length); - this.measure = measure; - } - - /** Distance measure used while clusterization. */ - public DistanceMeasure distanceMeasure() { - return measure; - } - - /** @inheritDoc */ - @Override public int clustersCount() { - return centers.length; - } - - /** @inheritDoc */ - @Override public Vector[] centers() { - return Arrays.copyOf(centers, centers.length); - } - - /** - * Predict closest center index for a given vector. - * - * @param val Vector. - * @return Index of the closest center or -1 if it can't be found. - */ - @Override public Integer apply(Vector val) { - int idx = -1; - double minDistance = Double.POSITIVE_INFINITY; - - for (int i = 0; i < centers.length; i++) { - double currDistance = measure.compute(val, centers[i]); - if (currDistance < minDistance) { - minDistance = currDistance; - idx = i; - } - } - - return idx; - } - - /** {@inheritDoc} */ - @Override public

    void saveModel(Exporter exporter, P path) { - FuzzyCMeansModelFormat mdlData = new FuzzyCMeansModelFormat(centers, measure); - - exporter.save(mdlData, path); - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/KMeansDistributedClusterer.java b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/KMeansDistributedClusterer.java deleted file mode 100644 index 5595b4cd3b83c..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/KMeansDistributedClusterer.java +++ /dev/null @@ -1,306 +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.clustering; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.stream.Collectors; -import javax.cache.Cache; -import org.apache.ignite.lang.IgniteBiTuple; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.VectorUtils; -import org.apache.ignite.ml.math.distances.DistanceMeasure; -import org.apache.ignite.ml.math.distributed.CacheUtils; -import org.apache.ignite.ml.math.distributed.keys.impl.SparseMatrixKey; -import org.apache.ignite.ml.math.exceptions.ConvergenceException; -import org.apache.ignite.ml.math.exceptions.MathIllegalArgumentException; -import org.apache.ignite.ml.math.functions.Functions; -import org.apache.ignite.ml.math.functions.IgniteBiFunction; -import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.ml.math.impls.storage.matrix.SparseDistributedMatrixStorage; -import org.apache.ignite.ml.math.util.MapUtil; -import org.apache.ignite.ml.math.util.MatrixUtil; - -import static org.apache.ignite.ml.math.distributed.CacheUtils.distributedFold; -import static org.apache.ignite.ml.math.util.MatrixUtil.localCopyOf; - -/** - * Clustering algorithm based on Bahmani et al. paper and Apache Spark class with corresponding functionality. - * - * TODO: IGNITE-6059, add block matrix support. - * - * @see Scalable K-Means++(wikipedia) - */ -public class KMeansDistributedClusterer extends BaseKMeansClusterer { - /** */ - private final int maxIterations; - - /** */ - private Random rnd; - - /** */ - private int initSteps; - - /** */ - private long seed; - - /** */ - private double epsilon = 1e-4; - - /** */ - public KMeansDistributedClusterer(DistanceMeasure measure, int initSteps, int maxIterations, Long seed) { - super(measure); - this.initSteps = initSteps; - - this.seed = seed != null ? seed : new Random().nextLong(); - - this.maxIterations = maxIterations; - rnd = new Random(this.seed); - } - - /** */ - @Override public KMeansModel cluster(SparseDistributedMatrix points, int k) throws - MathIllegalArgumentException, ConvergenceException { - SparseDistributedMatrix pointsCp = (SparseDistributedMatrix)points.like(points.rowSize(), points.columnSize()); - - String cacheName = ((SparseDistributedMatrixStorage)points.getStorage()).cacheName(); - - // TODO: IGNITE-5825, this copy is very ineffective, just for POC. Immutability of data should be guaranteed by other methods - // such as logical locks for example. - pointsCp.assign(points); - - Vector[] centers = initClusterCenters(pointsCp, k); - - boolean converged = false; - int iteration = 0; - int dim = pointsCp.viewRow(0).size(); - UUID uid = pointsCp.getUUID(); - - // Execute iterations of Lloyd's algorithm until converged - while (iteration < maxIterations && !converged) { - SumsAndCounts stats = getSumsAndCounts(centers, dim, uid, cacheName); - - converged = true; - - for (Integer ind : stats.sums.keySet()) { - Vector massCenter = stats.sums.get(ind).times(1.0 / stats.counts.get(ind)); - - if (converged && distance(massCenter, centers[ind]) > epsilon * epsilon) - converged = false; - - centers[ind] = massCenter; - } - - iteration++; - } - - pointsCp.destroy(); - - return new KMeansModel(centers, getDistanceMeasure()); - } - - /** Initialize cluster centers. */ - private Vector[] initClusterCenters(SparseDistributedMatrix points, int k) { - // Initialize empty centers and point costs. - int ptsCnt = points.rowSize(); - - String cacheName = ((SparseDistributedMatrixStorage)points.getStorage()).cacheName(); - - // Initialize the first center to a random point. - Vector sample = localCopyOf(points.viewRow(rnd.nextInt(ptsCnt))); - - List centers = new ArrayList<>(); - List newCenters = new ArrayList<>(); - newCenters.add(sample); - centers.add(sample); - - final ConcurrentHashMap costs = new ConcurrentHashMap<>(); - - // On each step, sample 2 * k points on average with probability proportional - // to their squared distance from the centers. Note that only distances between points - // and new centers are computed in each iteration. - int step = 0; - UUID uid = points.getUUID(); - - while (step < initSteps) { - // We assume here that costs can fit into memory of one node. - ConcurrentHashMap newCosts = getNewCosts(points, newCenters, cacheName); - - // Merge costs with new costs. - for (Integer ind : newCosts.keySet()) - costs.merge(ind, newCosts.get(ind), Math::min); - - double sumCosts = costs.values().stream().mapToDouble(Double::valueOf).sum(); - - newCenters = getNewCenters(k, costs, uid, sumCosts, cacheName); - centers.addAll(newCenters); - - step++; - } - - List distinctCenters = centers.stream().distinct().collect(Collectors.toList()); - - if (distinctCenters.size() <= k) - return distinctCenters.toArray(new Vector[] {}); - else { - // Finally, we might have a set of more than k distinct candidate centers; weight each - // candidate by the number of points in the dataset mapping to it and run a local k-means++ - // on the weighted centers to pick k of them - ConcurrentHashMap centerInd2Weight = weightCenters(uid, distinctCenters, cacheName); - - List weights = new ArrayList<>(centerInd2Weight.size()); - - for (int i = 0; i < distinctCenters.size(); i++) - weights.add(i, Double.valueOf(centerInd2Weight.getOrDefault(i, 0))); - - DenseLocalOnHeapMatrix dCenters = MatrixUtil.fromList(distinctCenters, true); - - return new KMeansLocalClusterer(getDistanceMeasure(), 30, seed).cluster(dCenters, k, weights).centers(); - } - } - - /** */ - private List getNewCenters(int k, ConcurrentHashMap costs, UUID uid, - double sumCosts, String cacheName) { - return distributedFold(cacheName, - (IgniteBiFunction>, - List, - List>)(vectorWithIndex, list) -> { - Integer ind = vectorWithIndex.getKey().index(); - - double prob = costs.get(ind) * 2.0 * k / sumCosts; - - if (new Random(seed ^ ind).nextDouble() < prob) - list.add(VectorUtils.fromMap(vectorWithIndex.getValue(), false)); - - return list; - }, - key -> key.dataStructureId().equals(uid), - (list1, list2) -> { - list1.addAll(list2); - return list1; - }, ArrayList::new - ); - } - - /** */ - private ConcurrentHashMap getNewCosts(SparseDistributedMatrix points, List newCenters, - String cacheName) { - return distributedFold(cacheName, - (IgniteBiFunction>, - ConcurrentHashMap, - ConcurrentHashMap>)(vectorWithIndex, map) -> { - for (Vector center : newCenters) - map.merge(vectorWithIndex.getKey().index(), distance(vectorWithIndex.getValue(), center), Functions.MIN); - - return map; - }, - key -> key.dataStructureId().equals(points.getUUID()), - (map1, map2) -> { - map1.putAll(map2); - return map1; - }, ConcurrentHashMap::new); - } - - /** */ - private ConcurrentHashMap weightCenters(UUID uid, List distinctCenters, - String cacheName) { - return distributedFold(cacheName, - (IgniteBiFunction>, - ConcurrentHashMap, - ConcurrentHashMap>)(vectorWithIndex, countMap) -> { - Integer resInd = -1; - Double resDist = Double.POSITIVE_INFINITY; - - int i = 0; - for (Vector cent : distinctCenters) { - double curDist = distance(vectorWithIndex.getValue(), cent); - - if (resDist > curDist) { - resDist = curDist; - resInd = i; - } - - i++; - } - - countMap.compute(resInd, (ind, v) -> v != null ? v + 1 : 1); - return countMap; - }, - key -> key.dataStructureId().equals(uid), - (map1, map2) -> MapUtil.mergeMaps(map1, map2, (integer, integer2) -> integer2 + integer, - ConcurrentHashMap::new), - ConcurrentHashMap::new); - } - - /** */ - private double distance(Map vecMap, Vector vector) { - return distance(VectorUtils.fromMap(vecMap, false), vector); - } - - /** */ - private SumsAndCounts getSumsAndCounts(Vector[] centers, int dim, UUID uid, String cacheName) { - return CacheUtils.distributedFold(cacheName, - (IgniteBiFunction>, SumsAndCounts, SumsAndCounts>)(entry, counts) -> { - Map vec = entry.getValue(); - - IgniteBiTuple closest = findClosest(centers, VectorUtils.fromMap(vec, false)); - int bestCenterIdx = closest.get1(); - - counts.totalCost += closest.get2(); - counts.sums.putIfAbsent(bestCenterIdx, VectorUtils.zeroes(dim)); - - counts.sums.compute(bestCenterIdx, - (IgniteBiFunction)(ind, v) -> v.plus(VectorUtils.fromMap(vec, false))); - - counts.counts.merge(bestCenterIdx, 1, - (IgniteBiFunction)(i1, i2) -> i1 + i2); - - return counts; - }, - key -> key.dataStructureId().equals(uid), - SumsAndCounts::merge, SumsAndCounts::new - ); - } - - /** Service class used for statistics. */ - private static class SumsAndCounts { - /** */ - public double totalCost; - - /** */ - public ConcurrentHashMap sums = new ConcurrentHashMap<>(); - - /** Count of points closest to the center with a given index. */ - public ConcurrentHashMap counts = new ConcurrentHashMap<>(); - - /** Merge current */ - public SumsAndCounts merge(SumsAndCounts other) { - this.totalCost += totalCost; - MapUtil.mergeMaps(sums, other.sums, Vector::plus, ConcurrentHashMap::new); - MapUtil.mergeMaps(counts, other.counts, (i1, i2) -> i1 + i2, ConcurrentHashMap::new); - return this; - } - } - -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/KMeansLocalClusterer.java b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/KMeansLocalClusterer.java deleted file mode 100644 index 8a50e65436ccd..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/KMeansLocalClusterer.java +++ /dev/null @@ -1,177 +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.clustering; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Random; -import org.apache.ignite.internal.util.GridArgumentCheck; -import org.apache.ignite.ml.math.Matrix; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.VectorUtils; -import org.apache.ignite.ml.math.distances.DistanceMeasure; -import org.apache.ignite.ml.math.exceptions.ConvergenceException; -import org.apache.ignite.ml.math.exceptions.MathIllegalArgumentException; -import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; - -import static org.apache.ignite.ml.math.util.MatrixUtil.localCopyOf; - -/** - * Perform clusterization on local data. - * This class is based on Apache Spark class with corresponding functionality. - */ -public class KMeansLocalClusterer extends BaseKMeansClusterer implements - WeightedClusterer { - /** */ - private int maxIterations; - - /** */ - private Random rand; - - /** - * Build a new clusterer with the given {@link DistanceMeasure}. - * - * @param measure Distance measure to use. - * @param maxIterations maximal number of iterations. - * @param seed Seed used in random parts of algorithm. - */ - public KMeansLocalClusterer(DistanceMeasure measure, int maxIterations, Long seed) { - super(measure); - this.maxIterations = maxIterations; - rand = seed != null ? new Random(seed) : new Random(); - } - - /** {@inheritDoc} */ - @Override public KMeansModel cluster( - DenseLocalOnHeapMatrix points, int k) throws MathIllegalArgumentException, ConvergenceException { - List ones = new ArrayList<>(Collections.nCopies(points.rowSize(), 1.0)); - return cluster(points, k, ones); - } - - /** {@inheritDoc} */ - @Override public KMeansModel cluster(DenseLocalOnHeapMatrix points, int k, - List weights) throws MathIllegalArgumentException, ConvergenceException { - - GridArgumentCheck.notNull(points, "points"); - - int dim = points.columnSize(); - Vector[] centers = new Vector[k]; - - centers[0] = pickWeighted(points, weights); - - Vector costs = points.foldRows(row -> distance(row, - centers[0])); - - for (int i = 0; i < k; i++) { - double weightedSum = weightedSum(costs, weights); - - double r = rand.nextDouble() * weightedSum; - double s = 0.0; - int j = 0; - - while (j < points.rowSize() && s < r) { - s += weights.get(j) * costs.get(j); - j++; - } - - if (j == 0) - // TODO: IGNITE-5825, Process this case more carefully - centers[i] = localCopyOf(points.viewRow(0)); - else - centers[i] = localCopyOf(points.viewRow(j - 1)); - - for (int p = 0; p < points.rowSize(); p++) - costs.setX(p, Math.min(getDistanceMeasure().compute(localCopyOf(points.viewRow(p)), centers[i]), - costs.get(p))); - } - - int[] oldClosest = new int[points.rowSize()]; - Arrays.fill(oldClosest, -1); - int iter = 0; - boolean moved = true; - - while (moved && iter < maxIterations) { - moved = false; - - double[] counts = new double[k]; - Arrays.fill(counts, 0.0); - Vector[] sums = new Vector[k]; - - Arrays.fill(sums, VectorUtils.zeroes(dim)); - - int i = 0; - - while (i < points.rowSize()) { - Vector p = localCopyOf(points.viewRow(i)); - - int ind = findClosest(centers, p).get1(); - sums[ind] = sums[ind].plus(p.times(weights.get(i))); - - counts[ind] += weights.get(i); - if (ind != oldClosest[i]) { - moved = true; - oldClosest[i] = ind; - } - i++; - } - // Update centers - int j = 0; - while (j < k) { - if (counts[j] == 0.0) { - // Assign center to a random point - centers[j] = points.viewRow(rand.nextInt(points.rowSize())); - } - else { - sums[j] = sums[j].times(1.0 / counts[j]); - centers[j] = sums[j]; - } - j++; - } - iter++; - } - - return new KMeansModel(centers, getDistanceMeasure()); - } - - /** Pick a random vector with a probability proportional to the corresponding weight. */ - private Vector pickWeighted(Matrix points, List weights) { - double r = rand.nextDouble() * weights.stream().mapToDouble(Double::valueOf).sum(); - - int i = 0; - double curWeight = 0.0; - - while (i < points.rowSize() && curWeight < r) { - curWeight += weights.get(i); - i += 1; - } - - return localCopyOf(points.viewRow(i - 1)); - } - - /** Get a weighted sum of a vector v. */ - private double weightedSum(Vector v, List weights) { - double res = 0.0; - - for (int i = 0; i < v.size(); i++) - res += v.getX(i) * weights.get(i); - - return res; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/WeightedClusterer.java b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/WeightedClusterer.java deleted file mode 100644 index 16880874eee95..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/WeightedClusterer.java +++ /dev/null @@ -1,38 +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.clustering; - -import java.util.List; -import org.apache.ignite.ml.Model; -import org.apache.ignite.ml.math.exceptions.ConvergenceException; -import org.apache.ignite.ml.math.exceptions.MathIllegalArgumentException; - -/** - * Support of clusterization with given weights. - */ -public interface WeightedClusterer extends Clusterer { - /** - * Perform clusterization of given points weighted by given weights. - * - * @param points Points. - * @param k count of centers. - * @param weights Weights. - */ - public M cluster(P points, int k, List weights) throws - MathIllegalArgumentException, ConvergenceException; -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/Clusterer.java b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/Clusterer.java similarity index 95% rename from modules/ml/src/main/java/org/apache/ignite/ml/clustering/Clusterer.java rename to modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/Clusterer.java index 204e28a7ab8f5..9930f23bce5e8 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/Clusterer.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/Clusterer.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.ignite.ml.clustering; +package org.apache.ignite.ml.clustering.kmeans; import org.apache.ignite.ml.Model; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/ClusterizationModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/ClusterizationModel.java similarity index 92% rename from modules/ml/src/main/java/org/apache/ignite/ml/clustering/ClusterizationModel.java rename to modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/ClusterizationModel.java index 99afec59cd9dc..474a463eb7550 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/ClusterizationModel.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/ClusterizationModel.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.ignite.ml.clustering; +package org.apache.ignite.ml.clustering.kmeans; import org.apache.ignite.ml.Model; /** Base interface for all clusterization models. */ public interface ClusterizationModel extends Model { /** Gets the clusters count. */ - public int clustersCount(); + public int amountOfClusters(); /** Get cluster centers. */ public P[] centers(); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/KMeansModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansModel.java similarity index 77% rename from modules/ml/src/main/java/org/apache/ignite/ml/clustering/KMeansModel.java rename to modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansModel.java index e1d783f7d2daf..c900efd6cbe1f 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/clustering/KMeansModel.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansModel.java @@ -15,43 +15,42 @@ * limitations under the License. */ -package org.apache.ignite.ml.clustering; +package org.apache.ignite.ml.clustering.kmeans; import java.util.Arrays; import org.apache.ignite.ml.Exportable; import org.apache.ignite.ml.Exporter; -import org.apache.ignite.ml.KMeansModelFormat; import org.apache.ignite.ml.math.Vector; import org.apache.ignite.ml.math.distances.DistanceMeasure; /** - * This class encapsulates result of clusterization. + * This class encapsulates result of clusterization by KMeans algorithm. */ public class KMeansModel implements ClusterizationModel, Exportable { /** Centers of clusters. */ private final Vector[] centers; /** Distance measure. */ - private final DistanceMeasure distance; + private final DistanceMeasure distanceMeasure; /** - * Construct KMeans model with given centers and distance measure. + * Construct KMeans model with given centers and distanceMeasure measure. * * @param centers Centers. - * @param distance Distance measure. + * @param distanceMeasure Distance measure. */ - public KMeansModel(Vector[] centers, DistanceMeasure distance) { + public KMeansModel(Vector[] centers, DistanceMeasure distanceMeasure) { this.centers = centers; - this.distance = distance; + this.distanceMeasure = distanceMeasure; } - /** Distance measure used while clusterization */ + /** Distance measure. */ public DistanceMeasure distanceMeasure() { - return distance; + return distanceMeasure; } - /** Count of centers in clusterization. */ - @Override public int clustersCount() { + /** Amount of centers in clusterization. */ + @Override public int amountOfClusters() { return centers.length; } @@ -70,7 +69,7 @@ public Integer apply(Vector vec) { double minDist = Double.POSITIVE_INFINITY; for (int i = 0; i < centers.length; i++) { - double curDist = distance.compute(centers[i], vec); + double curDist = distanceMeasure.compute(centers[i], vec); if (curDist < minDist) { minDist = curDist; res = i; @@ -82,7 +81,7 @@ public Integer apply(Vector vec) { /** {@inheritDoc} */ @Override public

    void saveModel(Exporter exporter, P path) { - KMeansModelFormat mdlData = new KMeansModelFormat(centers, distance); + KMeansModelFormat mdlData = new KMeansModelFormat(centers, distanceMeasure); exporter.save(mdlData, path); } @@ -91,7 +90,7 @@ public Integer apply(Vector vec) { @Override public int hashCode() { int res = 1; - res = res * 37 + distance.hashCode(); + res = res * 37 + distanceMeasure.hashCode(); res = res * 37 + Arrays.hashCode(centers); return res; @@ -107,7 +106,7 @@ public Integer apply(Vector vec) { KMeansModel that = (KMeansModel)obj; - return distance.equals(that.distance) && Arrays.deepEquals(centers, that.centers); + return distanceMeasure.equals(that.distanceMeasure) && Arrays.deepEquals(centers, that.centers); } } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/KMeansModelFormat.java b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansModelFormat.java similarity index 94% rename from modules/ml/src/main/java/org/apache/ignite/ml/KMeansModelFormat.java rename to modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansModelFormat.java index c013198af9808..266370115f1c6 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/KMeansModelFormat.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansModelFormat.java @@ -15,10 +15,12 @@ * limitations under the License. */ -package org.apache.ignite.ml; +package org.apache.ignite.ml.clustering.kmeans; import java.io.Serializable; import java.util.Arrays; +import org.apache.ignite.ml.Exportable; +import org.apache.ignite.ml.Exporter; import org.apache.ignite.ml.math.Vector; import org.apache.ignite.ml.math.distances.DistanceMeasure; 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 new file mode 100644 index 0000000000000..f65a3fed4cf2d --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/KMeansTrainer.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.ml.clustering.kmeans; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.ml.dataset.Dataset; +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.Vector; +import org.apache.ignite.ml.math.VectorUtils; +import org.apache.ignite.ml.math.distances.DistanceMeasure; +import org.apache.ignite.ml.math.distances.EuclideanDistance; +import org.apache.ignite.ml.math.functions.IgniteBiFunction; +import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; +import org.apache.ignite.ml.math.util.MapUtil; +import org.apache.ignite.ml.structures.LabeledDataset; +import org.apache.ignite.ml.structures.LabeledVector; +import org.apache.ignite.ml.structures.partition.LabeledDatasetPartitionDataBuilderOnHeap; +import org.apache.ignite.ml.trainers.SingleLabelDatasetTrainer; + +/** + * The trainer for KMeans algorithm. + */ +public class KMeansTrainer implements SingleLabelDatasetTrainer { + /** Amount of clusters. */ + private int k = 2; + + /** Amount of iterations. */ + private int maxIterations = 10; + + /** Delta of convergence. */ + private double epsilon = 1e-4; + + /** Distance measure. */ + private DistanceMeasure distance = new EuclideanDistance(); + + /** KMeans initializer. */ + private long seed; + + /** + * Trains model based on the specified data. + * + * @param datasetBuilder Dataset builder. + * @param featureExtractor Feature extractor. + * @param lbExtractor Label extractor. + * @return Model. + */ + @Override public KMeansModel fit(DatasetBuilder datasetBuilder, + IgniteBiFunction featureExtractor, IgniteBiFunction lbExtractor) { + assert datasetBuilder != null; + + PartitionDataBuilder> partDataBuilder = new LabeledDatasetPartitionDataBuilderOnHeap<>( + featureExtractor, + lbExtractor + ); + + Vector[] centers; + + 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); + centers = initClusterCentersRandomly(dataset, k); + + boolean converged = false; + int iteration = 0; + + while (iteration < maxIterations && !converged) { + Vector[] newCentroids = new DenseLocalOnHeapVector[k]; + + TotalCostAndCounts totalRes = calcDataForNewCentroids(centers, dataset, cols); + + converged = true; + + for (Integer ind : totalRes.sums.keySet()) { + Vector massCenter = totalRes.sums.get(ind).times(1.0 / totalRes.counts.get(ind)); + + if (converged && distance.compute(massCenter, centers[ind]) > epsilon * epsilon) + converged = false; + + newCentroids[ind] = massCenter; + } + + iteration++; + centers = newCentroids; + } + } + catch (Exception e) { + throw new RuntimeException(e); + } + return new KMeansModel(centers, distance); + } + + /** + * Prepares the data to define new centroids on current iteration. + * + * @param centers Current centers on the current iteration. + * @param dataset Dataset. + * @param cols Amount of columns. + * @return Helper data to calculate the new centroids. + */ + private TotalCostAndCounts calcDataForNewCentroids(Vector[] centers, + Dataset> dataset, int cols) { + final Vector[] finalCenters = centers; + + return dataset.compute(data -> { + + TotalCostAndCounts res = new TotalCostAndCounts(); + + for (int i = 0; i < data.rowSize(); i++) { + final IgniteBiTuple closestCentroid = findClosestCentroid(finalCenters, data.getRow(i)); + + int centroidIdx = closestCentroid.get1(); + + data.setLabel(i, centroidIdx); + + res.totalCost += closestCentroid.get2(); + res.sums.putIfAbsent(centroidIdx, VectorUtils.zeroes(cols)); + + int finalI = i; + res.sums.compute(centroidIdx, + (IgniteBiFunction)(ind, v) -> v.plus(data.getRow(finalI).features())); + + res.counts.merge(centroidIdx, 1, + (IgniteBiFunction)(i1, i2) -> i1 + i2); + } + return res; + }, (a, b) -> a == null ? b : 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. + */ + private IgniteBiTuple findClosestCentroid(Vector[] centers, LabeledVector pnt) { + double bestDistance = Double.POSITIVE_INFINITY; + int bestInd = 0; + + for (int i = 0; i < centers.length; i++) { + double dist = distance.compute(centers[i], pnt.features()); + if (dist < bestDistance) { + bestDistance = dist; + bestInd = i; + } + } + return new IgniteBiTuple<>(bestInd, bestDistance); + } + + /** + * K cluster centers are initialized randomly. + * + * @param dataset The dataset to pick up random centers. + * @param k Amount of clusters. + * @return K cluster centers. + */ + private Vector[] initClusterCentersRandomly(Dataset> dataset, + int k) { + + Vector[] initCenters = new DenseLocalOnHeapVector[k]; + + List rndPnts = dataset.compute(data -> { + List rndPnt = new ArrayList<>(); + rndPnt.add(data.getRow(new Random(seed).nextInt(data.rowSize()))); + return rndPnt; + }, (a, b) -> a == null ? b : Stream.concat(a.stream(), b.stream()).collect(Collectors.toList())); + + for (int i = 0; i < k; i++) { + final LabeledVector rndPnt = rndPnts.get(new Random(seed).nextInt(rndPnts.size())); + rndPnts.remove(rndPnt); + initCenters[i] = rndPnt.features(); + } + + return initCenters; + } + + /** Service class used for statistics. */ + private static class TotalCostAndCounts { + /** */ + double totalCost; + + /** */ + ConcurrentHashMap sums = new ConcurrentHashMap<>(); + + /** Count of points closest to the center with a given index. */ + ConcurrentHashMap counts = new ConcurrentHashMap<>(); + + /** Merge current */ + TotalCostAndCounts merge(TotalCostAndCounts other) { + this.totalCost += totalCost; + this.sums = MapUtil.mergeMaps(sums, other.sums, Vector::plus, ConcurrentHashMap::new); + this.counts = MapUtil.mergeMaps(counts, other.counts, (i1, i2) -> i1 + i2, ConcurrentHashMap::new); + return this; + } + } + + /** + * Gets the amount of clusters. + * + * @return The parameter value. + */ + public int getK() { + return k; + } + + /** + * Set up the amount of clusters. + * + * @param k The parameter value. + * @return Model with new amount of clusters parameter value. + */ + public KMeansTrainer withK(int k) { + this.k = k; + return this; + } + + /** + * Gets the max number of iterations before convergence. + * + * @return The parameter value. + */ + public int getMaxIterations() { + return maxIterations; + } + + /** + * 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 KMeansTrainer withMaxIterations(int maxIterations) { + this.maxIterations = maxIterations; + return this; + } + + /** + * Gets the epsilon. + * + * @return The parameter value. + */ + public double getEpsilon() { + return epsilon; + } + + /** + * Set up the epsilon. + * + * @param epsilon The parameter value. + * @return Model with new epsilon parameter value. + */ + public KMeansTrainer withEpsilon(double epsilon) { + this.epsilon = epsilon; + return this; + } + + /** + * Gets the distance. + * + * @return The parameter value. + */ + public DistanceMeasure getDistance() { + return distance; + } + + /** + * Set up the distance. + * + * @param distance The parameter value. + * @return Model with new distance parameter value. + */ + public KMeansTrainer withDistance(DistanceMeasure distance) { + this.distance = distance; + 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 KMeansTrainer withSeed(long seed) { + this.seed = seed; + return this; + } +} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansUtil.java b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/package-info.java similarity index 62% rename from modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansUtil.java rename to modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/package-info.java index 420678fe1a4c4..4d27b6e878b95 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansUtil.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/clustering/kmeans/package-info.java @@ -15,19 +15,8 @@ * limitations under the License. */ -package org.apache.ignite.ml.clustering; - -import org.apache.ignite.ml.math.Vector; - -import static org.junit.Assert.assertTrue; - -/** Utilities for k-means tests. */ -class KMeansUtil { - /** */ - static void checkIsInEpsilonNeighbourhood(Vector[] v1s, Vector[] v2s, double epsilon) { - for (int i = 0; i < v1s.length; i++) { - assertTrue("Not in epsilon neighbourhood (index " + i + ") ", - v1s[i].minus(v2s[i]).kNorm(2) < epsilon); - } - } -} +/** + * + * Contains kMeans clustering algorithm. + */ +package org.apache.ignite.ml.clustering.kmeans; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/LabellingMachine.java b/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/LabellingMachine.java deleted file mode 100644 index e8d88ada91d1d..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/LabellingMachine.java +++ /dev/null @@ -1,41 +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.structures.preprocessing; - -import org.apache.ignite.ml.Model; -import org.apache.ignite.ml.structures.LabeledDataset; - -/** Data pre-processing step which assigns labels to all observations according model. */ -public class LabellingMachine { - /** - * Set labels to each observation according passed model. - *

    - * NOTE: In-place operation. - *

    - * @param ds The given labeled dataset. - * @param mdl The given model. - * @return Dataset with predicted labels. - */ - public static LabeledDataset assignLabels(LabeledDataset ds, Model mdl) { - for (int i = 0; i < ds.rowSize(); i++) { - double predictedCls = (double) mdl.apply(ds.getRow(i).features()); - ds.setLabel(i, predictedCls); - } - return ds; - } -} diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/Normalizer.java b/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/Normalizer.java deleted file mode 100644 index 161ec355d673c..0000000000000 --- a/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/Normalizer.java +++ /dev/null @@ -1,80 +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.structures.preprocessing; - -import org.apache.ignite.ml.math.exceptions.UnsupportedOperationException; -import org.apache.ignite.ml.structures.Dataset; -import org.apache.ignite.ml.structures.DatasetRow; - -/** Data pre-processing step which scales features according normalization algorithms. */ -public class Normalizer { - /** - * Scales features in dataset with MiniMax algorithm x'=(x-MIN[X])/(MAX[X]-MIN[X]). This is an in-place operation. - *

    - * NOTE: Complexity 2*N^2. - *

    - * @param ds The given dataset. - * @return Transformed dataset. - */ - public static Dataset normalizeWithMiniMax(Dataset ds) { - int colSize = ds.colSize(); - double[] mins = new double[colSize]; - double[] maxs = new double[colSize]; - - int rowSize = ds.rowSize(); - DatasetRow[] data = ds.data(); - for (int j = 0; j < colSize; j++) { - double maxInCurrCol = Double.MIN_VALUE; - double minInCurrCol = Double.MAX_VALUE; - - for (int i = 0; i < rowSize; i++) { - double e = data[i].features().get(j); - maxInCurrCol = Math.max(e, maxInCurrCol); - minInCurrCol = Math.min(e, minInCurrCol); - } - - mins[j] = minInCurrCol; - maxs[j] = maxInCurrCol; - } - - for (int j = 0; j < colSize; j++) { - double div = maxs[j] - mins[j]; - if(div == 0) - continue; - - for (int i = 0; i < rowSize; i++) { - double oldVal = data[i].features().get(j); - double newVal = (oldVal - mins[j]) / div; - // x'=(x-MIN[X])/(MAX[X]-MIN[X]) - data[i].features().set(j, newVal); - } - } - - return ds; - } - - /** - * Scales features in dataset with Z-Normalization algorithm x'=(x-M[X])/\sigma [X]. This is an in-place operation. - * - * @param ds The given dataset. - * @return Transformed dataset. - */ - public static Dataset normalizeWithZNormalization(Dataset ds) { - throw new UnsupportedOperationException("Z-normalization is not supported yet"); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/LocalModelsTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/LocalModelsTest.java index 3f12bdcbd7560..353cc227377b0 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/LocalModelsTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/LocalModelsTest.java @@ -20,14 +20,18 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; import java.util.function.Function; -import org.apache.ignite.ml.clustering.KMeansLocalClusterer; -import org.apache.ignite.ml.clustering.KMeansModel; +import org.apache.ignite.ml.clustering.kmeans.KMeansModel; +import org.apache.ignite.ml.clustering.kmeans.KMeansModelFormat; +import org.apache.ignite.ml.clustering.kmeans.KMeansTrainer; +import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; import org.apache.ignite.ml.knn.classification.KNNClassificationModel; import org.apache.ignite.ml.knn.classification.KNNModelFormat; import org.apache.ignite.ml.knn.classification.KNNStrategy; import org.apache.ignite.ml.math.distances.EuclideanDistance; -import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; import org.apache.ignite.ml.regressions.linear.LinearRegressionModel; import org.apache.ignite.ml.svm.SVMLinearBinaryClassificationModel; @@ -140,14 +144,20 @@ private void executeModelTest(Function code) throws IOException { /** */ private KMeansModel getClusterModel() { - KMeansLocalClusterer clusterer = new KMeansLocalClusterer(new EuclideanDistance(), 1, 1L); + Map data = new HashMap<>(); + data.put(0, new double[] {1.0, 1959, 325100}); + data.put(1, new double[] {1.0, 1960, 373200}); - double[] v1 = new double[] {1959, 325100}; - double[] v2 = new double[] {1960, 373200}; + KMeansTrainer trainer = new KMeansTrainer() + .withK(1); - DenseLocalOnHeapMatrix points = new DenseLocalOnHeapMatrix(new double[][] {v1, v2}); + KMeansModel knnMdl = trainer.fit( + new LocalDatasetBuilder<>(data, 2), + (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1), + (k, v) -> v[2] + ); - return clusterer.cluster(points, 1); + return knnMdl; } /** */ diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/ClusteringTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/clustering/ClusteringTestSuite.java index 85f61fa2c0cfc..80538a0cb89ea 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/ClusteringTestSuite.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/clustering/ClusteringTestSuite.java @@ -25,11 +25,8 @@ */ @RunWith(Suite.class) @Suite.SuiteClasses({ - KMeansDistributedClustererTestSingleNode.class, - KMeansDistributedClustererTestMultiNode.class, - KMeansLocalClustererTest.class, - FuzzyCMeansDistributedClustererTest.class, - FuzzyCMeansLocalClustererTest.class + KMeansTrainerTest.class, + KMeansModelTest.class }) public class ClusteringTestSuite { } diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/FuzzyCMeansDistributedClustererTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/clustering/FuzzyCMeansDistributedClustererTest.java deleted file mode 100644 index 4b415bb99b723..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/FuzzyCMeansDistributedClustererTest.java +++ /dev/null @@ -1,180 +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.clustering; - -import java.util.Arrays; -import java.util.Comparator; -import java.util.Random; -import org.apache.ignite.Ignite; -import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.math.StorageConstants; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.distances.DistanceMeasure; -import org.apache.ignite.ml.math.distances.EuclideanDistance; -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; - -/** Tests that checks distributed Fuzzy C-Means clusterer. */ -public class FuzzyCMeansDistributedClustererTest extends GridCommonAbstractTest { - /** Number of nodes in grid. */ - private static final int NODE_COUNT = 3; - - /** Grid instance. */ - private Ignite ignite; - - /** Default constructor. */ - public FuzzyCMeansDistributedClustererTest() { - super(false); - } - - /** {@inheritDoc} */ - @Override protected void beforeTest() throws Exception { - ignite = grid(NODE_COUNT); - } - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - for (int i = 1; i <= NODE_COUNT; i++) - startGrid(i); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() throws Exception { - stopAllGrids(); - } - - /** Test that algorithm gives correct results on a small sample - 4 centers on the plane. */ - public void testTwoDimensionsLittleData() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - - FuzzyCMeansDistributedClusterer clusterer = new FuzzyCMeansDistributedClusterer(new EuclideanDistance(), - 2, BaseFuzzyCMeansClusterer.StopCondition.STABLE_MEMBERSHIPS, - 0.01, 500, null, 2, 50); - - double[][] points = new double[][]{{-10, -10}, {-9, -11}, {-10, -9}, {-11, -9}, - {10, 10}, {9, 11}, {10, 9}, {11, 9}, - {-10, 10}, {-9, 11}, {-10, 9}, {-11, 9}, - {10, -10}, {9, -11}, {10, -9}, {11, -9}}; - - SparseDistributedMatrix pntMatrix = new SparseDistributedMatrix(16, 2, - StorageConstants.ROW_STORAGE_MODE, StorageConstants.RANDOM_ACCESS_MODE); - for (int i = 0; i < 16; i++) - pntMatrix.setRow(i, points[i]); - - FuzzyCMeansModel mdl = clusterer.cluster(pntMatrix, 4); - - Vector[] centers = mdl.centers(); - Arrays.sort(centers, Comparator.comparing(vector -> Math.atan2(vector.get(1), vector.get(0)))); - - DistanceMeasure measure = mdl.distanceMeasure(); - - assertEquals(0, measure.compute(centers[0], new DenseLocalOnHeapVector(new double[]{-10, -10})), 1); - assertEquals(0, measure.compute(centers[1], new DenseLocalOnHeapVector(new double[]{10, -10})), 1); - assertEquals(0, measure.compute(centers[2], new DenseLocalOnHeapVector(new double[]{10, 10})), 1); - assertEquals(0, measure.compute(centers[3], new DenseLocalOnHeapVector(new double[]{-10, 10})), 1); - - pntMatrix.destroy(); - } - - /** Perform N tests each of which contains M random points placed around K centers on the plane. */ - public void testTwoDimensionsRandomlyPlacedPointsAndCenters() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - - final int numOfTests = 5; - - final double exponentialWeight = 2.0; - final double maxCentersDelta = 0.01; - final int maxIterations = 500; - final Long seed = 1L; - - DistanceMeasure measure = new EuclideanDistance(); - FuzzyCMeansDistributedClusterer distributedClusterer = new FuzzyCMeansDistributedClusterer(measure, - exponentialWeight, BaseFuzzyCMeansClusterer.StopCondition.STABLE_CENTERS, - maxCentersDelta, maxIterations, seed, 2, 50); - - for (int i = 0; i < numOfTests; i++) - performRandomTest(distributedClusterer, i); - } - - /** - * Test given clusterer on points placed randomly around vertexes of a regular polygon. - * - * @param distributedClusterer Tested clusterer. - * @param seed Seed for the random numbers generator. - */ - private void performRandomTest(FuzzyCMeansDistributedClusterer distributedClusterer, long seed) { - final int minNumCenters = 2; - final int maxNumCenters = 5; - final double maxRadius = 1000; - final int maxPoints = 1000; - final int minPoints = 300; - - Random random = new Random(seed); - - int numCenters = random.nextInt(maxNumCenters - minNumCenters) + minNumCenters; - - double[][] centers = new double[numCenters][2]; - - for (int i = 0; i < numCenters; i++) { - double angle = Math.PI * 2.0 * i / numCenters; - - centers[i][0] = Math.cos(angle) * maxRadius; - centers[i][1] = Math.sin(angle) * maxRadius; - } - - int numPoints = minPoints + random.nextInt(maxPoints - minPoints); - - double[][] points = new double[numPoints][2]; - - for (int i = 0; i < numPoints; i++) { - int center = random.nextInt(numCenters); - double randomDouble = random.nextDouble(); - double radius = randomDouble * randomDouble * maxRadius / 10; - double angle = random.nextDouble() * Math.PI * 2.0; - - points[i][0] = centers[center][0] + Math.cos(angle) * radius; - points[i][1] = centers[center][1] + Math.sin(angle) * radius; - } - - SparseDistributedMatrix pntMatrix = new SparseDistributedMatrix(numPoints, 2, - StorageConstants.ROW_STORAGE_MODE, StorageConstants.RANDOM_ACCESS_MODE); - - for (int i = 0; i < numPoints; i++) - pntMatrix.setRow(i, points[i]); - - FuzzyCMeansModel mdl = distributedClusterer.cluster(pntMatrix, numCenters); - Vector[] computedCenters = mdl.centers(); - DistanceMeasure measure = mdl.distanceMeasure(); - - int cntr = numCenters; - - for (int i = 0; i < numCenters; i++) { - for (int j = 0; j < numCenters; j++) { - if (measure.compute(computedCenters[i], new DenseLocalOnHeapVector(centers[j])) < 100) { - cntr--; - break; - } - } - } - - assertEquals(0, cntr); - - pntMatrix.destroy(); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/FuzzyCMeansLocalClustererTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/clustering/FuzzyCMeansLocalClustererTest.java deleted file mode 100644 index 4fe1eee3fa078..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/FuzzyCMeansLocalClustererTest.java +++ /dev/null @@ -1,202 +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.clustering; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.distances.DistanceMeasure; -import org.apache.ignite.ml.math.distances.EuclideanDistance; -import org.apache.ignite.ml.math.exceptions.MathIllegalArgumentException; -import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -/** Tests that checks local Fuzzy C-Means clusterer. */ -public class FuzzyCMeansLocalClustererTest { - /** Test FCM on points that forms three clusters on the line. */ - @Test - public void equalWeightsOneDimension() { - FuzzyCMeansLocalClusterer clusterer = new FuzzyCMeansLocalClusterer(new EuclideanDistance(), - 2, BaseFuzzyCMeansClusterer.StopCondition.STABLE_CENTERS, - 0.01, 10, null); - - double[][] points = new double[][]{{-10}, {-9}, {-8}, {-7}, - {7}, {8}, {9}, {10}, - {-1}, {0}, {1}}; - - DenseLocalOnHeapMatrix pntMatrix = new DenseLocalOnHeapMatrix(points); - - FuzzyCMeansModel mdl = clusterer.cluster(pntMatrix, 3); - - Vector[] centers = mdl.centers(); - Arrays.sort(centers, Comparator.comparing(vector -> vector.getX(0))); - assertEquals(-8.5, centers[0].getX(0), 2); - assertEquals(0, centers[1].getX(0), 2); - assertEquals(8.5, centers[2].getX(0), 2); - } - - /** Test FCM on points that forms four clusters on the plane. */ - @Test - public void equalWeightsTwoDimensions() { - FuzzyCMeansLocalClusterer clusterer = new FuzzyCMeansLocalClusterer(new EuclideanDistance(), - 2, BaseFuzzyCMeansClusterer.StopCondition.STABLE_CENTERS, - 0.01, 20, null); - - double[][] points = new double[][]{{-10, -10}, {-9, -11}, {-10, -9}, {-11, -9}, - {10, 10}, {9, 11}, {10, 9}, {11, 9}, - {-10, 10}, {-9, 11}, {-10, 9}, {-11, 9}, - {10, -10}, {9, -11}, {10, -9}, {11, -9}}; - - DenseLocalOnHeapMatrix pntMatrix = new DenseLocalOnHeapMatrix(points); - - FuzzyCMeansModel mdl = clusterer.cluster(pntMatrix, 4); - Vector[] centers = mdl.centers(); - Arrays.sort(centers, Comparator.comparing(vector -> Math.atan2(vector.get(1), vector.get(0)))); - - DistanceMeasure measure = mdl.distanceMeasure(); - - assertEquals(0, measure.compute(centers[0], new DenseLocalOnHeapVector(new double[]{-10, -10})), 1); - assertEquals(0, measure.compute(centers[1], new DenseLocalOnHeapVector(new double[]{10, -10})), 1); - assertEquals(0, measure.compute(centers[2], new DenseLocalOnHeapVector(new double[]{10, 10})), 1); - assertEquals(0, measure.compute(centers[3], new DenseLocalOnHeapVector(new double[]{-10, 10})), 1); - } - - /** Test FCM on points which have the equal coordinates. */ - @Test - public void checkCentersOfTheSamePointsTwoDimensions() { - FuzzyCMeansLocalClusterer clusterer = new FuzzyCMeansLocalClusterer(new EuclideanDistance(), - 2, BaseFuzzyCMeansClusterer.StopCondition.STABLE_MEMBERSHIPS, 0.01, 10, null); - - double[][] points = new double[][] {{3.3, 10}, {3.3, 10}, {3.3, 10}, {3.3, 10}, {3.3, 10}}; - - DenseLocalOnHeapMatrix pntMatrix = new DenseLocalOnHeapMatrix(points); - - int k = 2; - FuzzyCMeansModel mdl = clusterer.cluster(pntMatrix, k); - Vector exp = new DenseLocalOnHeapVector(new double[] {3.3, 10}); - for (int i = 0; i < k; i++) { - Vector center = mdl.centers()[i]; - - for (int j = 0; j < 2; j++) - assertEquals(exp.getX(j), center.getX(j), 1); - } - } - - /** Test FCM on points located on the circle. */ - @Test - public void checkCentersLocationOnSphere() { - FuzzyCMeansLocalClusterer clusterer = new FuzzyCMeansLocalClusterer(new EuclideanDistance(), - 2, BaseFuzzyCMeansClusterer.StopCondition.STABLE_CENTERS, 0.01, 100, null); - - int numOfPoints = 650; - double radius = 100.0; - double[][] points = new double [numOfPoints][2]; - - for (int i = 0; i < numOfPoints; i++) { - points[i][0] = Math.cos(Math.PI * 2 * i / numOfPoints) * radius; - points[i][1] = Math.sin(Math.PI * 2 * i / numOfPoints) * radius; - } - - DenseLocalOnHeapMatrix pntMatrix = new DenseLocalOnHeapMatrix(points); - - int k = 10; - FuzzyCMeansModel mdl = clusterer.cluster(pntMatrix, k); - - Vector sum = mdl.centers()[0]; - for (int i = 1; i < k; i++) - sum = sum.plus(mdl.centers()[i]); - - assertEquals(0, sum.kNorm(1), 1); - } - - /** Test FCM on points that forms the line located on the plane. */ - @Test - public void test2DLineClustering() { - FuzzyCMeansLocalClusterer clusterer = new FuzzyCMeansLocalClusterer(new EuclideanDistance(), - 2, BaseFuzzyCMeansClusterer.StopCondition.STABLE_CENTERS, 0.01, 50, null); - - double[][] points = new double[][]{{1, 2}, {3, 6}, {5, 10}}; - - DenseLocalOnHeapMatrix pntMatrix = new DenseLocalOnHeapMatrix(points); - - int k = 2; - FuzzyCMeansModel mdl = clusterer.cluster(pntMatrix, k); - Vector[] centers = mdl.centers(); - Arrays.sort(centers, Comparator.comparing(vector -> vector.getX(0))); - - Vector[] exp = {new DenseLocalOnHeapVector(new double[]{1.5, 3}), - new DenseLocalOnHeapVector(new double[]{4.5, 9})}; - - for (int i = 0; i < k; i++) { - Vector center = centers[i]; - - for (int j = 0; j < 2; j++) - assertEquals(exp[i].getX(j), center.getX(j), 0.5); - } - } - - /** Test FCM on points that have different weights. */ - @Test - public void differentWeightsOneDimension() { - FuzzyCMeansLocalClusterer clusterer = new FuzzyCMeansLocalClusterer(new EuclideanDistance(), - 2, BaseFuzzyCMeansClusterer.StopCondition.STABLE_CENTERS, - 0.01, 10, null); - - double[][] points = new double[][]{{1}, {2}, {3}, {4}, {5}, {6}}; - - DenseLocalOnHeapMatrix pntMatrix = new DenseLocalOnHeapMatrix(points); - ArrayList weights = new ArrayList<>(); - Collections.addAll(weights, 3.0, 2.0, 1.0, 1.0, 1.0, 1.0); - - Vector[] centers1 = clusterer.cluster(pntMatrix, 2).centers(); - Vector[] centers2 = clusterer.cluster(pntMatrix, 2, weights).centers(); - Arrays.sort(centers1, Comparator.comparing(vector -> vector.getX(0))); - Arrays.sort(centers2, Comparator.comparing(vector -> vector.getX(0))); - - assertTrue(centers1[0].get(0) - centers2[0].get(0) > 0.5); - } - - /** Test FCM on illegal number of clusters. */ - @Test(expected = MathIllegalArgumentException.class) - public void testIllegalNumberOfClusters() { - FuzzyCMeansLocalClusterer clusterer = new FuzzyCMeansLocalClusterer(new EuclideanDistance(), - 2, BaseFuzzyCMeansClusterer.StopCondition.STABLE_CENTERS, 0.01, 10, null); - double[][] points = new double[][]{{1}, {2}, {3}, {4}}; - - clusterer.cluster(new DenseLocalOnHeapMatrix(points), 1); - } - - /** Test FCM on different numbers of points and weights. */ - @Test(expected = MathIllegalArgumentException.class) - public void testDifferentAmountsOfPointsAndWeights(){ - FuzzyCMeansLocalClusterer clusterer = new FuzzyCMeansLocalClusterer(new EuclideanDistance(), - 2, BaseFuzzyCMeansClusterer.StopCondition.STABLE_CENTERS, 0.01, 10, null); - double[][] points = new double[][]{{1}, {2}, {3}, {4}}; - - ArrayList weights = new ArrayList<>(); - Collections.addAll(weights, 1.0, 34.0, 2.5, 5.0, 0.5); - - clusterer.cluster(new DenseLocalOnHeapMatrix(points), 2, weights); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansDistributedClustererTestMultiNode.java b/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansDistributedClustererTestMultiNode.java deleted file mode 100644 index 71be8bef77e58..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansDistributedClustererTestMultiNode.java +++ /dev/null @@ -1,138 +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.clustering; - -import java.io.IOException; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.ignite.Ignite; -import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.math.StorageConstants; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.distances.EuclideanDistance; -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; - -/** - * This test is made to make sure that K-Means distributed clustering does not crash on distributed environment. - * In {@link KMeansDistributedClustererTestSingleNode} we check logic of clustering (checks for clusters structures). - * In this class we just check that clusterer does not crash. There are two separate tests because we cannot - * guarantee order in which nodes return results of intermediate computations and therefore algorithm can return - * different results. - */ -public class KMeansDistributedClustererTestMultiNode extends GridCommonAbstractTest { - /** Number of nodes in grid. */ - private static final int NODE_COUNT = 3; - - /** Grid instance. */ - private Ignite ignite; - - /** - * Default constructor. - */ - public KMeansDistributedClustererTestMultiNode() { - super(false); - } - - /** - * {@inheritDoc} - */ - @Override protected void beforeTest() throws Exception { - ignite = grid(NODE_COUNT); - } - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - for (int i = 1; i <= NODE_COUNT; i++) - startGrid(i); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() throws Exception { - stopAllGrids(); - } - - /** */ - public void testPerformClusterAnalysisDegenerate() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - - KMeansDistributedClusterer clusterer = new KMeansDistributedClusterer(new EuclideanDistance(), 1, 1, 1L); - - double[] v1 = new double[] {1959, 325100}; - double[] v2 = new double[] {1960, 373200}; - - SparseDistributedMatrix points = new SparseDistributedMatrix(2, 2, StorageConstants.ROW_STORAGE_MODE, - StorageConstants.RANDOM_ACCESS_MODE); - - points.setRow(0, v1); - points.setRow(1, v2); - - clusterer.cluster(points, 1); - - points.destroy(); - } - - /** */ - public void testClusterizationOnDatasetWithObviousStructure() throws IOException { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - - int ptsCnt = 10000; - int squareSideLen = 10000; - - Random rnd = new Random(123456L); - - // Let centers be in the vertices of square. - Map centers = new HashMap<>(); - centers.put(100, new DenseLocalOnHeapVector(new double[] {0.0, 0.0})); - centers.put(900, new DenseLocalOnHeapVector(new double[] {squareSideLen, 0.0})); - centers.put(3000, new DenseLocalOnHeapVector(new double[] {0.0, squareSideLen})); - centers.put(6000, new DenseLocalOnHeapVector(new double[] {squareSideLen, squareSideLen})); - - SparseDistributedMatrix points = new SparseDistributedMatrix(ptsCnt, 2, StorageConstants.ROW_STORAGE_MODE, - StorageConstants.RANDOM_ACCESS_MODE); - - List permutation = IntStream.range(0, ptsCnt).boxed().collect(Collectors.toList()); - Collections.shuffle(permutation, rnd); - - int totalCnt = 0; - - for (Integer count : centers.keySet()) { - for (int i = 0; i < count; i++) { - Vector pnt = new DenseLocalOnHeapVector(2).assign(centers.get(count)); - // Perturbate point on random value. - pnt.map(val -> val + rnd.nextDouble() * squareSideLen / 100); - points.assignRow(permutation.get(totalCnt), pnt); - totalCnt++; - } - } - - EuclideanDistance dist = new EuclideanDistance(); - - KMeansDistributedClusterer clusterer = new KMeansDistributedClusterer(dist, 3, 100, 1L); - - clusterer.cluster(points, 4); - - points.destroy(); - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansDistributedClustererTestSingleNode.java b/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansDistributedClustererTestSingleNode.java deleted file mode 100644 index 705db7ad35157..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansDistributedClustererTestSingleNode.java +++ /dev/null @@ -1,198 +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.clustering; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.ignite.Ignite; -import org.apache.ignite.internal.util.IgniteUtils; -import org.apache.ignite.ml.math.StorageConstants; -import org.apache.ignite.ml.math.Vector; -import org.apache.ignite.ml.math.VectorUtils; -import org.apache.ignite.ml.math.distances.DistanceMeasure; -import org.apache.ignite.ml.math.distances.EuclideanDistance; -import org.apache.ignite.ml.math.functions.Functions; -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import org.junit.Assert; - -import static org.apache.ignite.ml.clustering.KMeansUtil.checkIsInEpsilonNeighbourhood; - -/** - * This test checks logic of clustering (checks for clusters structures). - */ -public class KMeansDistributedClustererTestSingleNode extends GridCommonAbstractTest { - /** - * Number of nodes in grid. We should use 1 in this test because otherwise algorithm will be unstable - * (We cannot guarantee the order in which results are returned from each node). - */ - private static final int NODE_COUNT = 1; - - /** Grid instance. */ - private Ignite ignite; - - /** - * Default constructor. - */ - public KMeansDistributedClustererTestSingleNode() { - super(false); - } - - /** - * {@inheritDoc} - */ - @Override protected void beforeTest() throws Exception { - ignite = grid(NODE_COUNT); - } - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - for (int i = 1; i <= NODE_COUNT; i++) - startGrid(i); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() throws Exception { - stopAllGrids(); - } - - /** */ - public void testPerformClusterAnalysisDegenerate() { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - - KMeansDistributedClusterer clusterer = new KMeansDistributedClusterer(new EuclideanDistance(), 1, 1, 1L); - - double[] v1 = new double[] {1959, 325100}; - double[] v2 = new double[] {1960, 373200}; - - SparseDistributedMatrix points = new SparseDistributedMatrix(2, 2, StorageConstants.ROW_STORAGE_MODE, - StorageConstants.RANDOM_ACCESS_MODE); - - points.setRow(0, v1); - points.setRow(1, v2); - - KMeansModel mdl = clusterer.cluster(points, 1); - - Assert.assertEquals(1, mdl.centers().length); - Assert.assertEquals(2, mdl.centers()[0].size()); - } - - /** */ - public void testClusterizationOnDatasetWithObviousStructure() throws IOException { - IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName()); - - int ptsCnt = 10000; - int squareSideLen = 10000; - - Random rnd = new Random(123456L); - - // Let centers be in the vertices of square. - Map centers = new HashMap<>(); - centers.put(100, new DenseLocalOnHeapVector(new double[] {0.0, 0.0})); - centers.put(900, new DenseLocalOnHeapVector(new double[] {squareSideLen, 0.0})); - centers.put(3000, new DenseLocalOnHeapVector(new double[] {0.0, squareSideLen})); - centers.put(6000, new DenseLocalOnHeapVector(new double[] {squareSideLen, squareSideLen})); - - int centersCnt = centers.size(); - - SparseDistributedMatrix points = new SparseDistributedMatrix(ptsCnt, 2, StorageConstants.ROW_STORAGE_MODE, - StorageConstants.RANDOM_ACCESS_MODE); - - List permutation = IntStream.range(0, ptsCnt).boxed().collect(Collectors.toList()); - Collections.shuffle(permutation, rnd); - - Vector[] mc = new Vector[centersCnt]; - Arrays.fill(mc, VectorUtils.zeroes(2)); - - int centIdx = 0; - int totalCnt = 0; - - List massCenters = new ArrayList<>(); - - for (Integer count : centers.keySet()) { - for (int i = 0; i < count; i++) { - Vector pnt = new DenseLocalOnHeapVector(2).assign(centers.get(count)); - // Perturbate point on random value. - pnt.map(val -> val + rnd.nextDouble() * squareSideLen / 100); - mc[centIdx] = mc[centIdx].plus(pnt); - points.assignRow(permutation.get(totalCnt), pnt); - totalCnt++; - } - massCenters.add(mc[centIdx].times(1 / (double)count)); - centIdx++; - } - - EuclideanDistance dist = new EuclideanDistance(); - OrderedNodesComparator comp = new OrderedNodesComparator(centers.values().toArray(new Vector[] {}), dist); - - massCenters.sort(comp); - KMeansDistributedClusterer clusterer = new KMeansDistributedClusterer(dist, 3, 100, 1L); - - KMeansModel mdl = clusterer.cluster(points, 4); - Vector[] resCenters = mdl.centers(); - Arrays.sort(resCenters, comp); - - checkIsInEpsilonNeighbourhood(resCenters, massCenters.toArray(new Vector[] {}), 30.0); - - points.destroy(); - } - - /** */ - private static class OrderedNodesComparator implements Comparator { - /** */ - private final DistanceMeasure measure; - - /** */ - List orderedNodes; - - /** */ - OrderedNodesComparator(Vector[] orderedNodes, DistanceMeasure measure) { - this.orderedNodes = Arrays.asList(orderedNodes); - this.measure = measure; - } - - /** */ - private int findClosestNodeIndex(Vector v) { - return Functions.argmin(orderedNodes, v1 -> measure.compute(v1, v)).get1(); - } - - /** */ - @Override public int compare(Vector v1, Vector v2) { - int ind1 = findClosestNodeIndex(v1); - int ind2 = findClosestNodeIndex(v2); - - int signum = (int)Math.signum(ind1 - ind2); - - if (signum != 0) - return signum; - - return (int)Math.signum(orderedNodes.get(ind1).minus(v1).kNorm(2) - - orderedNodes.get(ind2).minus(v2).kNorm(2)); - } - } -} diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansLocalClustererTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansLocalClustererTest.java deleted file mode 100644 index cd9b2ede9d5b5..0000000000000 --- a/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansLocalClustererTest.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.ml.clustering; - -import org.apache.ignite.ml.math.distances.EuclideanDistance; -import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; -import org.junit.Assert; -import org.junit.Test; - -/** */ -public class KMeansLocalClustererTest { - /** - * Two points, one cluster, one iteration - */ - @Test - public void testPerformClusterAnalysisDegenerate() { - KMeansLocalClusterer clusterer = new KMeansLocalClusterer(new EuclideanDistance(), 1, 1L); - - double[] v1 = new double[] {1959, 325100}; - double[] v2 = new double[] {1960, 373200}; - - DenseLocalOnHeapMatrix points = new DenseLocalOnHeapMatrix(new double[][] { - v1, - v2}); - - KMeansModel mdl = clusterer.cluster(points, 1); - - Assert.assertEquals(1, mdl.centers().length); - Assert.assertEquals(2, mdl.centers()[0].size()); - } -} 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 new file mode 100644 index 0000000000000..5b3ad85040482 --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansModelTest.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.ml.clustering; + +import org.apache.ignite.ml.TestUtils; +import org.apache.ignite.ml.clustering.kmeans.KMeansModel; +import org.apache.ignite.ml.math.Vector; +import org.apache.ignite.ml.math.distances.DistanceMeasure; +import org.apache.ignite.ml.math.distances.EuclideanDistance; +import org.apache.ignite.ml.math.exceptions.CardinalityException; +import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; +import org.apache.ignite.ml.regressions.linear.LinearRegressionModel; +import org.apache.ignite.ml.svm.SVMLinearBinaryClassificationModel; +import org.apache.ignite.ml.svm.SVMLinearMultiClassClassificationModel; +import org.junit.Assert; +import org.junit.Test; + +/** + * Tests for {@link KMeansModel}. + */ +public class KMeansModelTest { + /** Precision in test checks. */ + private static final double PRECISION = 1e-6; + + /** */ + @Test + public void predictClusters() { + DistanceMeasure distanceMeasure = new EuclideanDistance(); + + Vector[] centers = new DenseLocalOnHeapVector[4]; + + centers[0] = new DenseLocalOnHeapVector(new double[]{1.0, 1.0}); + centers[1] = new DenseLocalOnHeapVector(new double[]{-1.0, 1.0}); + centers[2] = new DenseLocalOnHeapVector(new double[]{1.0, -1.0}); + centers[3] = new DenseLocalOnHeapVector(new double[]{-1.0, -1.0}); + + KMeansModel mdl = new KMeansModel(centers, distanceMeasure); + + Assert.assertEquals(mdl.apply(new DenseLocalOnHeapVector(new double[]{1.1, 1.1})), 0.0, PRECISION); + Assert.assertEquals(mdl.apply(new DenseLocalOnHeapVector(new double[]{-1.1, 1.1})), 1.0, PRECISION); + Assert.assertEquals(mdl.apply(new DenseLocalOnHeapVector(new double[]{1.1, -1.1})), 2.0, PRECISION); + Assert.assertEquals(mdl.apply(new DenseLocalOnHeapVector(new double[]{-1.1, -1.1})), 3.0, PRECISION); + + Assert.assertEquals(mdl.distanceMeasure(), distanceMeasure); + Assert.assertEquals(mdl.amountOfClusters(), 4); + Assert.assertArrayEquals(mdl.centers(), 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 new file mode 100644 index 0000000000000..846d0de86013e --- /dev/null +++ b/modules/ml/src/test/java/org/apache/ignite/ml/clustering/KMeansTrainerTest.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.clustering; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.apache.ignite.ml.clustering.kmeans.KMeansModel; +import org.apache.ignite.ml.clustering.kmeans.KMeansTrainer; +import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder; +import org.apache.ignite.ml.math.Vector; +import org.apache.ignite.ml.math.distances.EuclideanDistance; +import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link KMeansTrainer}. + */ +public class KMeansTrainerTest { + /** Precision in test checks. */ + private static final double PRECISION = 1e-2; + + /** + * 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 = new KMeansTrainer() + .withDistance(new EuclideanDistance()) + .withK(1) + .withMaxIterations(1) + .withEpsilon(PRECISION); + + KMeansModel knnMdl = trainer.fit( + new LocalDatasetBuilder<>(data, 2), + (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1), + (k, v) -> v[2] + ); + + Vector firstVector = new DenseLocalOnHeapVector(new double[] {2.0, 2.0}); + assertEquals(knnMdl.apply(firstVector), 0.0, PRECISION); + Vector secondVector = new DenseLocalOnHeapVector(new double[] {-2.0, -2.0}); + assertEquals(knnMdl.apply(secondVector), 0.0, PRECISION); + assertEquals(trainer.getMaxIterations(), 1); + assertEquals(trainer.getEpsilon(), PRECISION, PRECISION); + } +} From 6ee5e9fe8c07353a55a2a02b84f7b94c43b5c49f Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Tue, 17 Apr 2018 11:46:45 +0700 Subject: [PATCH 0057/1463] IGNITE-8201 REST: Added AUTHENTICATE command. Fixed session tokens. Added new tests. (cherry picked from commit 1cfc989) --- .../client/suite/IgniteClientTestSuite.java | 6 +- .../JettyRestProcessorAbstractSelfTest.java | 60 +++++++++----- ...tyRestProcessorAuthenticationSelfTest.java | 45 ++--------- ...cessorAuthenticationWithCredsSelfTest.java | 32 ++++++++ ...cessorAuthenticationWithTokenSelfTest.java | 80 +++++++++++++++++++ .../processors/rest/GridRestCommand.java | 3 + .../processors/rest/GridRestProcessor.java | 26 +++--- .../auth/AuthenticationCommandHandler.java | 70 ++++++++++++++++ .../rest/handlers/auth/package-info.java | 22 +++++ .../http/jetty/GridJettyRestHandler.java | 55 ++++++++++--- 10 files changed, 322 insertions(+), 77 deletions(-) create mode 100644 modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationWithCredsSelfTest.java create mode 100644 modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationWithTokenSelfTest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/auth/AuthenticationCommandHandler.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/auth/package-info.java diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java index 79fcf38c67e1a..163f89acb7a35 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java @@ -49,7 +49,8 @@ import org.apache.ignite.internal.client.util.ClientConsistentHashSelfTest; import org.apache.ignite.internal.client.util.ClientJavaHasherSelfTest; import org.apache.ignite.internal.processors.rest.ClientMemcachedProtocolSelfTest; -import org.apache.ignite.internal.processors.rest.JettyRestProcessorAuthenticationSelfTest; +import org.apache.ignite.internal.processors.rest.JettyRestProcessorAuthenticationWithCredsSelfTest; +import org.apache.ignite.internal.processors.rest.JettyRestProcessorAuthenticationWithTokenSelfTest; import org.apache.ignite.internal.processors.rest.JettyRestProcessorSignedSelfTest; import org.apache.ignite.internal.processors.rest.JettyRestProcessorUnsignedSelfTest; import org.apache.ignite.internal.processors.rest.RestBinaryProtocolSelfTest; @@ -87,7 +88,8 @@ public static TestSuite suite() { // Test jetty rest processor suite.addTestSuite(JettyRestProcessorSignedSelfTest.class); suite.addTestSuite(JettyRestProcessorUnsignedSelfTest.class); - suite.addTestSuite(JettyRestProcessorAuthenticationSelfTest.class); + suite.addTestSuite(JettyRestProcessorAuthenticationWithCredsSelfTest.class); + suite.addTestSuite(JettyRestProcessorAuthenticationWithTokenSelfTest.class); // Test TCP rest processor with original memcache client. suite.addTestSuite(ClientMemcachedProtocolSelfTest.class); diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java index 5dc44c4145516..e36447bc3e0c4 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java @@ -294,14 +294,19 @@ protected void assertResponseContainsError(String content, String err) throws IO /** * @param content Content to check. + * @return JSON node with actual response. */ - private JsonNode jsonCacheOperationResponse(String content, boolean bulk) throws IOException { + protected JsonNode assertResponseSucceeded(String content, boolean bulk) throws IOException { assertNotNull(content); assertFalse(content.isEmpty()); JsonNode node = JSON_MAPPER.readTree(content); - assertEquals(bulk, node.get("affinityNodeId").isNull()); + JsonNode affNode = node.get("affinityNodeId"); + + if (affNode != null) + assertEquals(bulk, affNode.isNull()); + assertEquals(STATUS_SUCCESS, node.get("successStatus").asInt()); assertTrue(node.get("error").isNull()); @@ -315,7 +320,7 @@ private JsonNode jsonCacheOperationResponse(String content, boolean bulk) throws * @param res Response. */ private void assertCacheOperation(String content, Object res) throws IOException { - JsonNode ret = jsonCacheOperationResponse(content, false); + JsonNode ret = assertResponseSucceeded(content, false); assertEquals(String.valueOf(res), ret.isObject() ? ret.toString() : ret.asText()); } @@ -325,7 +330,7 @@ private void assertCacheOperation(String content, Object res) throws IOException * @param res Response. */ private void assertCacheBulkOperation(String content, Object res) throws IOException { - JsonNode ret = jsonCacheOperationResponse(content, true); + JsonNode ret = assertResponseSucceeded(content, true); assertEquals(String.valueOf(res), ret.asText()); } @@ -334,7 +339,7 @@ private void assertCacheBulkOperation(String content, Object res) throws IOExcep * @param content Content to check. */ private void assertCacheMetrics(String content) throws IOException { - JsonNode ret = jsonCacheOperationResponse(content, true); + JsonNode ret = assertResponseSucceeded(content, true); assertTrue(ret.isObject()); } @@ -349,7 +354,7 @@ protected JsonNode jsonResponse(String content) throws IOException { JsonNode node = JSON_MAPPER.readTree(content); - assertEquals(0, node.get("successStatus").asInt()); + assertEquals(STATUS_SUCCESS, node.get("successStatus").asInt()); assertTrue(node.get("error").isNull()); assertNotSame(securityEnabled(), node.get("sessionToken").isNull()); @@ -367,7 +372,7 @@ protected JsonNode jsonTaskResult(String content) throws IOException { JsonNode node = JSON_MAPPER.readTree(content); - assertEquals(0, node.get("successStatus").asInt()); + assertEquals(STATUS_SUCCESS, node.get("successStatus").asInt()); assertTrue(node.get("error").isNull()); assertFalse(node.get("response").isNull()); @@ -403,7 +408,7 @@ public void testGet() throws Exception { * @throws IOException If failed. */ private void checkJson(String json, Person p) throws IOException { - JsonNode res = jsonCacheOperationResponse(json, false); + JsonNode res = assertResponseSucceeded(json, false); assertEquals(p.id.intValue(), res.get("id").asInt()); assertEquals(p.getOrganizationId().intValue(), res.get("orgId").asInt()); @@ -455,7 +460,7 @@ public void testGetBinaryObjects() throws Exception { info("Get command result: " + ret); - JsonNode res = jsonCacheOperationResponse(ret, false); + JsonNode res = assertResponseSucceeded(ret, false); assertEquals("Alex", res.get("NAME").asText()); assertEquals(300, res.get("SALARY").asInt()); @@ -476,7 +481,7 @@ public void testGetBinaryObjects() throws Exception { info("Get command result: " + ret); - JsonNode json = jsonCacheOperationResponse(ret, false); + JsonNode json = assertResponseSucceeded(ret, false); assertEquals(ref1.name, json.get("name").asText()); ref2.ref(ref1); @@ -552,7 +557,7 @@ public void testSimpleObject() throws Exception { info("Get command result: " + ret); - JsonNode res = jsonCacheOperationResponse(ret, false); + JsonNode res = assertResponseSucceeded(ret, false); assertEquals(p.id, res.get("id").asInt()); assertEquals(p.name, res.get("name").asText()); @@ -637,7 +642,7 @@ public void testTuple() throws Exception { info("Get command result: " + ret); - JsonNode res = jsonCacheOperationResponse(ret, false); + JsonNode res = assertResponseSucceeded(ret, false); assertEquals(t.getKey(), res.get("key").asText()); assertEquals(t.getValue(), res.get("value").asText()); @@ -775,11 +780,11 @@ public void testGetAll() throws Exception { info("Get all command result: " + ret); - JsonNode res = jsonCacheOperationResponse(ret, true); + JsonNode res = assertResponseSucceeded(ret, true); assertTrue(res.isObject()); - assertTrue(entries.equals(JSON_MAPPER.treeToValue(res, Map.class))); + assertEquals(entries, JSON_MAPPER.treeToValue(res, Map.class)); } /** @@ -973,10 +978,25 @@ public void testPut() throws Exception { assertCacheOperation(ret, true); } + /** */ + private void failIgnite_5874() { + DataStorageConfiguration dsCfg = ignite(0).configuration().getDataStorageConfiguration(); + + if (dsCfg.getDefaultDataRegionConfiguration().isPersistenceEnabled()) + fail("IGNITE-5874"); + + for (DataRegionConfiguration dataRegCfg : dsCfg.getDataRegionConfigurations()) { + if (dataRegCfg.isPersistenceEnabled()) + fail("IGNITE-5874"); + } + } + /** * @throws Exception If failed. */ public void testPutWithExpiration() throws Exception { + failIgnite_5874(); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_PUT, "key", "putKey", "val", "putVal", @@ -1013,6 +1033,8 @@ public void testAdd() throws Exception { * @throws Exception If failed. */ public void testAddWithExpiration() throws Exception { + failIgnite_5874(); + String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_ADD, "key", "addKey", "val", "addVal", @@ -1100,7 +1122,7 @@ public void testRemoveAll() throws Exception { assertNull(jcache().localPeek("rmvKey2")); assertNull(jcache().localPeek("rmvKey3")); assertNull(jcache().localPeek("rmvKey4")); - assertTrue(jcache().localSize() == 0); + assertEquals(0, jcache().localSize()); assertCacheBulkOperation(ret, true); } @@ -1152,6 +1174,8 @@ public void testReplace() throws Exception { * @throws Exception If failed. */ public void testReplaceWithExpiration() throws Exception { + failIgnite_5874(); + jcache().put("replaceKey", "replaceVal"); assertEquals("replaceVal", jcache().get("replaceKey")); @@ -1353,20 +1377,20 @@ private void testMetadata(Collection metas, JsonNode arr) assertNotNull(keyClasses); assertFalse(keyClasses.isNull()); - assertTrue(meta.keyClasses().equals(JSON_MAPPER.treeToValue(keyClasses, Map.class))); + assertEquals(meta.keyClasses(), JSON_MAPPER.treeToValue(keyClasses, Map.class)); JsonNode valClasses = item.get("valClasses"); assertNotNull(valClasses); assertFalse(valClasses.isNull()); - assertTrue(meta.valClasses().equals(JSON_MAPPER.treeToValue(valClasses, Map.class))); + assertEquals(meta.valClasses(), JSON_MAPPER.treeToValue(valClasses, Map.class)); JsonNode fields = item.get("fields"); assertNotNull(fields); assertFalse(fields.isNull()); - assertTrue(meta.fields().equals(JSON_MAPPER.treeToValue(fields, Map.class))); + assertEquals(meta.fields(), JSON_MAPPER.treeToValue(fields, Map.class)); JsonNode indexesByType = item.get("indexes"); diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationSelfTest.java index ca62091a2ecee..27b8c03ff696d 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationSelfTest.java @@ -24,7 +24,6 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.processors.authentication.IgniteAccessControlException; import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor; -import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridTestUtils; @@ -33,18 +32,12 @@ /** * Test REST with enabled authentication. */ -public class JettyRestProcessorAuthenticationSelfTest extends JettyRestProcessorUnsignedSelfTest { +public abstract class JettyRestProcessorAuthenticationSelfTest extends JettyRestProcessorUnsignedSelfTest { /** */ - private static final String DFLT_LOGIN = "ignite"; + protected static final String DFLT_USER = "ignite"; /** */ - private static final String DFLT_PWD = "ignite"; - - /** */ - private String login = DFLT_LOGIN; - - /** */ - private String pwd = DFLT_PWD; + protected static final String DFLT_PWD = "ignite"; /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { @@ -54,11 +47,8 @@ public class JettyRestProcessorAuthenticationSelfTest extends JettyRestProcessor } /** {@inheritDoc} */ - @Override protected void beforeTest() throws Exception { - super.beforeTest(); - - login = DFLT_LOGIN; - pwd = DFLT_PWD; + @Override protected boolean securityEnabled() { + return true; } /** {@inheritDoc} */ @@ -97,32 +87,13 @@ public class JettyRestProcessorAuthenticationSelfTest extends JettyRestProcessor return cfg; } - /** {@inheritDoc} */ - @Override protected String restUrl() { - String url = super.restUrl(); - - if (!F.isEmpty(login)) { - url += "ignite.login=" + login; - - if (!F.isEmpty(pwd)) - url += "&ignite.password=" + pwd; - - url += '&'; - } - - return url; - } - /** * @throws Exception If failed. */ - public void testMissingCredentials() throws Exception { - login = null; - pwd = null; - - String ret = content(null, GridRestCommand.VERSION); + public void testAuthenticationCommand() throws Exception { + String ret = content(null, GridRestCommand.AUTHENTICATE); - assertResponseContainsError(ret, "The user name or password is incorrect"); + assertResponseSucceeded(ret, false); } /** diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationWithCredsSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationWithCredsSelfTest.java new file mode 100644 index 0000000000000..c75e8a9797a1c --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationWithCredsSelfTest.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.rest; + +/** + * Test REST with enabled authentication and credentials in each request. + */ +public class JettyRestProcessorAuthenticationWithCredsSelfTest extends JettyRestProcessorAuthenticationSelfTest { + /** {@inheritDoc} */ + @Override protected String restUrl() { + String url = super.restUrl(); + + url += "ignite.login=" + DFLT_USER + "&ignite.password=" + DFLT_PWD + "&"; + + return url; + } +} diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationWithTokenSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationWithTokenSelfTest.java new file mode 100644 index 0000000000000..5c046af320394 --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAuthenticationWithTokenSelfTest.java @@ -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. + */ + +package org.apache.ignite.internal.processors.rest; + +import org.apache.ignite.internal.util.typedef.F; + +/** + * Test REST with enabled authentication and token. + */ +public class JettyRestProcessorAuthenticationWithTokenSelfTest extends JettyRestProcessorAuthenticationSelfTest { + /** */ + private String tok = ""; + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + // Authenticate and extract token. + if (F.isEmpty(tok)) { + String ret = content(null, GridRestCommand.AUTHENTICATE, + "user", DFLT_USER, + "password", DFLT_PWD); + + int p1 = ret.indexOf("sessionToken"); + int p2 = ret.indexOf('"', p1 + 16); + + tok = ret.substring(p1 + 15, p2); + } + } + + /** {@inheritDoc} */ + @Override protected String restUrl() { + String url = super.restUrl(); + + if (!F.isEmpty(tok)) + url += "sessionToken=" + tok + "&"; + + return url; + } + + /** + * @throws Exception If failed. + */ + public void testInvalidSessionToken() throws Exception { + tok = null; + + String ret = content(null, GridRestCommand.VERSION); + + assertResponseContainsError(ret, "Failed to handle request - session token not found or invalid"); + + tok = "InvalidToken"; + + ret = content(null, GridRestCommand.VERSION); + + assertResponseContainsError(ret, "Failed to handle request - session token not found or invalid"); + + tok = "26BE027D32CC42329DEC92D517B44E9E"; + + ret = content(null, GridRestCommand.VERSION); + + assertResponseContainsError(ret, "Failed to handle request - unknown session token (maybe expired session)"); + + tok = null; // Cleanup token for next tests. + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java index 0b9a66212c52d..265fe4024c7c9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java @@ -168,6 +168,9 @@ public enum GridRestCommand { /** */ CLUSTER_CURRENT_STATE("currentstate"), + /** */ + AUTHENTICATE("authenticate"), + /** */ ADD_USER("adduser"), 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 4b8497eccf28d..da5e5c205525a 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 @@ -45,6 +45,7 @@ import org.apache.ignite.internal.processors.authentication.AuthorizationContext; import org.apache.ignite.internal.processors.rest.client.message.GridClientTaskResultBean; import org.apache.ignite.internal.processors.rest.handlers.GridRestCommandHandler; +import org.apache.ignite.internal.processors.rest.handlers.auth.AuthenticationCommandHandler; import org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler; import org.apache.ignite.internal.processors.rest.handlers.cluster.GridChangeStateCommandHandler; import org.apache.ignite.internal.processors.rest.handlers.datastructures.DataStructuresCommandHandler; @@ -81,6 +82,7 @@ import org.apache.ignite.thread.IgniteThread; import static org.apache.ignite.IgniteSystemProperties.IGNITE_REST_START_ON_CLIENT; +import static org.apache.ignite.internal.processors.rest.GridRestCommand.AUTHENTICATE; import static org.apache.ignite.internal.processors.rest.GridRestResponse.STATUS_AUTH_FAILED; import static org.apache.ignite.internal.processors.rest.GridRestResponse.STATUS_FAILED; import static org.apache.ignite.internal.processors.rest.GridRestResponse.STATUS_SECURITY_CHECK_FAILED; @@ -230,10 +232,11 @@ private IgniteInternalFuture handleRequest(final GridRestReque try { ses = session(req); } + catch (IgniteAuthenticationException e) { + return new GridFinishedFuture<>(new GridRestResponse(STATUS_AUTH_FAILED, e.getMessage())); + } catch (IgniteCheckedException e) { - GridRestResponse res = new GridRestResponse(STATUS_FAILED, e.getMessage()); - - return new GridFinishedFuture<>(res); + return new GridFinishedFuture<>(new GridRestResponse(STATUS_FAILED, e.getMessage())); } assert ses != null; @@ -284,9 +287,9 @@ private IgniteInternalFuture handleRequest(final GridRestReque throw new IgniteAuthenticationException("The user name or password is incorrect"); ses.authCtx = ctx.authentication().authenticate(login, pwd); - - req.authorizationContext(ses.authCtx); } + + req.authorizationContext(ses.authCtx); } catch (IgniteCheckedException e) { return new GridFinishedFuture<>(new GridRestResponse(STATUS_AUTH_FAILED, e.getMessage())); @@ -341,7 +344,7 @@ private IgniteInternalFuture handleRequest(final GridRestReque assert res != null; - if (ctx.security().enabled() && !failed) + if ((authenticationEnabled || securityEnabled) && !failed) res.sessionTokenBytes(req.sessionToken()); interceptResponse(res, req); @@ -362,6 +365,10 @@ private Session session(final GridRestRequest req) throws IgniteCheckedException while (true) { if (F.isEmpty(sesTok) && clientId == null) { + // TODO: In IGNITE 3.0 we should check credentials only for AUTHENTICATE command. + if (ctx.authentication().enabled() && req.command() != AUTHENTICATE && req.credentials() == null) + throw new IgniteAuthenticationException("Failed to handle request - session token not found or invalid"); + Session ses = Session.random(); UUID oldSesId = clientId2SesId.put(ses.clientId, ses.sesId); @@ -451,10 +458,7 @@ public GridRestProcessor(GridKernalContext ctx) { try { sesExpTime = System.getProperty(IgniteSystemProperties.IGNITE_REST_SESSION_TIMEOUT); - if (sesExpTime != null) - sesExpTime0 = Long.valueOf(sesExpTime) * 1000; - else - sesExpTime0 = DEFAULT_SES_TIMEOUT; + sesExpTime0 = sesExpTime != null ? Long.valueOf(sesExpTime) * 1000 : DEFAULT_SES_TIMEOUT; } catch (NumberFormatException ignore) { U.warn(log, "Failed parsing IGNITE_REST_SESSION_TIMEOUT system variable [IGNITE_REST_SESSION_TIMEOUT=" @@ -504,6 +508,7 @@ public GridRestProcessor(GridKernalContext ctx) { addHandler(new QueryCommandHandler(ctx)); addHandler(new GridLogCommandHandler(ctx)); addHandler(new GridChangeStateCommandHandler(ctx)); + addHandler(new AuthenticationCommandHandler(ctx)); addHandler(new UserActionCommandHandler(ctx)); // Start protocols. @@ -860,6 +865,7 @@ private void authorize(GridRestRequest req, SecurityContext sCtx) throws Securit case CLUSTER_CURRENT_STATE: case CLUSTER_ACTIVE: case CLUSTER_INACTIVE: + case AUTHENTICATE: case ADD_USER: case REMOVE_USER: case UPDATE_USER: diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/auth/AuthenticationCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/auth/AuthenticationCommandHandler.java new file mode 100644 index 0000000000000..aa9bbbdacd50b --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/auth/AuthenticationCommandHandler.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.rest.handlers.auth; + +import java.util.Collection; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.rest.GridRestCommand; +import org.apache.ignite.internal.processors.rest.GridRestResponse; +import org.apache.ignite.internal.processors.rest.handlers.GridRestCommandHandlerAdapter; +import org.apache.ignite.internal.processors.rest.request.GridRestRequest; +import org.apache.ignite.internal.util.future.GridFinishedFuture; +import org.apache.ignite.internal.util.typedef.internal.U; + +import static org.apache.ignite.internal.processors.rest.GridRestCommand.AUTHENTICATE; + +/** + * Authentication handler. + */ +public class AuthenticationCommandHandler extends GridRestCommandHandlerAdapter { + /** Commands. */ + private static final Collection SUPPORTED_COMMANDS = U.sealList(AUTHENTICATE); + + /** + * @param ctx Context. + */ + public AuthenticationCommandHandler(GridKernalContext ctx) { + super(ctx); + } + + /** {@inheritDoc} */ + @Override public Collection supportedCommands() { + return SUPPORTED_COMMANDS; + } + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture handleAsync(GridRestRequest req) { + assert req != null; + + if (log.isDebugEnabled()) + log.debug("Handling topology REST request: " + req); + + try { + if (log.isDebugEnabled()) + log.debug("Handled topology REST request [req=" + req + ']'); + + return new GridFinishedFuture<>(new GridRestResponse(true)); + } + catch (Throwable e) { + log.error("Failed to handle REST request [req=" + req + ']', e); + + return new GridFinishedFuture<>(e); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/auth/package-info.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/auth/package-info.java new file mode 100644 index 0000000000000..1d58218422ed1 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/auth/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. + */ + +/** + * + * REST authentication command. + */ +package org.apache.ignite.internal.processors.rest.handlers.auth; diff --git a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java index b3fbddd93ff4b..99a884411cfd9 100644 --- a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java +++ b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java @@ -85,6 +85,12 @@ public class GridJettyRestHandler extends AbstractHandler { /** Used to sent request charset. */ private static final String CHARSET = StandardCharsets.UTF_8.name(); + /** */ + private static final String USER_PARAM = "user"; + + /** */ + private static final String PWD_PARAM = "password"; + /** */ private static final String CACHE_NAME_PARAM = "cacheName"; @@ -100,10 +106,10 @@ public class GridJettyRestHandler extends AbstractHandler { /** */ private static final String WRITE_SYNCHRONIZATION_MODE_PARAM = "writeSynchronizationMode"; - /** */ + /**@deprecated Should be replaced with AUTHENTICATION + token in IGNITE 3.0 */ private static final String IGNITE_LOGIN = "ignite.login"; - /** */ + /**@deprecated Should be replaced with AUTHENTICATION + token in IGNITE 3.0 */ private static final String IGNITE_PASSWORD = "ignite.password"; /** */ @@ -731,6 +737,12 @@ private Object convert(String type, Object obj) throws IgniteCheckedException { break; } + case AUTHENTICATE: { + restReq = new GridRestRequest(); + + break; + } + case ADD_USER: case REMOVE_USER: case UPDATE_USER: { @@ -840,12 +852,9 @@ private Object convert(String type, Object obj) throws IgniteCheckedException { restReq.command(cmd); - if (params.containsKey(IGNITE_LOGIN) || params.containsKey(IGNITE_PASSWORD)) { - SecurityCredentials cred = new SecurityCredentials( - (String)params.get(IGNITE_LOGIN), (String)params.get(IGNITE_PASSWORD)); - - restReq.credentials(cred); - } + // TODO: In IGNITE 3.0 we should check credentials only for AUTHENTICATE command. + if (!credentials(params, IGNITE_LOGIN, IGNITE_PASSWORD, restReq)) + credentials(params, USER_PARAM, PWD_PARAM, restReq); String clientId = (String)params.get("clientId"); @@ -870,8 +879,13 @@ private Object convert(String type, Object obj) throws IgniteCheckedException { String sesTokStr = (String)params.get("sessionToken"); try { - if (sesTokStr != null) - restReq.sessionToken(U.hexString2ByteArray(sesTokStr)); + if (sesTokStr != null) { + // Token is a UUID encoded as 16 bytes as HEX. + byte[] bytes = U.hexString2ByteArray(sesTokStr); + + if (bytes.length == 16) + restReq.sessionToken(bytes); + } } catch (IllegalArgumentException ignored) { // Ignore invalid session token. @@ -880,6 +894,27 @@ private Object convert(String type, Object obj) throws IgniteCheckedException { return restReq; } + /** + * + * @param params Parameters. + * @param userParam Parameter name to take user name. + * @param pwdParam Parameter name to take password. + * @param restReq Request to add credentials if any. + * @return {@code true} If params contains credentials. + */ + private boolean credentials(Map params, String userParam, String pwdParam, GridRestRequest restReq) { + boolean hasCreds = params.containsKey(userParam) || params.containsKey(pwdParam); + + if (hasCreds) { + SecurityCredentials cred = new SecurityCredentials((String)params.get(userParam), + (String)params.get(pwdParam)); + + restReq.credentials(cred); + } + + return hasCreds; + } + /** * Gets values referenced by sequential keys, e.g. {@code key1...keyN}. * From 7457fd319a372d54de68271be7fddbb634cb6070 Mon Sep 17 00:00:00 2001 From: Anton Kalashnikov Date: Tue, 17 Apr 2018 10:30:52 +0300 Subject: [PATCH 0058/1463] IGNITE-8048 Store dynamic indexes to cache data on node join - Fixes #3719. Signed-off-by: Alexey Goncharuk --- .../org/apache/ignite/cache/QueryEntity.java | 175 +++++- .../apache/ignite/cache/QueryEntityPatch.java | 118 ++++ .../cache/CacheJoinNodeDiscoveryData.java | 15 +- .../processors/cache/ClusterCachesInfo.java | 428 ++++++++++---- .../cache/DynamicCacheDescriptor.java | 28 + .../processors/cache/GridCacheProcessor.java | 94 +++- .../cluster/GridClusterStateProcessor.java | 8 +- .../processors/query/QuerySchema.java | 84 ++- .../processors/query/QuerySchemaPatch.java | 96 ++++ ...ActivateDeactivateTestWithPersistence.java | 18 +- .../cache/IgniteDynamicSqlRestoreTest.java | 529 ++++++++++++++++++ ...amicColumnsAbstractConcurrentSelfTest.java | 3 +- .../IgniteCacheQuerySelfTestSuite.java | 2 + 13 files changed, 1470 insertions(+), 128 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/cache/QueryEntityPatch.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchemaPatch.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicSqlRestoreTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java index 0065bae959d73..81fd50b8ee5fa 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java @@ -26,9 +26,11 @@ import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.UUID; import javax.cache.CacheException; import org.apache.ignite.cache.query.annotations.QueryGroupIndex; import org.apache.ignite.cache.query.annotations.QuerySqlField; @@ -36,12 +38,17 @@ import org.apache.ignite.internal.processors.cache.query.QueryEntityClassProperty; import org.apache.ignite.internal.processors.cache.query.QueryEntityTypeDescriptor; import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor; +import org.apache.ignite.internal.processors.query.QueryField; import org.apache.ignite.internal.processors.query.QueryUtils; +import org.apache.ignite.internal.processors.query.schema.operation.SchemaAbstractOperation; +import org.apache.ignite.internal.processors.query.schema.operation.SchemaAlterTableAddColumnOperation; +import org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexCreateOperation; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; /** @@ -143,6 +150,172 @@ public QueryEntity(Class keyCls, Class valCls) { this(convert(processKeyAndValueClasses(keyCls, valCls))); } + /** + * Make query entity patch. This patch can only add properties to entity and can't remove them. + * Other words, the patch will contain only add operations(e.g. add column, create index) and not remove ones. + * + * @param target Query entity to which this entity should be expanded. + * @return Patch which contains operations for expanding this entity. + */ + @NotNull public QueryEntityPatch makePatch(QueryEntity target) { + if (target == null) + return QueryEntityPatch.empty(); + + StringBuilder conflicts = new StringBuilder(); + + checkEquals(conflicts, "keyType", keyType, target.keyType); + checkEquals(conflicts, "valType", valType, target.valType); + checkEquals(conflicts, "keyFieldName", keyFieldName, target.keyFieldName); + checkEquals(conflicts, "valueFieldName", valueFieldName, target.valueFieldName); + checkEquals(conflicts, "tableName", tableName, target.tableName); + + List queryFieldsToAdd = checkFields(target, conflicts); + + Collection indexesToAdd = checkIndexes(target, conflicts); + + if (conflicts.length() != 0) + return QueryEntityPatch.conflict(tableName + " conflict: \n" + conflicts.toString()); + + Collection patchOperations = new ArrayList<>(); + + if (!queryFieldsToAdd.isEmpty()) + patchOperations.add(new SchemaAlterTableAddColumnOperation( + UUID.randomUUID(), + null, + null, + tableName, + queryFieldsToAdd, + true, + true + )); + + if (!indexesToAdd.isEmpty()) { + for (QueryIndex index : indexesToAdd) { + patchOperations.add(new SchemaIndexCreateOperation( + UUID.randomUUID(), + null, + null, + tableName, + index, + true, + 0 + )); + } + } + + return QueryEntityPatch.patch(patchOperations); + } + + /** + * Comparing local fields and target fields. + * + * @param target Query entity for check. + * @param conflicts Storage of conflicts. + * @return Indexes which exist in target and not exist in local. + */ + @NotNull private Collection checkIndexes(QueryEntity target, StringBuilder conflicts) { + HashSet indexesToAdd = new HashSet<>(); + + Map currentIndexes = new HashMap<>(); + + for (QueryIndex index : getIndexes()) { + if (currentIndexes.put(index.getName(), index) != null) + throw new IllegalStateException("Duplicate key"); + } + + for (QueryIndex queryIndex : target.getIndexes()) { + if(currentIndexes.containsKey(queryIndex.getName())) { + checkEquals( + conflicts, + "index " + queryIndex.getName(), + currentIndexes.get(queryIndex.getName()), + queryIndex + ); + } + else + indexesToAdd.add(queryIndex); + } + return indexesToAdd; + } + + /** + * Comparing local entity fields and target entity fields. + * + * @param target Query entity for check. + * @param conflicts Storage of conflicts. + * @return Fields which exist in target and not exist in local. + */ + private List checkFields(QueryEntity target, StringBuilder conflicts) { + List queryFieldsToAdd = new ArrayList<>(); + + for (Map.Entry targetField : target.getFields().entrySet()) { + String targetFieldName = targetField.getKey(); + String targetFieldType = targetField.getValue(); + + if (getFields().containsKey(targetFieldName)) { + checkEquals( + conflicts, + "fieldType of " + targetFieldName, + getFields().get(targetFieldName), + targetFieldType + ); + + checkEquals( + conflicts, + "nullable of " + targetFieldName, + contains(getNotNullFields(), targetFieldName), + contains(target.getNotNullFields(), targetFieldName) + ); + + checkEquals( + conflicts, + "default value of " + targetFieldName, + getFromMap(getDefaultFieldValues(), targetFieldName), + getFromMap(target.getDefaultFieldValues(), targetFieldName) + ); + } + else { + queryFieldsToAdd.add(new QueryField( + targetFieldName, + targetFieldType, + !contains(target.getNotNullFields(),targetFieldName), + getFromMap(target.getDefaultFieldValues(), targetFieldName) + )); + } + } + + return queryFieldsToAdd; + } + + /** + * @param collection Collection for checking. + * @param elementToCheck Element for checking to containing in collection. + * @return {@code true} if collection contain elementToCheck. + */ + private static boolean contains(Collection collection, String elementToCheck) { + return collection != null && collection.contains(elementToCheck); + } + + /** + * @return Value from sourceMap or null if map is null. + */ + private static Object getFromMap(Map sourceMap, String key) { + return sourceMap == null ? null : sourceMap.get(key); + } + + /** + * Comparing two objects and add formatted text to conflicts if needed. + * + * @param conflicts Storage of conflicts resulting error message. + * @param name Name of comparing object. + * @param local Local object. + * @param received Received object. + */ + private void checkEquals(StringBuilder conflicts, String name, Object local, Object received) { + if (!Objects.equals(local, received)) + conflicts.append(String.format("%s is different: local=%s, received=%s\n", name, local, received)); + } + /** * Gets key type for this query pair. * @@ -310,7 +483,7 @@ public QueryEntity setValueFieldName(String valueFieldName) { * * @return Collection of index entities. */ - public Collection getIndexes() { + @NotNull public Collection getIndexes() { return idxs == null ? Collections.emptyList() : idxs; } diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntityPatch.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntityPatch.java new file mode 100644 index 0000000000000..38e1b2acdef28 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntityPatch.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.cache; + +import java.util.Collection; +import java.util.Objects; +import org.apache.ignite.internal.processors.query.schema.operation.SchemaAbstractOperation; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * Query entity patch which contain {@link SchemaAbstractOperation} operations for changing query entity. + * This patch can only add properties to entity and can't remove them. + * Other words, the patch will contain only add operations + * (e.g.: + * {@link org.apache.ignite.internal.processors.query.schema.operation.SchemaAlterTableAddColumnOperation}, + * {@link org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexCreateOperation} + * ) and not remove ones. + * + * It contain only add operation because at the moment we don't have history of schema operations + * and by current state we can't understand some property was already deleted or it has not been added yet. + */ +public class QueryEntityPatch { + /** Empty query entity patch. */ + private static final QueryEntityPatch EMPTY_QUERY_ENTITY_PATCH = new QueryEntityPatch(null, null); + + /** Message which described conflicts during creating this patch. */ + private String conflictsMessage; + + /** Operations for modification query entity. */ + private Collection patchOperations; + + /** + * Create patch. + */ + private QueryEntityPatch(String conflictsMessage, Collection patchOperations) { + this.conflictsMessage = conflictsMessage; + this.patchOperations = patchOperations; + } + + /** + * Builder method for patch with conflicts. + * + * @param conflicts Conflicts. + * @return Query entity patch with conflicts. + */ + public static QueryEntityPatch conflict(String conflicts) { + return new QueryEntityPatch(conflicts, null); + } + + /** + * Builder method for empty patch. + * + * @return Query entity patch. + */ + public static QueryEntityPatch empty() { + return EMPTY_QUERY_ENTITY_PATCH; + } + + /** + * Builder method for patch with operations. + * + * @param patchOperations Operations for modification. + * @return Query entity patch which contain {@link SchemaAbstractOperation} operations for changing query entity. + */ + public static QueryEntityPatch patch(Collection patchOperations) { + return new QueryEntityPatch(null, patchOperations); + } + + /** + * Check for conflict in this patch. + * + * @return {@code true} if patch has conflict. + */ + public boolean hasConflict() { + return conflictsMessage != null; + } + + /** + * @return {@code true} if patch is empty and can't be applying. + */ + public boolean isEmpty() { + return patchOperations == null || patchOperations.isEmpty(); + } + + /** + * @return Conflicts. + */ + public String getConflictsMessage() { + return conflictsMessage; + } + + /** + * @return Patch operations for applying. + */ + public Collection getPatchOperations() { + return patchOperations; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(QueryEntityPatch.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java index 6d2688c948adb..a3902de71b7ac 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheJoinNodeDiscoveryData.java @@ -112,17 +112,23 @@ public static class CacheInfo implements Serializable { /** Flags added for future usage. */ private final long flags; + /** Statically configured flag */ + private final boolean staticallyConfigured; + /** * @param cacheData Cache data. * @param cacheType Cache type. * @param sql SQL flag - {@code true} if cache was created with {@code CREATE TABLE}. * @param flags Flags (for future usage). + * @param staticallyConfigured {@code true} if it was configured by static config and {@code false} otherwise. */ - public CacheInfo(StoredCacheData cacheData, CacheType cacheType, boolean sql, long flags) { + public CacheInfo(StoredCacheData cacheData, CacheType cacheType, boolean sql, long flags, + boolean staticallyConfigured) { this.cacheData = cacheData; this.cacheType = cacheType; this.sql = sql; this.flags = flags; + this.staticallyConfigured = staticallyConfigured; } /** @@ -146,6 +152,13 @@ public boolean sql() { return sql; } + /** + * @return {@code true} if it was configured by static config and {@code false} otherwise. + */ + public boolean isStaticallyConfigured() { + return staticallyConfigured; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(CacheInfo.class, this); 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 2b2fb559c182e..975617ee8dc0a 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 @@ -35,31 +35,35 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.CacheExistsException; +import org.apache.ignite.cache.QueryEntity; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.GridCachePluginContext; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.managers.discovery.DiscoCache; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; 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.processors.query.QuerySchema; +import org.apache.ignite.internal.processors.query.QuerySchemaPatch; import org.apache.ignite.internal.processors.query.QueryUtils; import org.apache.ignite.internal.processors.query.schema.SchemaOperationException; import org.apache.ignite.internal.util.GridConcurrentHashSet; +import org.apache.ignite.internal.util.lang.GridFunc; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.CachePluginContext; import org.apache.ignite.plugin.CachePluginProvider; import org.apache.ignite.plugin.PluginProvider; import org.apache.ignite.spi.discovery.DiscoveryDataBag; -import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.cache.CacheMode.LOCAL; @@ -71,6 +75,9 @@ * Logic related to cache discovery data processing. */ class ClusterCachesInfo { + /** Version since which merge of config is supports. */ + private static final IgniteProductVersion V_MERGE_CONFIG_SINCE = IgniteProductVersion.fromString("2.5.0"); + /** */ private final GridKernalContext ctx; @@ -987,54 +994,77 @@ public void onGridDataReceived(DiscoveryDataBag.GridDiscoveryData data) { // CacheGroup configurations that were created from local node configuration. Map locCacheGrps = new HashMap<>(registeredCacheGroups()); - // Replace locally registered data with actual data received from cluster. - registeredCaches.clear(); - registeredCacheGrps.clear(); - ctx.discovery().cleanCachesAndGroups(); + //Replace locally registered data with actual data received from cluster. + cleanCachesAndGroups(); - for (CacheGroupData grpData : cachesData.cacheGroups().values()) { - CacheGroupDescriptor grpDesc = new CacheGroupDescriptor( - grpData.config(), - grpData.groupName(), - grpData.groupId(), - grpData.receivedFrom(), - grpData.startTopologyVersion(), - grpData.deploymentId(), - grpData.caches(), - grpData.persistenceEnabled(), - grpData.walEnabled(), - grpData.walChangeRequests()); + registerReceivedCacheGroups(cachesData, locCacheGrps); - if (locCacheGrps.containsKey(grpDesc.groupId())) { - CacheGroupDescriptor locGrpCfg = locCacheGrps.get(grpDesc.groupId()); + registerReceivedCacheTemplates(cachesData); - grpDesc.mergeWith(locGrpCfg); - } + registerReceivedCaches(cachesData); - CacheGroupDescriptor old = registeredCacheGrps.put(grpDesc.groupId(), grpDesc); + addReceivedClientNodesToDiscovery(cachesData); - assert old == null : old; + String conflictErr = validateRegisteredCaches(); - ctx.discovery().addCacheGroup(grpDesc, - grpData.config().getNodeFilter(), - grpData.config().getCacheMode()); + gridData = new GridData(joinDiscoData, cachesData, conflictErr); + + if (cachesOnDisconnect == null || cachesOnDisconnect.clusterActive()) + initStartCachesForLocalJoin(false, disconnectedState()); + } + + /** + * Validation {@link #registeredCaches} on conflicts. + * + * @return Error message if conflicts was found. + */ + @Nullable private String validateRegisteredCaches() { + String conflictErr = null; + + if (joinDiscoData != null) { + for (Map.Entry e : joinDiscoData.caches().entrySet()) { + if (!registeredCaches.containsKey(e.getKey())) { + conflictErr = checkCacheConflict(e.getValue().cacheData().config()); + + if (conflictErr != null) { + conflictErr = "Failed to start configured cache due to conflict with started caches. " + + conflictErr; + + break; + } + } + } } - for (CacheData cacheData : cachesData.templates().values()) { - DynamicCacheDescriptor desc = new DynamicCacheDescriptor( - ctx, - cacheData.cacheConfiguration(), - cacheData.cacheType(), - null, - true, - cacheData.receivedFrom(), - cacheData.staticallyConfigured(), - false, - cacheData.deploymentId(), - cacheData.schema()); + return conflictErr; + } - registeredTemplates.put(cacheData.cacheConfiguration().getName(), desc); + /** + * Adding received client nodes to discovery if needed. + * + * @param cachesData Data received from cluster. + */ + private void addReceivedClientNodesToDiscovery(CacheNodeCommonDiscoveryData cachesData) { + if (!F.isEmpty(cachesData.clientNodesMap())) { + for (Map.Entry> entry : cachesData.clientNodesMap().entrySet()) { + String cacheName = entry.getKey(); + + for (Map.Entry tup : entry.getValue().entrySet()) + ctx.discovery().addClientNode(cacheName, tup.getKey(), tup.getValue()); + } } + } + + /** + * Register caches received from cluster. + * + * @param cachesData Data received from cluster. + */ + private void registerReceivedCaches(CacheNodeCommonDiscoveryData cachesData) { + Map patchesToApply = new HashMap<>(); + Collection cachesToSave = new HashSet<>(); + + boolean hasSchemaPatchConflict = false; for (CacheData cacheData : cachesData.caches().values()) { CacheGroupDescriptor grpDesc = registeredCacheGrps.get(cacheData.groupId()); @@ -1053,7 +1083,22 @@ public void onGridDataReceived(DiscoveryDataBag.GridDiscoveryData data) { cacheData.staticallyConfigured(), cacheData.sql(), cacheData.deploymentId(), - cacheData.schema()); + new QuerySchema(cacheData.schema().entities()) + ); + + Collection localQueryEntities = getLocalQueryEntities(cfg.getName()); + + QuerySchemaPatch schemaPatch = desc.makeSchemaPatch(localQueryEntities); + + if (schemaPatch.hasConflicts()) { + hasSchemaPatchConflict = true; + + log.warning("Skipping apply patch because conflicts : " + schemaPatch.getConflictsMessage()); + } + else if (!schemaPatch.isEmpty()) + patchesToApply.put(desc, schemaPatch); + else if (!GridFunc.eqNotOrdered(desc.schema().entities(), localQueryEntities)) + cachesToSave.add(desc); //received config is different of local config - need to resave desc.receivedOnDiscovery(true); @@ -1066,36 +1111,140 @@ public void onGridDataReceived(DiscoveryDataBag.GridDiscoveryData data) { cfg.getNearConfiguration() != null); } - if (!F.isEmpty(cachesData.clientNodesMap())) { - for (Map.Entry> entry : cachesData.clientNodesMap().entrySet()) { - String cacheName = entry.getKey(); + updateRegisteredCachesIfNeeded(patchesToApply, cachesToSave, hasSchemaPatchConflict); + } - for (Map.Entry tup : entry.getValue().entrySet()) - ctx.discovery().addClientNode(cacheName, tup.getKey(), tup.getValue()); + /** + * Merging config or resaving it if it needed. + * + * @param patchesToApply Patches which need to apply. + * @param cachesToSave Caches which need to resave. + * @param hasSchemaPatchConflict {@code true} if we have conflict during making patch. + */ + private void updateRegisteredCachesIfNeeded(Map patchesToApply, + Collection cachesToSave, boolean hasSchemaPatchConflict) { + //Skip merge of config if least one conflict was found. + if (!hasSchemaPatchConflict && isMergeConfigSupports(ctx.discovery().localNode())) { + boolean isClusterActive = ctx.state().clusterState().active(); + + //Merge of config for cluster only for inactive grid. + if (!isClusterActive && !patchesToApply.isEmpty()) { + for (Map.Entry entry : patchesToApply.entrySet()) { + if (entry.getKey().applySchemaPatch(entry.getValue())) + saveCacheConfiguration(entry.getKey()); + } + + for (DynamicCacheDescriptor descriptor : cachesToSave) { + saveCacheConfiguration(descriptor); + } + } + else if (patchesToApply.isEmpty()) { + for (DynamicCacheDescriptor descriptor : cachesToSave) { + saveCacheConfiguration(descriptor); + } } } + } - String conflictErr = null; + /** + * Register cache templates received from cluster. + * + * @param cachesData Data received from cluster. + */ + private void registerReceivedCacheTemplates(CacheNodeCommonDiscoveryData cachesData) { + for (CacheData cacheData : cachesData.templates().values()) { + DynamicCacheDescriptor desc = new DynamicCacheDescriptor( + ctx, + cacheData.cacheConfiguration(), + cacheData.cacheType(), + null, + true, + cacheData.receivedFrom(), + cacheData.staticallyConfigured(), + false, + cacheData.deploymentId(), + cacheData.schema()); - if (joinDiscoData != null) { - for (Map.Entry e : joinDiscoData.caches().entrySet()) { - if (!registeredCaches.containsKey(e.getKey())) { - conflictErr = checkCacheConflict(e.getValue().cacheData().config()); + registeredTemplates.put(cacheData.cacheConfiguration().getName(), desc); + } + } - if (conflictErr != null) { - conflictErr = "Failed to start configured cache due to conflict with started caches. " + - conflictErr; + /** + * Register cache groups received from cluster. + * + * @param cachesData Data received from cluster. + * @param locCacheGrps Current local cache groups. + */ + private void registerReceivedCacheGroups(CacheNodeCommonDiscoveryData cachesData, + Map locCacheGrps) { + for (CacheGroupData grpData : cachesData.cacheGroups().values()) { + CacheGroupDescriptor grpDesc = new CacheGroupDescriptor( + grpData.config(), + grpData.groupName(), + grpData.groupId(), + grpData.receivedFrom(), + grpData.startTopologyVersion(), + grpData.deploymentId(), + grpData.caches(), + grpData.persistenceEnabled(), + grpData.walEnabled(), + grpData.walChangeRequests()); - break; - } - } + if (locCacheGrps.containsKey(grpDesc.groupId())) { + CacheGroupDescriptor locGrpCfg = locCacheGrps.get(grpDesc.groupId()); + + grpDesc.mergeWith(locGrpCfg); } + + CacheGroupDescriptor old = registeredCacheGrps.put(grpDesc.groupId(), grpDesc); + + assert old == null : old; + + ctx.discovery().addCacheGroup(grpDesc, + grpData.config().getNodeFilter(), + grpData.config().getCacheMode()); } + } - gridData = new GridData(joinDiscoData, cachesData, conflictErr); + /** + * Clean local registered caches and groups + */ + private void cleanCachesAndGroups() { + registeredCaches.clear(); + registeredCacheGrps.clear(); + ctx.discovery().cleanCachesAndGroups(); + } - if (cachesOnDisconnect == null || cachesOnDisconnect.clusterActive()) - initStartCachesForLocalJoin(false, disconnectedState()); + /** + * Save dynamic cache descriptor on disk. + * + * @param desc Cache to save. + */ + private void saveCacheConfiguration(DynamicCacheDescriptor desc) { + try { + ctx.cache().saveCacheConfiguration(desc); + } + catch (IgniteCheckedException e) { + log.error("Error while saving cache configuration to disk, cfg = " + desc.cacheConfiguration(), e); + } + } + + /** + * Get started node query entities by cacheName. + * + * @param cacheName Cache for which query entities will be returned. + * @return Local query entities. + */ + private Collection getLocalQueryEntities(String cacheName) { + if (joinDiscoData == null) + return Collections.emptyList(); + + CacheJoinNodeDiscoveryData.CacheInfo cacheInfo = joinDiscoData.caches().get(cacheName); + + if (cacheInfo == null) + return Collections.emptyList(); + + return cacheInfo.cacheData().queryEntities(); } /** @@ -1144,7 +1293,7 @@ private void initStartCachesForLocalJoin(boolean firstNode, boolean reconnect) { desc.staticallyConfigured(), desc.sql(), desc.deploymentId(), - new QuerySchema(locCfg.cacheData().queryEntities())); + desc.schema().copy()); desc0.startTopologyVersion(desc.startTopologyVersion()); desc0.receivedFromStartVersion(desc.receivedFromStartVersion()); @@ -1385,26 +1534,14 @@ private String checkCacheConflict(CacheConfiguration cfg) { * @return Configuration conflict error. */ private String processJoiningNode(CacheJoinNodeDiscoveryData joinData, UUID nodeId, boolean locJoin) { - for (CacheJoinNodeDiscoveryData.CacheInfo cacheInfo : joinData.templates().values()) { - CacheConfiguration cfg = cacheInfo.cacheData().config(); + registerNewCacheTemplates(joinData, nodeId); - if (!registeredTemplates.containsKey(cfg.getName())) { - DynamicCacheDescriptor desc = new DynamicCacheDescriptor(ctx, - cfg, - cacheInfo.cacheType(), - null, - true, - nodeId, - true, - false, - joinData.cacheDeploymentId(), - new QuerySchema(cacheInfo.cacheData().queryEntities())); + Map patchesToApply = new HashMap<>(); - DynamicCacheDescriptor old = registeredTemplates.put(cfg.getName(), desc); + boolean hasSchemaPatchConflict = false; + boolean active = ctx.state().clusterState().active(); - assert old == null : old; - } - } + boolean isMergeConfigSupport = isMergeConfigSupports(null); for (CacheJoinNodeDiscoveryData.CacheInfo cacheInfo : joinData.caches().values()) { CacheConfiguration cfg = cacheInfo.cacheData().config(); @@ -1421,49 +1558,138 @@ private String processJoiningNode(CacheJoinNodeDiscoveryData joinData, UUID node continue; } - int cacheId = CU.cacheId(cfg.getName()); + registerNewCache(joinData, nodeId, cacheInfo); + } + else if (!active && isMergeConfigSupport) { + DynamicCacheDescriptor desc = registeredCaches.get(cfg.getName()); + + QuerySchemaPatch schemaPatch = desc.makeSchemaPatch(cacheInfo.cacheData().queryEntities()); - CacheGroupDescriptor grpDesc = registerCacheGroup(null, - null, - cfg, - cacheId, + if (schemaPatch.hasConflicts()) { + hasSchemaPatchConflict = true; + + log.error("Error during making schema patch : " + schemaPatch.getConflictsMessage()); + } + else if (!schemaPatch.isEmpty() && !hasSchemaPatchConflict) + patchesToApply.put(desc, schemaPatch); + } + + ctx.discovery().addClientNode(cfg.getName(), nodeId, cfg.getNearConfiguration() != null); + } + + //If conflict was detected we don't merge config and we leave existed config. + if (!hasSchemaPatchConflict && !patchesToApply.isEmpty()) + for(Map.Entry entry: patchesToApply.entrySet()){ + if (entry.getKey().applySchemaPatch(entry.getValue())) + saveCacheConfiguration(entry.getKey()); + } + + if (joinData.startCaches()) { + for (DynamicCacheDescriptor desc : registeredCaches.values()) { + ctx.discovery().addClientNode(desc.cacheName(), nodeId, - joinData.cacheDeploymentId()); + desc.cacheConfiguration().getNearConfiguration() != null); + } + } + + return null; + } + + /** + * Register new cache received from joining node. + * + * @param joinData Data from joining node. + * @param nodeId Joining node id. + * @param cacheInfo Cache info of new node. + */ + private void registerNewCache( + CacheJoinNodeDiscoveryData joinData, + UUID nodeId, + CacheJoinNodeDiscoveryData.CacheInfo cacheInfo) { + CacheConfiguration cfg = cacheInfo.cacheData().config(); + + int cacheId = CU.cacheId(cfg.getName()); + + CacheGroupDescriptor grpDesc = registerCacheGroup(null, + null, + cfg, + cacheId, + nodeId, + joinData.cacheDeploymentId()); + + ctx.discovery().setCacheFilter( + cacheId, + grpDesc.groupId(), + cfg.getName(), + cfg.getNearConfiguration() != null); + + DynamicCacheDescriptor desc = new DynamicCacheDescriptor(ctx, + cfg, + cacheInfo.cacheType(), + grpDesc, + false, + nodeId, + cacheInfo.isStaticallyConfigured(), + cacheInfo.sql(), + joinData.cacheDeploymentId(), + new QuerySchema(cacheInfo.cacheData().queryEntities())); + + DynamicCacheDescriptor old = registeredCaches.put(cfg.getName(), desc); - ctx.discovery().setCacheFilter( - cacheId, - grpDesc.groupId(), - cfg.getName(), - cfg.getNearConfiguration() != null); + assert old == null : old; + } + + /** + * Register new cache templates received from joining node. + * + * @param joinData Data from joining node. + * @param nodeId Joining node id. + */ + private void registerNewCacheTemplates(CacheJoinNodeDiscoveryData joinData, UUID nodeId) { + for (CacheJoinNodeDiscoveryData.CacheInfo cacheInfo : joinData.templates().values()) { + CacheConfiguration cfg = cacheInfo.cacheData().config(); + if (!registeredTemplates.containsKey(cfg.getName())) { DynamicCacheDescriptor desc = new DynamicCacheDescriptor(ctx, cfg, cacheInfo.cacheType(), - grpDesc, - false, + null, + true, nodeId, true, - cacheInfo.sql(), + false, joinData.cacheDeploymentId(), new QuerySchema(cacheInfo.cacheData().queryEntities())); - DynamicCacheDescriptor old = registeredCaches.put(cfg.getName(), desc); + DynamicCacheDescriptor old = registeredTemplates.put(cfg.getName(), desc); assert old == null : old; } - - ctx.discovery().addClientNode(cfg.getName(), nodeId, cfg.getNearConfiguration() != null); } + } - if (joinData.startCaches()) { - for (DynamicCacheDescriptor desc : registeredCaches.values()) { - ctx.discovery().addClientNode(desc.cacheName(), - nodeId, - desc.cacheConfiguration().getNearConfiguration() != null); - } + /** + * @return {@code true} if grid supports merge of config and {@code False} otherwise. + */ + public boolean isMergeConfigSupports(ClusterNode joiningNode) { + DiscoCache discoCache = ctx.discovery().discoCache(); + + if (discoCache == null) + return true; + + if (joiningNode != null && joiningNode.version().compareToIgnoreTimestamp(V_MERGE_CONFIG_SINCE) < 0) + return false; + + Collection nodes = discoCache.allNodes(); + + for (ClusterNode node : nodes) { + IgniteProductVersion version = node.version(); + + if (version.compareToIgnoreTimestamp(V_MERGE_CONFIG_SINCE) < 0) + return false; } - return null; + return true; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java index cad84144b519f..93882a253bf65 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java @@ -17,14 +17,17 @@ package org.apache.ignite.internal.processors.cache; +import java.util.Collection; import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.QueryEntity; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor; import org.apache.ignite.internal.processors.query.QuerySchema; +import org.apache.ignite.internal.processors.query.QuerySchemaPatch; import org.apache.ignite.internal.processors.query.schema.message.SchemaFinishDiscoveryMessage; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.internal.CU; @@ -345,6 +348,31 @@ public void schemaChangeFinish(SchemaFinishDiscoveryMessage msg) { } } + /** + * Make schema patch for this cache. + * + * @param target Query entity list which current schema should be expanded to. + * @return Patch which contains operations for expanding schema of this cache. + * @see QuerySchemaPatch + */ + public QuerySchemaPatch makeSchemaPatch(Collection target) { + synchronized (schemaMux) { + return schema.makePatch(target); + } + } + + /** + * Apply query schema patch for changing current schema. + * + * @param patch patch to apply. + * @return {@code true} if applying was success and {@code false} otherwise. + */ + public boolean applySchemaPatch(QuerySchemaPatch patch) { + synchronized (schemaMux) { + return schema.applyPatch(patch); + } + } + /** * Form a {@link StoredCacheData} with all data to correctly restore cache params when its configuration is read * from page store. Essentially, this method takes from {@link DynamicCacheDescriptor} all that's needed to start 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 3aa6603a47d96..36edd72dd952f 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 @@ -117,6 +117,7 @@ import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor; import org.apache.ignite.internal.processors.plugin.CachePluginManager; import org.apache.ignite.internal.processors.query.QuerySchema; +import org.apache.ignite.internal.processors.query.QuerySchemaPatch; import org.apache.ignite.internal.processors.query.QueryUtils; import org.apache.ignite.internal.processors.query.schema.SchemaExchangeWorkerTask; import org.apache.ignite.internal.processors.query.schema.SchemaNodeLeaveExchangeWorkerTask; @@ -183,6 +184,14 @@ */ @SuppressWarnings({"unchecked", "TypeMayBeWeakened", "deprecation"}) public class GridCacheProcessor extends GridProcessorAdapter { + /** Template of message of conflicts during configuration merge*/ + private static final String MERGE_OF_CONFIG_CONFLICTS_MESSAGE = + "Conflicts during configuration merge for cache '%s' : \n%s"; + + /** Template of message of node join was fail because it requires to merge of config */ + private static final String MERGE_OF_CONFIG_REQUIRED_MESSAGE = "Failed to join node to the active cluster " + + "(the config of the cache '%s' has to be merged which is impossible on active grid). " + + "Deactivate grid and retry node join or clean the joining node."; /** */ private final boolean startClientCaches = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_START_CACHES_ON_JOIN, false); @@ -742,15 +751,29 @@ private void addCacheOnJoin(CacheConfiguration cfg, boolean sql, if (cacheType != CacheType.USER && cfg.getDataRegionName() == null) cfg.setDataRegionName(sharedCtx.database().systemDateRegionName()); - if (!cacheType.userCache()) - stopSeq.addLast(cacheName); - else - stopSeq.addFirst(cacheName); - - caches.put(cacheName, new CacheJoinNodeDiscoveryData.CacheInfo(cacheData, cacheType, cacheData.sql(), 0)); + addStoredCache(caches, cacheData, cacheName, cacheType, true); } else - templates.put(cacheName, new CacheJoinNodeDiscoveryData.CacheInfo(cacheData, CacheType.USER, false, 0)); + templates.put(cacheName, new CacheInfo(cacheData, CacheType.USER, false, 0, true)); + } + + /** + * Add stored cache data to caches storage. + * + * @param caches Cache storage. + * @param cacheData Cache data to add. + * @param cacheName Cache name. + * @param cacheType Cache type. + * @param isStaticalyConfigured Statically configured flag. + */ + private void addStoredCache(Map caches, StoredCacheData cacheData, String cacheName, + CacheType cacheType, boolean isStaticalyConfigured) { + if (!cacheType.userCache()) + stopSeq.addLast(cacheName); + else + stopSeq.addFirst(cacheName); + + caches.put(cacheName, new CacheInfo(cacheData, cacheType, cacheData.sql(), 0, isStaticalyConfigured)); } /** @@ -774,6 +797,19 @@ private void addCacheOnJoinFromConfig( addCacheOnJoin(cfg, false, caches, templates); } + + if (CU.isPersistenceEnabled(ctx.config()) && ctx.cache().context().pageStore() != null) { + Map storedCaches = ctx.cache().context().pageStore().readCacheConfigurations(); + + if (!F.isEmpty(storedCaches)) + for (StoredCacheData storedCacheData : storedCaches.values()) { + String cacheName = storedCacheData.config().getName(); + + //Ignore stored caches if it already added by static config(static config has higher priority). + if (!caches.containsKey(cacheName)) + addStoredCache(caches, storedCacheData, cacheName, cacheType(cacheName), false); + } + } } /** @@ -2439,6 +2475,50 @@ private GridCacheSharedContext createSharedContext(GridKernalContext kernalCtx, sharedCtx.walState().onCachesInfoCollected(); } + /** {@inheritDoc} */ + @Nullable @Override public IgniteNodeValidationResult validateNode( + ClusterNode node, JoiningNodeDiscoveryData discoData + ) { + if(!cachesInfo.isMergeConfigSupports(node)) + return null; + + if (discoData.hasJoiningNodeData() && discoData.joiningNodeData() instanceof CacheJoinNodeDiscoveryData) { + CacheJoinNodeDiscoveryData nodeData = (CacheJoinNodeDiscoveryData)discoData.joiningNodeData(); + + boolean isGridActive = ctx.state().clusterState().active(); + + StringBuilder errorMessage = new StringBuilder(); + + for (CacheJoinNodeDiscoveryData.CacheInfo cacheInfo : nodeData.caches().values()) { + DynamicCacheDescriptor localDesc = cacheDescriptor(cacheInfo.cacheData().config().getName()); + + if (localDesc == null) + continue; + + QuerySchemaPatch schemaPatch = localDesc.makeSchemaPatch(cacheInfo.cacheData().queryEntities()); + + if (schemaPatch.hasConflicts() || (isGridActive && !schemaPatch.isEmpty())) { + if (errorMessage.length() > 0) + errorMessage.append("\n"); + + if (schemaPatch.hasConflicts()) + errorMessage.append(String.format(MERGE_OF_CONFIG_CONFLICTS_MESSAGE, + localDesc.cacheName(), schemaPatch.getConflictsMessage())); + else + errorMessage.append(String.format(MERGE_OF_CONFIG_REQUIRED_MESSAGE, localDesc.cacheName())); + } + } + + if (errorMessage.length() > 0) { + String msg = errorMessage.toString(); + + return new IgniteNodeValidationResult(node.id(), msg, msg); + } + } + + return null; + } + /** * @param msg Message. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java index 81a5b4e5c2f3c..2700a20f46f20 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java @@ -1035,7 +1035,13 @@ private boolean sendComputeCheckGlobalState() { ", client=" + ctx.clientNode() + ", daemon" + ctx.isDaemon() + "]"); } - IgniteCompute comp = ((ClusterGroupAdapter)ctx.cluster().get().forServers()).compute(); + + ClusterGroupAdapter clusterGroupAdapter = (ClusterGroupAdapter)ctx.cluster().get().forServers(); + + if (F.isEmpty(clusterGroupAdapter.nodes())) + return false; + + IgniteCompute comp = clusterGroupAdapter.compute(); return comp.call(new IgniteCallable() { @IgniteInstanceResource diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java index 5cbae29d01a6d..569a02e0b2cbc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java @@ -20,12 +20,14 @@ import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.HashSet; import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.Set; - import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.cache.QueryEntityPatch; import org.apache.ignite.cache.QueryIndex; import org.apache.ignite.internal.processors.query.schema.message.SchemaFinishDiscoveryMessage; import org.apache.ignite.internal.processors.query.schema.operation.SchemaAbstractOperation; @@ -84,15 +86,91 @@ public QuerySchema copy() { } } + /** + * Make query schema patch. + * + * @param target Query entity list to which current schema should be expanded. + * @return Patch to achieve entity which is a result of merging current one and target. + * @see QuerySchemaPatch + */ + public QuerySchemaPatch makePatch(Collection target) { + synchronized (mux) { + Map localEntities = new HashMap<>(); + + for (QueryEntity entity : entities) { + if (localEntities.put(entity.getTableName(), entity) != null) + throw new IllegalStateException("Duplicate key"); + } + + Collection patchOperations = new ArrayList<>(); + Collection entityToAdd = new ArrayList<>(); + + StringBuilder conflicts = new StringBuilder(); + + for (QueryEntity queryEntity : target) { + if (localEntities.containsKey(queryEntity.getTableName())) { + QueryEntity localEntity = localEntities.get(queryEntity.getTableName()); + + QueryEntityPatch entityPatch = localEntity.makePatch(queryEntity); + + if (entityPatch.hasConflict()) { + if (conflicts.length() > 0) + conflicts.append("\n"); + + conflicts.append(entityPatch.getConflictsMessage()); + } + + if (!entityPatch.isEmpty()) + patchOperations.addAll(entityPatch.getPatchOperations()); + } + else + entityToAdd.add(QueryUtils.copy(queryEntity)); + } + + return new QuerySchemaPatch(patchOperations, entityToAdd, conflicts.toString()); + } + } + + /** + * Apply query schema patch for changing this schema. + * + * @param patch Patch to apply. + * @return {@code true} if applying was success and {@code false} otherwise. + */ + public boolean applyPatch(QuerySchemaPatch patch) { + synchronized (mux) { + if (patch.hasConflicts()) + return false; + + if (patch.isEmpty()) + return true; + + for (SchemaAbstractOperation operation : patch.getPatchOperations()) { + finish(operation); + } + + entities.addAll(patch.getEntityToAdd()); + + return true; + } + } + /** * Process finish message. * * @param msg Message. */ public void finish(SchemaFinishDiscoveryMessage msg) { - synchronized (mux) { - SchemaAbstractOperation op = msg.operation(); + finish(msg.operation()); + } + /** + * Process operation. + * + * @param op Operation for handle. + */ + public void finish(SchemaAbstractOperation op) { + synchronized (mux) { if (op instanceof SchemaIndexCreateOperation) { SchemaIndexCreateOperation op0 = (SchemaIndexCreateOperation)op; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchemaPatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchemaPatch.java new file mode 100644 index 0000000000000..68beb049da9ee --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchemaPatch.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.query; + +import java.util.Collection; +import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.cache.QueryEntityPatch; +import org.apache.ignite.internal.processors.query.schema.operation.SchemaAbstractOperation; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.NotNull; + +/** + * Query schema patch which contains {@link SchemaAbstractOperation} operations for changing query entities. + * This patch is high level path on {@link org.apache.ignite.cache.QueryEntityPatch} but + * it has operations for all {@link QueryEntity} in schema + * and also contains {@link QueryEntity} for adding to schema by whole. + * + * @see org.apache.ignite.cache.QueryEntityPatch + */ +public class QuerySchemaPatch { + /** Message which described conflicts during creating this patch. */ + private String conflictsMessage; + + /** Operations for modification query entity. */ + private Collection patchOperations; + + /** Entities which should be added by whole. */ + private Collection entityToAdd; + + /** + * Create patch. + */ + public QuerySchemaPatch( + @NotNull Collection patchOperations, + @NotNull Collection entityToAdd, + String conflictsMessage) { + this.patchOperations = patchOperations; + this.entityToAdd = entityToAdd; + this.conflictsMessage = conflictsMessage; + } + + /** + * @return {@code true} if patch has conflict. + */ + public boolean hasConflicts() { + return conflictsMessage != null && !conflictsMessage.isEmpty(); + } + + /** + * @return Conflicts message. + */ + public String getConflictsMessage() { + return conflictsMessage; + } + + /** + * @return {@code true} if patch is empty and can't be applying. + */ + public boolean isEmpty() { + return patchOperations.isEmpty() && entityToAdd.isEmpty(); + } + + /** + * @return Patch operations for applying. + */ + @NotNull public Collection getPatchOperations() { + return patchOperations; + } + + /** + * @return Entities which should be added by whole. + */ + @NotNull public Collection getEntityToAdd() { + return entityToAdd; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(QuerySchemaPatch.class, this); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java index 58511ee306fdd..8bae136107ca0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java @@ -23,12 +23,13 @@ import java.util.concurrent.CountDownLatch; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; 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.testframework.GridTestUtils; @@ -253,22 +254,15 @@ public void testActivateCacheRestoreConfigurationConflict() throws Exception { ccfg.setGroupName(DEFAULT_CACHE_NAME); - ccfgs = new CacheConfiguration[]{ccfg}; - - startGrids(SRVS); + ccfgs = new CacheConfiguration[] {ccfg}; try { - ignite(0).active(true); + startGrids(SRVS); fail(); } - catch (IgniteException e) { - // Expected error. + catch (IgniteCheckedException e) { + assertTrue(X.getCause(e).getMessage().contains("Failed to start configured cache.")); } - - for (int i = 0; i < SRVS; i++) - assertFalse(ignite(i).active()); - - checkNoCaches(SRVS); } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicSqlRestoreTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicSqlRestoreTest.java new file mode 100644 index 0000000000000..f7dc7b41ba6de --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicSqlRestoreTest.java @@ -0,0 +1,529 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedHashMap; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.cache.query.SqlFieldsQuery; +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.util.typedef.X; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.NotNull; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.MatcherAssert.assertThat; + +/** + * + */ +public class IgniteDynamicSqlRestoreTest extends GridCommonAbstractTest implements Serializable { + + public static final String TEST_CACHE_NAME = "test"; + public static final String TEST_INDEX_OBJECT = "TestIndexObject"; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setAutoActivationEnabled(false); + + DataStorageConfiguration memCfg = new DataStorageConfiguration() + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration().setMaxSize(200 * 1024 * 1024).setPersistenceEnabled(true)) + .setWalMode(WALMode.LOG_ONLY); + + cfg.setDataStorageConfiguration(memCfg); + + cfg.setConsistentId(gridName); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** + * @throws Exception if failed. + */ + public void testMergeChangedConfigOnCoordinator() throws Exception { + { + //given: two started nodes with test table + Ignite ig = startGrid(0); + startGrid(1); + + ig.cluster().active(true); + + IgniteCache cache = ig.getOrCreateCache(getTestTableConfiguration()); + + fillTestData(ig); + + //when: stop one node and create indexes on other node + stopGrid(1); + + cache.query(new SqlFieldsQuery("create index myindexa on TestIndexObject(a)")).getAll(); + cache.query(new SqlFieldsQuery("create index myindexb on TestIndexObject(b)")).getAll(); + cache.query(new SqlFieldsQuery("alter table TestIndexObject add column (c int)")).getAll(); + + //and: stop all grid + stopAllGrids(); + } + + { + //and: start cluster from node without index + IgniteEx ig = startGrid(1); + startGrid(0); + + ig.cluster().active(true); + + //and: change data + try (IgniteDataStreamer s = ig.dataStreamer(TEST_CACHE_NAME)) { + s.allowOverwrite(true); + for (int i = 0; i < 5_000; i++) + s.addData(i, null); + } + + stopAllGrids(); + } + + { + //when: start node from first node + IgniteEx ig0 = startGrid(0); + IgniteEx ig1 = startGrid(1); + + ig0.cluster().active(true); + + //then: everything is ok + try (IgniteDataStreamer s = ig1.dataStreamer(TEST_CACHE_NAME)) { + s.allowOverwrite(true); + for (int i = 0; i < 50_000; i++) { + BinaryObject bo = ig1.binary().builder(TEST_INDEX_OBJECT) + .setField("a", i, Object.class) + .setField("b", String.valueOf(i), Object.class) + .setField("c", i, Object.class) + .build(); + + s.addData(i, bo); + } + } + + IgniteCache cache = ig1.cache(TEST_CACHE_NAME); + + assertThat(doExplainPlan(cache, "explain select * from TestIndexObject where a > 5"), containsString("myindexa")); + assertFalse(cache.query(new SqlFieldsQuery("SELECT a,b,c FROM TestIndexObject limit 1")).getAll().isEmpty()); + } + } + + /** + * @throws Exception if failed. + */ + public void testTakeConfigFromJoiningNodeOnInactiveGrid() throws Exception { + { + //given: two started nodes with test table + Ignite ig = startGrid(0); + startGrid(1); + + ig.cluster().active(true); + + IgniteCache cache = ig.getOrCreateCache(getTestTableConfiguration()); + + fillTestData(ig); + + stopGrid(1); + + cache.query(new SqlFieldsQuery("create index myindexa on TestIndexObject(a)")).getAll(); + cache.query(new SqlFieldsQuery("create index myindexb on TestIndexObject(b)")).getAll(); + cache.query(new SqlFieldsQuery("alter table TestIndexObject add column (c int)")).getAll(); + + stopAllGrids(); + } + + { + //and: start cluster from node without cache + IgniteEx ig = startGrid(1); + startGrid(0); + + ig.cluster().active(true); + + //then: config for cache was applying successful + IgniteCache cache = ig.cache(TEST_CACHE_NAME); + + assertThat(doExplainPlan(cache, "explain select * from TestIndexObject where a > 5"), containsString("myindexa")); + assertFalse(cache.query(new SqlFieldsQuery("SELECT a,b,c FROM TestIndexObject limit 1")).getAll().isEmpty()); + } + } + + /** + * @throws Exception if failed. + */ + public void testResaveConfigAfterMerge() throws Exception { + { + //given: two started nodes with test table + Ignite ig = startGrid(0); + startGrid(1); + + ig.cluster().active(true); + + IgniteCache cache = ig.getOrCreateCache(getTestTableConfiguration()); + + fillTestData(ig); + + stopGrid(1); + + cache.query(new SqlFieldsQuery("create index myindexa on TestIndexObject(a)")).getAll(); + cache.query(new SqlFieldsQuery("create index myindexb on TestIndexObject(b)")).getAll(); + cache.query(new SqlFieldsQuery("alter table TestIndexObject add column (c int)")).getAll(); + + stopAllGrids(); + } + + { + //when: start cluster from node without cache + IgniteEx ig = startGrid(1); + startGrid(0); + + ig.cluster().active(true); + + stopAllGrids(); + } + + { + //then: start only one node which originally was without index + IgniteEx ig = startGrid(1); + + ig.cluster().active(true); + + IgniteCache cache = ig.cache(TEST_CACHE_NAME); + + assertThat(doExplainPlan(cache, "explain select * from TestIndexObject where a > 5"), containsString("myindexa")); + assertFalse(cache.query(new SqlFieldsQuery("SELECT a,b,c FROM TestIndexObject limit 1")).getAll().isEmpty()); + } + } + + /** + * @throws Exception if failed. + */ + public void testMergeChangedConfigOnInactiveGrid() throws Exception { + { + //given: two started nodes with test table + Ignite ig = startGrid(0); + startGrid(1); + + ig.cluster().active(true); + LinkedHashMap fields = new LinkedHashMap<>(); + fields.put("A", "java.lang.Integer"); + fields.put("B", "java.lang.String"); + + CacheConfiguration ccfg = new CacheConfiguration<>(TEST_CACHE_NAME); + + ccfg.setQueryEntities(Arrays.asList( + new QueryEntity() + .setKeyType("java.lang.Integer") + .setValueType("TestIndexObject") + .setFields(fields) + )); + + IgniteCache cache = ig.getOrCreateCache(ccfg); + + fillTestData(ig); + + cache.query(new SqlFieldsQuery("create index myindexb on TestIndexObject(b)")).getAll(); + + //and: stop one node and create index on other node + stopGrid(1); + + cache.query(new SqlFieldsQuery("create index myindexa on TestIndexObject(a)")).getAll(); + cache.query(new SqlFieldsQuery("drop index myindexb")).getAll(); + cache.query(new SqlFieldsQuery("alter table TestIndexObject drop column b")).getAll(); + + //and: stop all grid + stopAllGrids(); + } + + { + //and: start cluster + IgniteEx ig0 = startGrid(0); + IgniteEx ig1 = startGrid(1); + + ig0.cluster().active(true); + + //then: config should be merged + try (IgniteDataStreamer s = ig1.dataStreamer(TEST_CACHE_NAME)) { + s.allowOverwrite(true); + for (int i = 0; i < 5_000; i++) { + BinaryObject bo = ig1.binary().builder("TestIndexObject") + .setField("a", i, Object.class) + .setField("b", String.valueOf(i), Object.class) + .build(); + + s.addData(i, bo); + } + } + IgniteCache cache = ig1.cache(TEST_CACHE_NAME); + + //then: index "myindexa" and column "b" restored from node "1" + assertThat(doExplainPlan(cache, "explain select * from TestIndexObject where a > 5"), containsString("myindexa")); + assertThat(doExplainPlan(cache, "explain select * from TestIndexObject where b > 5"), containsString("myindexb")); + assertFalse(cache.query(new SqlFieldsQuery("SELECT a,b FROM TestIndexObject limit 1")).getAll().isEmpty()); + } + } + + /** + * @throws Exception if failed. + */ + public void testTakeChangedConfigOnActiveGrid() throws Exception { + { + //given: two started nodes with test table + Ignite ig = startGrid(0); + startGrid(1); + + ig.cluster().active(true); + + IgniteCache cache = ig.getOrCreateCache(getTestTableConfiguration()); + + fillTestData(ig); + + //stop one node and create index on other node + stopGrid(1); + + cache.query(new SqlFieldsQuery("create index myindexa on TestIndexObject(a)")).getAll(); + cache.query(new SqlFieldsQuery("create index myindexb on TestIndexObject(b)")).getAll(); + cache.query(new SqlFieldsQuery("alter table TestIndexObject add column (c int)")).getAll(); + + stopAllGrids(); + } + + { + //and: start cluster + IgniteEx ig = startGrid(0); + ig.cluster().active(true); + + ig = startGrid(1); + + //then: config should be merged + try (IgniteDataStreamer s = ig.dataStreamer(TEST_CACHE_NAME)) { + s.allowOverwrite(true); + for (int i = 0; i < 5_000; i++) { + BinaryObject bo = ig.binary().builder("TestIndexObject") + .setField("a", i, Object.class) + .setField("b", String.valueOf(i), Object.class) + .setField("c", i, Object.class) + .build(); + + s.addData(i, bo); + } + } + IgniteCache cache = ig.getOrCreateCache(TEST_CACHE_NAME); + + cache.indexReadyFuture().get(); + + assertThat(doExplainPlan(cache, "explain select * from TestIndexObject where a > 5"), containsString("myindexa")); + assertFalse(cache.query(new SqlFieldsQuery("SELECT a,b,c FROM TestIndexObject limit 1")).getAll().isEmpty()); + } + } + + /** + * @throws Exception if failed. + */ + public void testFailJoiningNodeBecauseDifferentSql() throws Exception { + { + //given: two started nodes with test table + Ignite ig = startGrid(0); + startGrid(1); + + ig.cluster().active(true); + + IgniteCache cache = ig.getOrCreateCache(getTestTableConfiguration()); + + cache.query(new SqlFieldsQuery("create index myindexa on TestIndexObject(a)")).getAll(); + + //stop one node and create index on other node + stopGrid(1); + + cache.query(new SqlFieldsQuery("drop index myindexa")).getAll(); + cache.query(new SqlFieldsQuery("alter table TestIndexObject drop column b")).getAll(); + cache.query(new SqlFieldsQuery("alter table TestIndexObject add column (b int)")).getAll(); + cache.query(new SqlFieldsQuery("create index myindexa on TestIndexObject(b)")).getAll(); + + //and: stopped all grid + stopAllGrids(); + } + + { + //and: start cluster + startGrid(0); + try { + startGrid(1); + + fail("Node should start with fail"); + } + catch (Exception e) { + String cause = X.cause(e, IgniteSpiException.class).getMessage(); + assertThat(cause, containsString("fieldType of B is different")); + assertThat(cause, containsString("index MYINDEXA is different")); + } + } + + } + + /** + * @throws Exception if failed. + */ + public void testFailJoiningNodeBecauseFieldInlineSizeIsDifferent() throws Exception { + { + //given: two started nodes with test table + Ignite ig = startGrid(0); + startGrid(1); + + ig.cluster().active(true); + + IgniteCache cache = ig.getOrCreateCache(getTestTableConfiguration()); + + cache.query(new SqlFieldsQuery("create index myindexa on TestIndexObject(a) INLINE_SIZE 1000")).getAll(); + + //stop one node and create index on other node + stopGrid(1); + + cache.query(new SqlFieldsQuery("drop index myindexa")).getAll(); + cache.query(new SqlFieldsQuery("create index myindexa on TestIndexObject(a) INLINE_SIZE 2000")).getAll(); + + //and: stopped all grid + stopAllGrids(); + } + + { + //and: start cluster + startGrid(0); + try { + startGrid(1); + + fail("Node should start with fail"); + } + catch (Exception e) { + assertThat(X.cause(e, IgniteSpiException.class).getMessage(), containsString("index MYINDEXA is different")); + } + } + + } + + /** + * @throws Exception if failed. + */ + public void testFailJoiningNodeBecauseNeedConfigUpdateOnActiveGrid() throws Exception { + { + startGrid(0); + startGrid(1); + + CacheConfiguration ccfg = getTestTableConfiguration(); + + Ignite ig = ignite(0); + + ig.cluster().active(true); + + IgniteCache cache = ig.getOrCreateCache(ccfg); + + fillTestData(ig); + + stopGrid(1); + + cache.query(new SqlFieldsQuery("create index myindexa on TestIndexObject(a)")).getAll(); + cache.query(new SqlFieldsQuery("create index myindexb on TestIndexObject(b)")).getAll(); + + stopGrid(0); + } + + { + IgniteEx ig = startGrid(1); + ig.cluster().active(true); + + try { + startGrid(0); + + fail("Node should start with fail"); + } + catch (Exception e) { + assertThat(X.cause(e, IgniteSpiException.class).getMessage(), containsString("Failed to join node to the active cluster")); + } + } + } + + /** + * @return result of explain plan + */ + @NotNull private String doExplainPlan(IgniteCache cache, String sql) { + return cache.query(new SqlFieldsQuery(sql)).getAll().get(0).get(0).toString().toLowerCase(); + } + + /** + * fill data by default + */ + private void fillTestData(Ignite ig) { + try (IgniteDataStreamer s = ig.dataStreamer(TEST_CACHE_NAME)) { + for (int i = 0; i < 50_000; i++) { + BinaryObject bo = ig.binary().builder("TestIndexObject") + .setField("a", i, Object.class) + .setField("b", String.valueOf(i), Object.class) + .build(); + + s.addData(i, bo); + } + } + } + + /** + * @return cache configuration with test table + */ + @NotNull private CacheConfiguration getTestTableConfiguration() { + LinkedHashMap fields = new LinkedHashMap<>(); + fields.put("a", "java.lang.Integer"); + fields.put("B", "java.lang.String"); + + CacheConfiguration ccfg = new CacheConfiguration<>(TEST_CACHE_NAME); + + ccfg.setQueryEntities(Collections.singletonList( + new QueryEntity() + .setKeyType("java.lang.Integer") + .setValueType("TestIndexObject") + .setFields(fields) + )); + return ccfg; + } +} 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 dcb3722e9a4db..3f090620287d2 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 @@ -315,7 +315,7 @@ public void testNodeJoinOnPendingDropOperation() throws Exception { * @throws Exception If failed. */ private void checkNodeJoinOnPendingOperation(boolean addOrRemove) throws Exception { - CountDownLatch finishLatch = new CountDownLatch(4); + CountDownLatch finishLatch = new CountDownLatch(3); IgniteEx srv1 = ignitionStart(serverConfiguration(1), finishLatch); @@ -334,7 +334,6 @@ private void checkNodeJoinOnPendingOperation(boolean addOrRemove) throws Excepti ignitionStart(serverConfiguration(2), finishLatch); ignitionStart(serverConfiguration(3, true), finishLatch); - ignitionStart(clientConfiguration(4), finishLatch); assertFalse(idxFut.isDone()); 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 e10fff19a8d87..8a88602768177 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 @@ -74,6 +74,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheUpdateSqlQuerySelfTest; import org.apache.ignite.internal.processors.cache.IgniteCheckClusterStateBeforeExecuteQueryTest; 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.QueryEntityCaseMismatchTest; import org.apache.ignite.internal.processors.cache.SqlFieldsQuerySelfTest; @@ -229,6 +230,7 @@ public static TestSuite suite() throws Exception { // Config. suite.addTestSuite(IgniteCacheDuplicateEntityConfigurationSelfTest.class); suite.addTestSuite(IncorrectQueryEntityTest.class); + suite.addTestSuite(IgniteDynamicSqlRestoreTest.class); // Queries tests. suite.addTestSuite(LazyQuerySelfTest.class); From 2883ff4e958747916e7d9eec671100c366cad66b Mon Sep 17 00:00:00 2001 From: Ilya Borisov Date: Tue, 17 Apr 2018 15:01:36 +0700 Subject: [PATCH 0059/1463] IGNITE-8291 Web Console: Fixed Docker file generation. (cherry picked from commit 5614621) --- .../configuration/generator/Docker.service.js | 94 ++- .../generator/Docker.service.spec.js | 133 +++++ .../web-console/frontend/package-lock.json | 550 +++++++++--------- modules/web-console/frontend/package.json | 1 + 4 files changed, 475 insertions(+), 303 deletions(-) create mode 100644 modules/web-console/frontend/app/modules/configuration/generator/Docker.service.spec.js diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js index ea49c41927828..8b03e9aa43e8c 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js @@ -15,6 +15,15 @@ * limitations under the License. */ +import {outdent} from 'outdent/lib'; +import VersionService from 'app/services/Version.service'; +import POM_DEPENDENCIES from 'app/data/pom-dependencies.json'; +import get from 'lodash/get'; + +const version = new VersionService(); + +const ALPINE_DOCKER_SINCE = '2.1.0'; + /** * Docker file generation entry point. */ @@ -29,10 +38,10 @@ export default class IgniteDockerGenerator { * @returns {String} */ from(cluster, targetVer) { - return [ - '# Start from Apache Ignite image.', - `FROM apacheignite/ignite:${targetVer.ignite}` - ].join('\n'); + return outdent` + # Start from Apache Ignite image.', + FROM apacheignite/ignite:${targetVer.ignite} + `; } /** @@ -42,36 +51,59 @@ export default class IgniteDockerGenerator { * @param {Object} targetVer Target version. */ generate(cluster, targetVer) { + return outdent` + ${this.from(cluster, targetVer)} + + # Set config uri for node. + ENV CONFIG_URI ${this.escapeFileName(cluster.name)}-server.xml + + # Copy optional libs. + ENV OPTION_LIBS ${this.optionLibs(cluster, targetVer).join(',')} + + # Update packages and install maven. + ${this.packages(cluster, targetVer)} + + # Append project to container. + ADD . ${cluster.name} + + # Build project in container. + RUN mvn -f ${cluster.name}/pom.xml clean package -DskipTests + + # Copy project jars to node classpath. + RUN mkdir $IGNITE_HOME/libs/${cluster.name} && \\ + find ${cluster.name}/target -name "*.jar" -type f -exec cp {} $IGNITE_HOME/libs/${cluster.name} \\; + `; + } + + optionLibs(cluster, targetVer) { return [ - this.from(cluster, targetVer), - '', - '# Set config uri for node.', - `ENV CONFIG_URI ${this.escapeFileName(cluster.name)}-server.xml`, - '', - '# Copy ignite-http-rest from optional.', - 'ENV OPTION_LIBS ignite-rest-http', - '', - '# Update packages and install maven.', - 'RUN \\', - ' apt-get update &&\\', - ' apt-get install -y maven', - '', - '# Append project to container.', - `ADD . ${cluster.name}`, - '', - '# Build project in container.', - `RUN mvn -f ${cluster.name}/pom.xml clean package -DskipTests`, - '', - '# Copy project jars to node classpath.', - `RUN mkdir $IGNITE_HOME/libs/${cluster.name} && \\`, - ` find ${cluster.name}/target -name "*.jar" -type f -exec cp {} $IGNITE_HOME/libs/${cluster.name} \\;` - ].join('\n'); + 'ignite-rest-http', + get(POM_DEPENDENCIES, [get(cluster, 'discovery.kind'), 'artifactId']) + ].filter(Boolean); + } + + packages(cluster, targetVer) { + return version.since(targetVer.ignite, ALPINE_DOCKER_SINCE) + ? outdent` + RUN set -x \\ + && apk add --no-cache \\ + openjdk8 + + RUN apk --update add \\ + maven \\ + && rm -rfv /var/cache/apk/* + ` + : outdent` + RUN \\ + apt-get update &&\\ + apt-get install -y maven + `; } ignoreFile() { - return [ - 'target', - 'Dockerfile' - ].join('\n'); + return outdent` + target + Dockerfile + `; } } diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.spec.js b/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.spec.js new file mode 100644 index 0000000000000..becc35988dc30 --- /dev/null +++ b/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.spec.js @@ -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. + */ + +import DockerGenerator from './Docker.service'; +import {assert} from 'chai'; +import {outdent} from 'outdent/lib'; + +suite.only('Dockerfile generator', () => { + const generator = new DockerGenerator(); + + test('Target 2.0', () => { + const cluster = { + name: 'FooBar' + }; + + const version = {ignite: '2.0.0'}; + + assert.equal( + generator.generate(cluster, version), + outdent` + # Start from Apache Ignite image.', + FROM apacheignite/ignite:2.0.0 + + # Set config uri for node. + ENV CONFIG_URI FooBar-server.xml + + # Copy optional libs. + ENV OPTION_LIBS ignite-rest-http + + # Update packages and install maven. + RUN \\ + apt-get update &&\\ + apt-get install -y maven + + # Append project to container. + ADD . FooBar + + # Build project in container. + RUN mvn -f FooBar/pom.xml clean package -DskipTests + + # Copy project jars to node classpath. + RUN mkdir $IGNITE_HOME/libs/FooBar && \\ + find FooBar/target -name "*.jar" -type f -exec cp {} $IGNITE_HOME/libs/FooBar \\; + ` + ); + }); + test('Target 2.1', () => { + const cluster = { + name: 'FooBar' + }; + const version = {ignite: '2.1.0'}; + assert.equal( + generator.generate(cluster, version), + outdent` + # Start from Apache Ignite image.', + FROM apacheignite/ignite:2.1.0 + + # Set config uri for node. + ENV CONFIG_URI FooBar-server.xml + + # Copy optional libs. + ENV OPTION_LIBS ignite-rest-http + + # Update packages and install maven. + RUN set -x \\ + && apk add --no-cache \\ + openjdk8 + + RUN apk --update add \\ + maven \\ + && rm -rfv /var/cache/apk/* + + # Append project to container. + ADD . FooBar + + # Build project in container. + RUN mvn -f FooBar/pom.xml clean package -DskipTests + + # Copy project jars to node classpath. + RUN mkdir $IGNITE_HOME/libs/FooBar && \\ + find FooBar/target -name "*.jar" -type f -exec cp {} $IGNITE_HOME/libs/FooBar \\; + ` + ); + }); + + test('Discovery optional libs', () => { + const generateWithDiscovery = (discovery) => generator.generate({name: 'foo', discovery: {kind: discovery}}, {ignite: '2.1.0'}); + + assert.include( + generateWithDiscovery('Cloud'), + `ENV OPTION_LIBS ignite-rest-http,ignite-cloud`, + 'Adds Apache jclouds lib' + ); + + assert.include( + generateWithDiscovery('S3'), + `ENV OPTION_LIBS ignite-rest-http,ignite-aws`, + 'Adds Amazon AWS lib' + ); + + assert.include( + generateWithDiscovery('GoogleStorage'), + `ENV OPTION_LIBS ignite-rest-http,ignite-gce`, + 'Adds Google Cloud Engine lib' + ); + + assert.include( + generateWithDiscovery('ZooKeeper'), + `ENV OPTION_LIBS ignite-rest-http,ignite-zookeeper`, + 'Adds Zookeeper lib' + ); + + assert.include( + generateWithDiscovery('Kubernetes'), + `ENV OPTION_LIBS ignite-rest-http,ignite-kubernetes`, + 'Adds Kubernetes lib' + ); + }); +}); diff --git a/modules/web-console/frontend/package-lock.json b/modules/web-console/frontend/package-lock.json index 1834621f6b316..e28ef4d01c123 100644 --- a/modules/web-console/frontend/package-lock.json +++ b/modules/web-console/frontend/package-lock.json @@ -83,9 +83,9 @@ "dev": true }, "@types/lodash": { - "version": "4.14.106", - "resolved": "https://registry.npmjs.org/@types/lodash/-/lodash-4.14.106.tgz", - "integrity": "sha512-tOSvCVrvSqFZ4A/qrqqm6p37GZoawsZtoR0SJhlF7EonNZUgrn8FfT+RNQ11h+NUpMt6QVe36033f3qEKBwfWA==", + "version": "4.14.107", + "resolved": "https://registry.npmjs.org/@types/lodash/-/lodash-4.14.107.tgz", + "integrity": "sha512-afvjfP2rl3yvtv2qrCRN23zIQcDinF+munMJCoHEw2BXF22QJogTlVfNPTACQ6ieDyA6VnyKT4WLuN/wK368ng==", "dev": true }, "@types/mocha": { @@ -95,9 +95,9 @@ "dev": true }, "@types/node": { - "version": "7.0.60", - "resolved": "https://registry.npmjs.org/@types/node/-/node-7.0.60.tgz", - "integrity": "sha512-ZfCUDgCOPBDn0aAsyBOcNh1nLksuGp3LAL+8GULccZN2IkMBG2KfiwFIRrIuQkLKg1W1dIB9kQZ9MIF3IgAqlw==", + "version": "9.6.5", + "resolved": "https://registry.npmjs.org/@types/node/-/node-9.6.5.tgz", + "integrity": "sha512-NOLEgsT6UiDTjnWG5Hd2Mg25LRyz/oe8ql3wbjzgSFeRzRROhPmtlsvIrei4B46UjERF0td9SZ1ZXPLOdcrBHg==", "dev": true }, "@types/sinon": { @@ -107,9 +107,9 @@ "dev": true }, "@types/tapable": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/@types/tapable/-/tapable-1.0.1.tgz", - "integrity": "sha512-zRc13uGALq6rmLOYmpdI8X5TK6ATuf9jITC7iKTxaHqb/se7vBdiC8BEp1vM2VJQVSt3N53kDDzJOYeVkUKO/Q==", + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/@types/tapable/-/tapable-1.0.2.tgz", + "integrity": "sha512-42zEJkBpNfMEAvWR5WlwtTH22oDzcMjFsL9gDGExwF8X8WvAiw7Vwop7hPw03QT8TKfec83LwbHj6SvpqM4ELQ==", "dev": true }, "@types/uglify-js": { @@ -145,8 +145,8 @@ "integrity": "sha512-NoGVTCumOsyFfuy3934f3ktiJi+wcXHJFxT47tby3iCpuo6M/WjFA9VqT5bYO+FE46i3R0N00RpJX75HxHKDaQ==", "dev": true, "requires": { - "@types/node": "7.0.60", - "@types/tapable": "1.0.1", + "@types/node": "9.6.5", + "@types/tapable": "1.0.2", "@types/uglify-js": "3.0.2", "source-map": "0.6.1" }, @@ -200,7 +200,7 @@ "resolved": "https://registry.npmjs.org/@uirouter/visualizer/-/visualizer-4.0.2.tgz", "integrity": "sha512-95T0g9HHAjEa+sqwzfSbF6HxBG3shp2oTeGvqYk3VcLEHzrgNopEKJojd+3GNcVznQ+MUAaX4EDHXrzaHKJT6Q==", "requires": { - "d3-hierarchy": "1.1.5", + "d3-hierarchy": "1.1.6", "d3-interpolate": "1.1.6", "preact": "7.2.1" } @@ -933,7 +933,7 @@ "requires": { "bn.js": "4.11.8", "inherits": "2.0.3", - "minimalistic-assert": "1.0.0" + "minimalistic-assert": "1.0.1" } }, "assert": { @@ -1022,7 +1022,7 @@ "integrity": "sha1-Hb0cg1ZY41zj+ZhAmdsAWFx4IBQ=", "requires": { "browserslist": "1.7.7", - "caniuse-db": "1.0.30000827", + "caniuse-db": "1.0.30000830", "normalize-range": "0.1.2", "num2fraction": "1.2.2", "postcss": "5.2.18", @@ -2097,7 +2097,7 @@ "querystring-es3": "0.2.1", "read-only-stream": "2.0.0", "readable-stream": "2.3.6", - "resolve": "1.7.0", + "resolve": "1.7.1", "shasum": "1.0.2", "shell-quote": "1.6.1", "stream-browserify": "2.0.1", @@ -2202,26 +2202,26 @@ "requires": { "buffer-xor": "1.0.3", "cipher-base": "1.0.4", - "create-hash": "1.1.3", + "create-hash": "1.2.0", "evp_bytestokey": "1.0.3", "inherits": "2.0.3", "safe-buffer": "5.1.1" } }, "browserify-cipher": { - "version": "1.0.0", - "resolved": "https://registry.npmjs.org/browserify-cipher/-/browserify-cipher-1.0.0.tgz", - "integrity": "sha1-mYgkSHS/XtTijalWZtzWasj8Njo=", + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/browserify-cipher/-/browserify-cipher-1.0.1.tgz", + "integrity": "sha512-sPhkz0ARKbf4rRQt2hTpAHqn47X3llLkUGn+xEJzLjwY8LRs2p0v7ljvI5EyoRO/mexrNunNECisZs+gw2zz1w==", "requires": { "browserify-aes": "1.2.0", - "browserify-des": "1.0.0", + "browserify-des": "1.0.1", "evp_bytestokey": "1.0.3" } }, "browserify-des": { - "version": "1.0.0", - "resolved": "https://registry.npmjs.org/browserify-des/-/browserify-des-1.0.0.tgz", - "integrity": "sha1-2qJ3cXRwki7S/hhZQRihdUOXId0=", + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/browserify-des/-/browserify-des-1.0.1.tgz", + "integrity": "sha512-zy0Cobe3hhgpiOM32Tj7KQ3Vl91m0njwsjzZQK1L+JDf11dzP9qIvjreVinsvXrgfjhStXwUWAEpB9D7Gwmayw==", "requires": { "cipher-base": "1.0.4", "des.js": "1.0.0", @@ -2244,11 +2244,11 @@ "requires": { "bn.js": "4.11.8", "browserify-rsa": "4.0.1", - "create-hash": "1.1.3", - "create-hmac": "1.1.6", + "create-hash": "1.2.0", + "create-hmac": "1.1.7", "elliptic": "6.4.0", "inherits": "2.0.3", - "parse-asn1": "5.1.0" + "parse-asn1": "5.1.1" } }, "browserify-zlib": { @@ -2264,7 +2264,7 @@ "resolved": "https://registry.npmjs.org/browserslist/-/browserslist-1.7.7.tgz", "integrity": "sha1-C9dnBCWL6CmyOYu1Dkti0aFmsLk=", "requires": { - "caniuse-db": "1.0.30000827", + "caniuse-db": "1.0.30000830", "electron-to-chromium": "1.3.42" } }, @@ -2476,15 +2476,15 @@ "integrity": "sha1-tTTnxzTE+B7F++isoq0kNUuWLGw=", "requires": { "browserslist": "1.7.7", - "caniuse-db": "1.0.30000827", + "caniuse-db": "1.0.30000830", "lodash.memoize": "4.1.2", "lodash.uniq": "4.5.0" } }, "caniuse-db": { - "version": "1.0.30000827", - "resolved": "https://registry.npmjs.org/caniuse-db/-/caniuse-db-1.0.30000827.tgz", - "integrity": "sha1-vSg53Rlgk7RMKMF/k1ExQMnZJYg=" + "version": "1.0.30000830", + "resolved": "https://registry.npmjs.org/caniuse-db/-/caniuse-db-1.0.30000830.tgz", + "integrity": "sha1-bkUlWzRWSf0V/1kHLaHhK7PeLxM=" }, "caseless": { "version": "0.11.0", @@ -2674,9 +2674,9 @@ "integrity": "sha1-4qdQQqlVGQi+vSW4Uj1fl2nXkYE=" }, "chrome-trace-event": { - "version": "0.1.2", - "resolved": "https://registry.npmjs.org/chrome-trace-event/-/chrome-trace-event-0.1.2.tgz", - "integrity": "sha1-kPNohdU0WlBiEzLwcXtZWIPV2YI=" + "version": "0.1.3", + "resolved": "https://registry.npmjs.org/chrome-trace-event/-/chrome-trace-event-0.1.3.tgz", + "integrity": "sha512-sjndyZHrrWiu4RY7AkHgjn80GfAM2ZSzUkZLV/Js59Ldmh6JDThf0SUmOHU53rFu2rVxxfCzJ30Ukcfch3Gb/A==" }, "cipher-base": { "version": "1.0.4", @@ -3234,32 +3234,33 @@ "integrity": "sha1-tf1UIgqivFq1eqtxQMlAdUUDwac=" }, "create-ecdh": { - "version": "4.0.0", - "resolved": "https://registry.npmjs.org/create-ecdh/-/create-ecdh-4.0.0.tgz", - "integrity": "sha1-iIxyNZbN92EvZJgjPuvXo1MBc30=", + "version": "4.0.1", + "resolved": "https://registry.npmjs.org/create-ecdh/-/create-ecdh-4.0.1.tgz", + "integrity": "sha512-iZvCCg8XqHQZ1ioNBTzXS/cQSkqkqcPs8xSX4upNB+DAk9Ht3uzQf2J32uAHNCne8LDmKr29AgZrEs4oIrwLuQ==", "requires": { "bn.js": "4.11.8", "elliptic": "6.4.0" } }, "create-hash": { - "version": "1.1.3", - "resolved": "https://registry.npmjs.org/create-hash/-/create-hash-1.1.3.tgz", - "integrity": "sha1-YGBCrIuSYnUPSDyt2rD1gZFy2P0=", + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/create-hash/-/create-hash-1.2.0.tgz", + "integrity": "sha512-z00bCGNHDG8mHAkP7CtT1qVu+bFQUPjYq/4Iv3C3kWjTFV10zIjfSoeqXo9Asws8gwSHDGj/hl2u4OGIjapeCg==", "requires": { "cipher-base": "1.0.4", "inherits": "2.0.3", + "md5.js": "1.3.4", "ripemd160": "2.0.1", "sha.js": "2.4.11" } }, "create-hmac": { - "version": "1.1.6", - "resolved": "https://registry.npmjs.org/create-hmac/-/create-hmac-1.1.6.tgz", - "integrity": "sha1-rLniIaThe9sHbpBlfEK5PjcmzwY=", + "version": "1.1.7", + "resolved": "https://registry.npmjs.org/create-hmac/-/create-hmac-1.1.7.tgz", + "integrity": "sha512-MJG9liiZ+ogc4TzUwuvbER1JRdgvUFSB5+VR/g5h82fGaIRWMWddtKBHi7/sVhfjQZ6SehlyhvQYrcYkaUIpLg==", "requires": { "cipher-base": "1.0.4", - "create-hash": "1.1.3", + "create-hash": "1.2.0", "inherits": "2.0.3", "ripemd160": "2.0.1", "safe-buffer": "5.1.1", @@ -3289,15 +3290,15 @@ "resolved": "https://registry.npmjs.org/crypto-browserify/-/crypto-browserify-3.12.0.tgz", "integrity": "sha512-fz4spIh+znjO2VjL+IdhEpRJ3YN6sMzITSBijk6FK2UvTqruSQW+/cCZTSNsMiZNvUeq0CqurF+dAbyiGOY6Wg==", "requires": { - "browserify-cipher": "1.0.0", + "browserify-cipher": "1.0.1", "browserify-sign": "4.0.4", - "create-ecdh": "4.0.0", - "create-hash": "1.1.3", - "create-hmac": "1.1.6", - "diffie-hellman": "5.0.2", + "create-ecdh": "4.0.1", + "create-hash": "1.2.0", + "create-hmac": "1.1.7", + "diffie-hellman": "5.0.3", "inherits": "2.0.3", "pbkdf2": "3.0.14", - "public-encrypt": "4.0.0", + "public-encrypt": "4.0.2", "randombytes": "2.0.6", "randomfill": "1.0.4" } @@ -3424,7 +3425,7 @@ "resolved": "https://registry.npmjs.org/css-tree/-/css-tree-1.0.0-alpha25.tgz", "integrity": "sha512-XC6xLW/JqIGirnZuUWHXCHRaAjje2b3OIB0Vj5RIJo6mIi/AdJo30quQl5LxUl0gkXDIrTrFGbMlcZjyFplz1A==", "requires": { - "mdn-data": "1.1.0", + "mdn-data": "1.1.1", "source-map": "0.5.7" } }, @@ -3521,9 +3522,9 @@ "integrity": "sha1-vHZD/KjlOoNH4vva/6I2eWtYUJs=" }, "d3-hierarchy": { - "version": "1.1.5", - "resolved": "https://registry.npmjs.org/d3-hierarchy/-/d3-hierarchy-1.1.5.tgz", - "integrity": "sha1-ochFxC+Eoga88cAcAQmOpN2qeiY=" + "version": "1.1.6", + "resolved": "https://registry.npmjs.org/d3-hierarchy/-/d3-hierarchy-1.1.6.tgz", + "integrity": "sha512-nn4bhBnwWnMSoZgkBXD7vRyZ0xVUsNMQRKytWYHhP1I4qHw+qzApCTgSQTZqMdf4XXZbTMqA59hFusga+THA/g==" }, "d3-interpolate": { "version": "1.1.6", @@ -3759,7 +3760,7 @@ "integrity": "sha1-wHTS4qpqipoH29YfmhXCzYPsjsw=", "requires": { "inherits": "2.0.3", - "minimalistic-assert": "1.0.0" + "minimalistic-assert": "1.0.1" } }, "destroy": { @@ -3807,9 +3808,9 @@ "integrity": "sha512-A46qtFgd+g7pDZinpnwiRJtxbC1hpgf0uzP3iG89scHk0AUC7A1TGxf5OiiOUv/JMZR8GOt8hL900hV0bOy5xA==" }, "diffie-hellman": { - "version": "5.0.2", - "resolved": "https://registry.npmjs.org/diffie-hellman/-/diffie-hellman-5.0.2.tgz", - "integrity": "sha1-tYNXOScM/ias9jIJn97SoH8gnl4=", + "version": "5.0.3", + "resolved": "https://registry.npmjs.org/diffie-hellman/-/diffie-hellman-5.0.3.tgz", + "integrity": "sha512-kqag/Nl+f3GwyK25fhUMYj81BUOrZ9IuJsjIcDE5icNM9FJHAVm3VcUDxdLPoQtTuUylWm6ZIknYJwwaPxsUzg==", "requires": { "bn.js": "4.11.8", "miller-rabin": "4.0.1", @@ -3995,7 +3996,7 @@ "hash.js": "1.1.3", "hmac-drbg": "1.0.1", "inherits": "2.0.3", - "minimalistic-assert": "1.0.0", + "minimalistic-assert": "1.0.1", "minimalistic-crypto-utils": "1.0.1" } }, @@ -5104,9 +5105,9 @@ "integrity": "sha1-2uRqnXj74lKSJYzB54CkHZXAN4I=" }, "flow-parser": { - "version": "0.69.0", - "resolved": "https://registry.npmjs.org/flow-parser/-/flow-parser-0.69.0.tgz", - "integrity": "sha1-N4tRKNbQtVSosvFqTKPhq5ZJ8A4=" + "version": "0.70.0", + "resolved": "https://registry.npmjs.org/flow-parser/-/flow-parser-0.70.0.tgz", + "integrity": "sha512-gGdyVUZWswG5jcINrVDHd3RY4nJptBTAx9mR9thGsrGGmAUR7omgJXQSpR+fXrLtxSTAea3HpAZNU/yzRJc2Cg==" }, "flush-write-stream": { "version": "1.0.3", @@ -5727,11 +5728,12 @@ } }, "hash-base": { - "version": "2.0.2", - "resolved": "https://registry.npmjs.org/hash-base/-/hash-base-2.0.2.tgz", - "integrity": "sha1-ZuodhW206KVHDK32/OI65SRO8uE=", + "version": "3.0.4", + "resolved": "https://registry.npmjs.org/hash-base/-/hash-base-3.0.4.tgz", + "integrity": "sha1-X8hoaEfs1zSZQDMZprCj8/auSRg=", "requires": { - "inherits": "2.0.3" + "inherits": "2.0.3", + "safe-buffer": "5.1.1" } }, "hash.js": { @@ -5740,7 +5742,7 @@ "integrity": "sha512-/UETyP0W22QILqS+6HowevwhEFJ3MBJnwTf75Qob9Wz9t0DPuisL8kW8YZMK62dHAKE1c1p+gY1TtOLY+USEHA==", "requires": { "inherits": "2.0.3", - "minimalistic-assert": "1.0.0" + "minimalistic-assert": "1.0.1" } }, "hawk": { @@ -5765,7 +5767,7 @@ "integrity": "sha1-0nRXAQJabHdabFRXk+1QL8DGSaE=", "requires": { "hash.js": "1.1.3", - "minimalistic-assert": "1.0.0", + "minimalistic-assert": "1.0.1", "minimalistic-crypto-utils": "1.0.1" } }, @@ -5830,9 +5832,9 @@ } }, "html-minifier": { - "version": "3.5.14", - "resolved": "https://registry.npmjs.org/html-minifier/-/html-minifier-3.5.14.tgz", - "integrity": "sha512-sZjw6zhQgyUnIlIPU+W80XpRjWjdxHtNcxjfyOskOsCTDKytcfLY04wsQY/83Yqb4ndoiD2FtauiL7Yg6uUQFQ==", + "version": "3.5.15", + "resolved": "https://registry.npmjs.org/html-minifier/-/html-minifier-3.5.15.tgz", + "integrity": "sha512-OZa4rfb6tZOZ3Z8Xf0jKxXkiDcFWldQePGYFDcgKqES2sXeWaEv9y6QQvWUtX3ySI3feApQi5uCsHLINQ6NoAw==", "requires": { "camel-case": "3.0.0", "clean-css": "4.1.11", @@ -5840,7 +5842,7 @@ "he": "1.1.1", "param-case": "2.1.1", "relateurl": "0.2.7", - "uglify-js": "3.3.20" + "uglify-js": "3.3.21" }, "dependencies": { "source-map": { @@ -5849,9 +5851,9 @@ "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==" }, "uglify-js": { - "version": "3.3.20", - "resolved": "https://registry.npmjs.org/uglify-js/-/uglify-js-3.3.20.tgz", - "integrity": "sha512-WpLkWCf9sGvGZnIvBV0PNID9BATQNT/IXKAmqegfKzIPcTmTV3FP8NQpoogQkt/Y402x2sOFdaHUmqFY9IZp+g==", + "version": "3.3.21", + "resolved": "https://registry.npmjs.org/uglify-js/-/uglify-js-3.3.21.tgz", + "integrity": "sha512-uy82472lH8tshK3jS3c5IFb5MmNKd/5qyBd0ih8sM42L3jWvxnE339U9gZU1zufnLVs98Stib9twq8dLm2XYCA==", "requires": { "commander": "2.15.1", "source-map": "0.6.1" @@ -5864,7 +5866,7 @@ "resolved": "https://registry.npmjs.org/html-webpack-plugin/-/html-webpack-plugin-3.2.0.tgz", "integrity": "sha1-sBq71yOsqqeze2r0SS69oD2d03s=", "requires": { - "html-minifier": "3.5.14", + "html-minifier": "3.5.15", "loader-utils": "0.2.17", "lodash": "4.17.5", "pretty-error": "2.1.1", @@ -5902,7 +5904,7 @@ "posthtml": "0.11.3", "posthtml-render": "1.1.3", "svgo": "1.0.5", - "uglify-js": "3.3.20" + "uglify-js": "3.3.21" }, "dependencies": { "coa": { @@ -5926,7 +5928,7 @@ "resolved": "https://registry.npmjs.org/css-tree/-/css-tree-1.0.0-alpha.27.tgz", "integrity": "sha512-BAYp9FyN4jLXjfvRpTDchBllDptqlK9I7OsagXCG9Am5C+5jc8eRZHgqb9x500W2OKS14MMlpQc/nmh/aA7TEQ==", "requires": { - "mdn-data": "1.1.0", + "mdn-data": "1.1.1", "source-map": "0.5.7" } } @@ -5962,15 +5964,15 @@ "mkdirp": "0.5.1", "object.values": "1.0.4", "sax": "1.2.4", - "stable": "0.1.6", + "stable": "0.1.7", "unquote": "1.1.1", "util.promisify": "1.0.0" } }, "uglify-js": { - "version": "3.3.20", - "resolved": "https://registry.npmjs.org/uglify-js/-/uglify-js-3.3.20.tgz", - "integrity": "sha512-WpLkWCf9sGvGZnIvBV0PNID9BATQNT/IXKAmqegfKzIPcTmTV3FP8NQpoogQkt/Y402x2sOFdaHUmqFY9IZp+g==", + "version": "3.3.21", + "resolved": "https://registry.npmjs.org/uglify-js/-/uglify-js-3.3.21.tgz", + "integrity": "sha512-uy82472lH8tshK3jS3c5IFb5MmNKd/5qyBd0ih8sM42L3jWvxnE339U9gZU1zufnLVs98Stib9twq8dLm2XYCA==", "requires": { "commander": "2.15.1", "source-map": "0.6.1" @@ -6104,13 +6106,13 @@ } }, "chalk": { - "version": "2.3.2", - "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.3.2.tgz", - "integrity": "sha512-ZM4j2/ld/YZDc3Ma8PgN7gyAk+kHMMMyzLNryCPGhWrsfAuDVeuid5bpRFTDgMH9JBK2lA4dyyAkkZYF/WcqDQ==", + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.0.tgz", + "integrity": "sha512-Wr/w0f4o9LuE7K53cD0qmbAMM+2XNLzR29vFn5hqko4sxGlUsyy363NvmyGIyk5tpe9cjTr9SJYbysEyPkRnFw==", "requires": { "ansi-styles": "3.2.1", "escape-string-regexp": "1.0.5", - "supports-color": "5.3.0" + "supports-color": "5.4.0" } }, "has-flag": { @@ -6123,9 +6125,9 @@ "resolved": "https://registry.npmjs.org/postcss/-/postcss-6.0.21.tgz", "integrity": "sha512-y/bKfbQz2Nn/QBC08bwvYUxEFOVGfPIUOTsJ2CK5inzlXW9SdYR1x4pEsG9blRAF/PX+wRNdOah+gx/hv4q7dw==", "requires": { - "chalk": "2.3.2", + "chalk": "2.4.0", "source-map": "0.6.1", - "supports-color": "5.3.0" + "supports-color": "5.4.0" } }, "source-map": { @@ -6134,9 +6136,9 @@ "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==" }, "supports-color": { - "version": "5.3.0", - "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.3.0.tgz", - "integrity": "sha512-0aP01LLIskjKs3lq52EC0aGBAJhLq7B2Rd8HC/DR/PtNNpcLilNmHC12O+hu0usQpo7wtHNRqtrhBwtDb0+dNg==", + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", "requires": { "has-flag": "3.0.0" } @@ -6245,7 +6247,7 @@ "integrity": "sha512-h+xtnyk4EwKvFWHrUYsWErEVR+igKtLdchu+o0Z1RL7VU/jVMFbYir2bp6bAj8efFNxWqHX0dIss6fJQ+/+qeQ==", "requires": { "ansi-escapes": "3.1.0", - "chalk": "2.3.2", + "chalk": "2.4.0", "cli-cursor": "2.1.0", "cli-width": "2.2.0", "external-editor": "2.2.0", @@ -6274,13 +6276,13 @@ } }, "chalk": { - "version": "2.3.2", - "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.3.2.tgz", - "integrity": "sha512-ZM4j2/ld/YZDc3Ma8PgN7gyAk+kHMMMyzLNryCPGhWrsfAuDVeuid5bpRFTDgMH9JBK2lA4dyyAkkZYF/WcqDQ==", + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.0.tgz", + "integrity": "sha512-Wr/w0f4o9LuE7K53cD0qmbAMM+2XNLzR29vFn5hqko4sxGlUsyy363NvmyGIyk5tpe9cjTr9SJYbysEyPkRnFw==", "requires": { "ansi-styles": "3.2.1", "escape-string-regexp": "1.0.5", - "supports-color": "5.3.0" + "supports-color": "5.4.0" } }, "has-flag": { @@ -6297,9 +6299,9 @@ } }, "supports-color": { - "version": "5.3.0", - "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.3.0.tgz", - "integrity": "sha512-0aP01LLIskjKs3lq52EC0aGBAJhLq7B2Rd8HC/DR/PtNNpcLilNmHC12O+hu0usQpo7wtHNRqtrhBwtDb0+dNg==", + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", "requires": { "has-flag": "3.0.0" } @@ -6793,10 +6795,10 @@ "babel-register": "6.26.0", "babylon": "7.0.0-beta.44", "colors": "1.1.2", - "flow-parser": "0.69.0", + "flow-parser": "0.70.0", "lodash": "4.17.5", "micromatch": "2.3.11", - "neo-async": "2.5.0", + "neo-async": "2.5.1", "node-dir": "0.1.8", "nomnom": "1.8.1", "recast": "0.14.7", @@ -7626,7 +7628,7 @@ "resolved": "https://registry.npmjs.org/log-symbols/-/log-symbols-2.2.0.tgz", "integrity": "sha512-VeIAFslyIerEJLXHziedo2basKbMKtTw3vfn5IzG0XTjhAVEJyNHnL2p7vc+wBDSdQuUpNw3M2u6xb9QsAY5Eg==", "requires": { - "chalk": "2.3.2" + "chalk": "2.4.0" }, "dependencies": { "ansi-styles": { @@ -7638,13 +7640,13 @@ } }, "chalk": { - "version": "2.3.2", - "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.3.2.tgz", - "integrity": "sha512-ZM4j2/ld/YZDc3Ma8PgN7gyAk+kHMMMyzLNryCPGhWrsfAuDVeuid5bpRFTDgMH9JBK2lA4dyyAkkZYF/WcqDQ==", + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.0.tgz", + "integrity": "sha512-Wr/w0f4o9LuE7K53cD0qmbAMM+2XNLzR29vFn5hqko4sxGlUsyy363NvmyGIyk5tpe9cjTr9SJYbysEyPkRnFw==", "requires": { "ansi-styles": "3.2.1", "escape-string-regexp": "1.0.5", - "supports-color": "5.3.0" + "supports-color": "5.4.0" } }, "has-flag": { @@ -7653,9 +7655,9 @@ "integrity": "sha1-tdRU3CGZriJWmfNGfloH87lVuv0=" }, "supports-color": { - "version": "5.3.0", - "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.3.0.tgz", - "integrity": "sha512-0aP01LLIskjKs3lq52EC0aGBAJhLq7B2Rd8HC/DR/PtNNpcLilNmHC12O+hu0usQpo7wtHNRqtrhBwtDb0+dNg==", + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", "requires": { "has-flag": "3.0.0" } @@ -7706,7 +7708,7 @@ "integrity": "sha512-YL/qpTxYtK0iWWbuKCrevDZz5lh+OjyHHD+mICqpjnYGKdNRBvPeh/1uYjkKUemT1CSO4wwLOwphWMpKAnD9kw==", "dev": true, "requires": { - "circular-json": "0.5.1", + "circular-json": "0.5.3", "date-format": "1.2.0", "debug": "3.1.0", "semver": "5.5.0", @@ -7714,9 +7716,9 @@ }, "dependencies": { "circular-json": { - "version": "0.5.1", - "resolved": "https://registry.npmjs.org/circular-json/-/circular-json-0.5.1.tgz", - "integrity": "sha512-UjgcRlTAhAkLeXmDe2wK7ktwy/tgAqxiSndTIPiFZuIPLZmzHzWMwUIe9h9m/OokypG7snxCDEuwJshGBdPvaw==", + "version": "0.5.3", + "resolved": "https://registry.npmjs.org/circular-json/-/circular-json-0.5.3.tgz", + "integrity": "sha512-YlxLOimeIoQGHnMe3kbf8qIV2Bj7uXLbljMPRguNT49GmSAzooNfS9EJ91rSJKbLBOOzM5agvtx0WyechZN/Hw==", "dev": true }, "debug": { @@ -7838,23 +7840,12 @@ "requires": { "hash-base": "3.0.4", "inherits": "2.0.3" - }, - "dependencies": { - "hash-base": { - "version": "3.0.4", - "resolved": "https://registry.npmjs.org/hash-base/-/hash-base-3.0.4.tgz", - "integrity": "sha1-X8hoaEfs1zSZQDMZprCj8/auSRg=", - "requires": { - "inherits": "2.0.3", - "safe-buffer": "5.1.1" - } - } } }, "mdn-data": { - "version": "1.1.0", - "resolved": "https://registry.npmjs.org/mdn-data/-/mdn-data-1.1.0.tgz", - "integrity": "sha512-jC6B3BFC07cCOU8xx1d+sQtDkVIpGKWv4TzK7pN7PyObdbwlIFJbHYk8ofvr0zrU8SkV1rSi87KAHhWCdLGw1Q==" + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/mdn-data/-/mdn-data-1.1.1.tgz", + "integrity": "sha512-2J5JENcb4yD5AzBI4ilTakiq2P9gHSsi4LOygnMu/bkchgTiA63AjsHAhDc+3U36AJHRfcz30Qv6Tb7i/Qsiew==" }, "media-typer": { "version": "0.3.0", @@ -8058,9 +8049,9 @@ "integrity": "sha1-3z02Uqc/3ta5sLJBRub9BSNTRY4=" }, "minimalistic-assert": { - "version": "1.0.0", - "resolved": "https://registry.npmjs.org/minimalistic-assert/-/minimalistic-assert-1.0.0.tgz", - "integrity": "sha1-cCvi3aazf0g2vLP121ZkG2Sh09M=" + "version": "1.0.1", + "resolved": "https://registry.npmjs.org/minimalistic-assert/-/minimalistic-assert-1.0.1.tgz", + "integrity": "sha512-UtJcAD4yEaGtjPezWuO9wC4nwUnVH/8/Im3yEHQP4b67cXlD/Qr9hdITCU1xDbSEXg2XKNaP8jsReV7vQd00/A==" }, "minimalistic-crypto-utils": { "version": "1.0.1", @@ -8244,7 +8235,7 @@ "inherits": "2.0.3", "parents": "1.0.1", "readable-stream": "2.3.6", - "resolve": "1.7.0", + "resolve": "1.7.1", "stream-combiner2": "1.1.1", "subarg": "1.0.0", "through2": "2.0.3", @@ -8460,9 +8451,9 @@ "integrity": "sha1-KzJxhOiZIQEXeyhWP7XnECrNDKk=" }, "neo-async": { - "version": "2.5.0", - "resolved": "https://registry.npmjs.org/neo-async/-/neo-async-2.5.0.tgz", - "integrity": "sha512-nJmSswG4As/MkRq7QZFuH/sf/yuv8ODdMZrY4Bedjp77a5MK4A6s7YbBB64c9u79EBUOfXUXBvArmvzTD0X+6g==" + "version": "2.5.1", + "resolved": "https://registry.npmjs.org/neo-async/-/neo-async-2.5.1.tgz", + "integrity": "sha512-3KL3fvuRkZ7s4IFOMfztb7zJp3QaVWnBeGoJlgB38XnCRPj/0tLzzLG5IB8NYOHbJ8g8UGrgZv44GLDk6CxTxA==" }, "nice-try": { "version": "1.0.4", @@ -8547,7 +8538,7 @@ "stream-browserify": "2.0.1", "stream-http": "2.8.1", "string_decoder": "1.1.1", - "timers-browserify": "2.0.6", + "timers-browserify": "2.0.7", "tty-browserify": "0.0.0", "url": "0.11.0", "util": "0.10.3", @@ -9027,6 +9018,11 @@ "os-tmpdir": "1.0.2" } }, + "outdent": { + "version": "0.5.0", + "resolved": "https://registry.npmjs.org/outdent/-/outdent-0.5.0.tgz", + "integrity": "sha512-/jHxFIzoMXdqPzTaCpFzAAWhpkSjZPF4Vsn6jAfNpmbH/ymsmd7Qc6VE9BGn0L6YMj6uwpQLxCECpus4ukKS9Q==" + }, "p-cancelable": { "version": "0.4.1", "resolved": "https://registry.npmjs.org/p-cancelable/-/p-cancelable-0.4.1.tgz", @@ -9127,13 +9123,13 @@ } }, "parse-asn1": { - "version": "5.1.0", - "resolved": "https://registry.npmjs.org/parse-asn1/-/parse-asn1-5.1.0.tgz", - "integrity": "sha1-N8T5t+06tlx0gXtfJICTf7+XxxI=", + "version": "5.1.1", + "resolved": "https://registry.npmjs.org/parse-asn1/-/parse-asn1-5.1.1.tgz", + "integrity": "sha512-KPx7flKXg775zZpnp9SxJlz00gTd4BmJ2yJufSc44gMCRrRQ7NSzAcSJQfifuOLgW6bEi+ftrALtsgALeB2Adw==", "requires": { "asn1.js": "4.10.1", "browserify-aes": "1.2.0", - "create-hash": "1.1.3", + "create-hash": "1.2.0", "evp_bytestokey": "1.0.3", "pbkdf2": "3.0.14" } @@ -9261,8 +9257,8 @@ "resolved": "https://registry.npmjs.org/pbkdf2/-/pbkdf2-3.0.14.tgz", "integrity": "sha512-gjsZW9O34fm0R7PaLHRJmLLVfSoesxztjPjE9o6R+qtVJij90ltg1joIovN9GKrRW3t1PzhDDG3UMEMFfZ+1wA==", "requires": { - "create-hash": "1.1.3", - "create-hmac": "1.1.6", + "create-hash": "1.2.0", + "create-hmac": "1.1.7", "ripemd160": "2.0.1", "safe-buffer": "5.1.1", "sha.js": "2.4.11" @@ -9533,13 +9529,13 @@ } }, "chalk": { - "version": "2.3.2", - "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.3.2.tgz", - "integrity": "sha512-ZM4j2/ld/YZDc3Ma8PgN7gyAk+kHMMMyzLNryCPGhWrsfAuDVeuid5bpRFTDgMH9JBK2lA4dyyAkkZYF/WcqDQ==", + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.0.tgz", + "integrity": "sha512-Wr/w0f4o9LuE7K53cD0qmbAMM+2XNLzR29vFn5hqko4sxGlUsyy363NvmyGIyk5tpe9cjTr9SJYbysEyPkRnFw==", "requires": { "ansi-styles": "3.2.1", "escape-string-regexp": "1.0.5", - "supports-color": "5.3.0" + "supports-color": "5.4.0" } }, "has-flag": { @@ -9552,9 +9548,9 @@ "resolved": "https://registry.npmjs.org/postcss/-/postcss-6.0.21.tgz", "integrity": "sha512-y/bKfbQz2Nn/QBC08bwvYUxEFOVGfPIUOTsJ2CK5inzlXW9SdYR1x4pEsG9blRAF/PX+wRNdOah+gx/hv4q7dw==", "requires": { - "chalk": "2.3.2", + "chalk": "2.4.0", "source-map": "0.6.1", - "supports-color": "5.3.0" + "supports-color": "5.4.0" } }, "source-map": { @@ -9563,9 +9559,9 @@ "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==" }, "supports-color": { - "version": "5.3.0", - "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.3.0.tgz", - "integrity": "sha512-0aP01LLIskjKs3lq52EC0aGBAJhLq7B2Rd8HC/DR/PtNNpcLilNmHC12O+hu0usQpo7wtHNRqtrhBwtDb0+dNg==", + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", "requires": { "has-flag": "3.0.0" } @@ -9590,13 +9586,13 @@ } }, "chalk": { - "version": "2.3.2", - "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.3.2.tgz", - "integrity": "sha512-ZM4j2/ld/YZDc3Ma8PgN7gyAk+kHMMMyzLNryCPGhWrsfAuDVeuid5bpRFTDgMH9JBK2lA4dyyAkkZYF/WcqDQ==", + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.0.tgz", + "integrity": "sha512-Wr/w0f4o9LuE7K53cD0qmbAMM+2XNLzR29vFn5hqko4sxGlUsyy363NvmyGIyk5tpe9cjTr9SJYbysEyPkRnFw==", "requires": { "ansi-styles": "3.2.1", "escape-string-regexp": "1.0.5", - "supports-color": "5.3.0" + "supports-color": "5.4.0" } }, "has-flag": { @@ -9609,9 +9605,9 @@ "resolved": "https://registry.npmjs.org/postcss/-/postcss-6.0.21.tgz", "integrity": "sha512-y/bKfbQz2Nn/QBC08bwvYUxEFOVGfPIUOTsJ2CK5inzlXW9SdYR1x4pEsG9blRAF/PX+wRNdOah+gx/hv4q7dw==", "requires": { - "chalk": "2.3.2", + "chalk": "2.4.0", "source-map": "0.6.1", - "supports-color": "5.3.0" + "supports-color": "5.4.0" } }, "source-map": { @@ -9620,9 +9616,9 @@ "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==" }, "supports-color": { - "version": "5.3.0", - "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.3.0.tgz", - "integrity": "sha512-0aP01LLIskjKs3lq52EC0aGBAJhLq7B2Rd8HC/DR/PtNNpcLilNmHC12O+hu0usQpo7wtHNRqtrhBwtDb0+dNg==", + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", "requires": { "has-flag": "3.0.0" } @@ -9647,13 +9643,13 @@ } }, "chalk": { - "version": "2.3.2", - "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.3.2.tgz", - "integrity": "sha512-ZM4j2/ld/YZDc3Ma8PgN7gyAk+kHMMMyzLNryCPGhWrsfAuDVeuid5bpRFTDgMH9JBK2lA4dyyAkkZYF/WcqDQ==", + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.0.tgz", + "integrity": "sha512-Wr/w0f4o9LuE7K53cD0qmbAMM+2XNLzR29vFn5hqko4sxGlUsyy363NvmyGIyk5tpe9cjTr9SJYbysEyPkRnFw==", "requires": { "ansi-styles": "3.2.1", "escape-string-regexp": "1.0.5", - "supports-color": "5.3.0" + "supports-color": "5.4.0" } }, "has-flag": { @@ -9666,9 +9662,9 @@ "resolved": "https://registry.npmjs.org/postcss/-/postcss-6.0.21.tgz", "integrity": "sha512-y/bKfbQz2Nn/QBC08bwvYUxEFOVGfPIUOTsJ2CK5inzlXW9SdYR1x4pEsG9blRAF/PX+wRNdOah+gx/hv4q7dw==", "requires": { - "chalk": "2.3.2", + "chalk": "2.4.0", "source-map": "0.6.1", - "supports-color": "5.3.0" + "supports-color": "5.4.0" } }, "source-map": { @@ -9677,9 +9673,9 @@ "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==" }, "supports-color": { - "version": "5.3.0", - "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.3.0.tgz", - "integrity": "sha512-0aP01LLIskjKs3lq52EC0aGBAJhLq7B2Rd8HC/DR/PtNNpcLilNmHC12O+hu0usQpo7wtHNRqtrhBwtDb0+dNg==", + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", "requires": { "has-flag": "3.0.0" } @@ -9704,13 +9700,13 @@ } }, "chalk": { - "version": "2.3.2", - "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.3.2.tgz", - "integrity": "sha512-ZM4j2/ld/YZDc3Ma8PgN7gyAk+kHMMMyzLNryCPGhWrsfAuDVeuid5bpRFTDgMH9JBK2lA4dyyAkkZYF/WcqDQ==", + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.0.tgz", + "integrity": "sha512-Wr/w0f4o9LuE7K53cD0qmbAMM+2XNLzR29vFn5hqko4sxGlUsyy363NvmyGIyk5tpe9cjTr9SJYbysEyPkRnFw==", "requires": { "ansi-styles": "3.2.1", "escape-string-regexp": "1.0.5", - "supports-color": "5.3.0" + "supports-color": "5.4.0" } }, "has-flag": { @@ -9723,9 +9719,9 @@ "resolved": "https://registry.npmjs.org/postcss/-/postcss-6.0.21.tgz", "integrity": "sha512-y/bKfbQz2Nn/QBC08bwvYUxEFOVGfPIUOTsJ2CK5inzlXW9SdYR1x4pEsG9blRAF/PX+wRNdOah+gx/hv4q7dw==", "requires": { - "chalk": "2.3.2", + "chalk": "2.4.0", "source-map": "0.6.1", - "supports-color": "5.3.0" + "supports-color": "5.4.0" } }, "source-map": { @@ -9734,9 +9730,9 @@ "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==" }, "supports-color": { - "version": "5.3.0", - "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.3.0.tgz", - "integrity": "sha512-0aP01LLIskjKs3lq52EC0aGBAJhLq7B2Rd8HC/DR/PtNNpcLilNmHC12O+hu0usQpo7wtHNRqtrhBwtDb0+dNg==", + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", "requires": { "has-flag": "3.0.0" } @@ -9873,9 +9869,9 @@ } }, "posthtml-rename-id": { - "version": "1.0.3", - "resolved": "https://registry.npmjs.org/posthtml-rename-id/-/posthtml-rename-id-1.0.3.tgz", - "integrity": "sha512-zaaHJSTihw1fsx2L81npO6gmDYu4yZuHfRX89IsJDhcRIV1P8SKJY5m1xDRZQh542flidwNS+70/pVAK8yMYOA==", + "version": "1.0.4", + "resolved": "https://registry.npmjs.org/posthtml-rename-id/-/posthtml-rename-id-1.0.4.tgz", + "integrity": "sha512-bxsGN02JGqcihc9eztWu8Qlj2P/If9sY0ckYmEL+6hqrWRvwJw4RvnXSnlKmjS4yDBcT4cSpJdMy+xsSuHDvZw==", "requires": { "escape-string-regexp": "1.0.5" } @@ -9937,9 +9933,9 @@ "integrity": "sha1-gV7R9uvGWSb4ZbMQwHE7yzMVzks=" }, "prettier": { - "version": "1.11.1", - "resolved": "https://registry.npmjs.org/prettier/-/prettier-1.11.1.tgz", - "integrity": "sha512-T/KD65Ot0PB97xTrG8afQ46x3oiVhnfGjGESSI9NWYcG92+OUPZKkwHqGWXH2t9jK1crnQjubECW0FuOth+hxw==" + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/prettier/-/prettier-1.12.1.tgz", + "integrity": "sha1-wa0g6APndJ+vkFpAnSNn4Gu+cyU=" }, "pretty-bytes": { "version": "4.0.2", @@ -10037,14 +10033,14 @@ "integrity": "sha1-8FKijacOYYkX7wqKw0wa5aaChrM=" }, "public-encrypt": { - "version": "4.0.0", - "resolved": "https://registry.npmjs.org/public-encrypt/-/public-encrypt-4.0.0.tgz", - "integrity": "sha1-OfaZ86RlYN1eusvKaTyvfGXBjMY=", + "version": "4.0.2", + "resolved": "https://registry.npmjs.org/public-encrypt/-/public-encrypt-4.0.2.tgz", + "integrity": "sha512-4kJ5Esocg8X3h8YgJsKAuoesBgB7mqH3eowiDzMUPKiRDDE7E/BqqZD1hnTByIaAFiwAw246YEltSq7tdrOH0Q==", "requires": { "bn.js": "4.11.8", "browserify-rsa": "4.0.1", - "create-hash": "1.1.3", - "parse-asn1": "5.1.0", + "create-hash": "1.2.0", + "parse-asn1": "5.1.1", "randombytes": "2.0.6" } }, @@ -10103,7 +10099,7 @@ "jstransformer": "1.0.0", "pug-error": "1.3.2", "pug-walk": "1.1.7", - "resolve": "1.7.0", + "resolve": "1.7.1", "uglify-js": "2.8.29" } }, @@ -10164,7 +10160,7 @@ "requires": { "loader-utils": "1.1.0", "pug-walk": "1.1.7", - "resolve": "1.7.0" + "resolve": "1.7.1" } }, "pug-parser": { @@ -10482,7 +10478,7 @@ "resolved": "https://registry.npmjs.org/rechoir/-/rechoir-0.6.2.tgz", "integrity": "sha1-hSBLVNuoLVdC4oyWdW70OvUOM4Q=", "requires": { - "resolve": "1.7.0" + "resolve": "1.7.1" } }, "redent": { @@ -10791,9 +10787,9 @@ "integrity": "sha1-kl0mAdOaxIXgkc8NpcbmlNw9yv8=" }, "resolve": { - "version": "1.7.0", - "resolved": "https://registry.npmjs.org/resolve/-/resolve-1.7.0.tgz", - "integrity": "sha512-QdgZ5bjR1WAlpLaO5yHepFvC+o3rCr6wpfE2tpJNMkXdulf2jKomQBdNRQITF3ZKHNlT71syG98yQP03gasgnA==", + "version": "1.7.1", + "resolved": "https://registry.npmjs.org/resolve/-/resolve-1.7.1.tgz", + "integrity": "sha512-c7rwLofp8g1U+h1KNyHL/jicrKg1Ek4q+Lr33AL65uZTinUZHe30D5HlyN5V9NW0JX1D5dXQ4jqW5l7Sy/kGfw==", "requires": { "path-parse": "1.0.5" } @@ -10921,6 +10917,16 @@ "requires": { "hash-base": "2.0.2", "inherits": "2.0.3" + }, + "dependencies": { + "hash-base": { + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/hash-base/-/hash-base-2.0.2.tgz", + "integrity": "sha1-ZuodhW206KVHDK32/OI65SRO8uE=", + "requires": { + "inherits": "2.0.3" + } + } } }, "roboto-font": { @@ -10944,7 +10950,7 @@ "acorn": "5.5.3", "estree-walker": "0.5.1", "magic-string": "0.22.5", - "resolve": "1.7.0", + "resolve": "1.7.1", "rollup-pluginutils": "2.0.1" } }, @@ -10955,7 +10961,7 @@ "requires": { "builtin-modules": "2.0.0", "is-module": "1.0.0", - "resolve": "1.7.0" + "resolve": "1.7.1" } }, "rollup-plugin-progress": { @@ -11173,7 +11179,7 @@ "clone-deep": "2.0.2", "loader-utils": "1.1.0", "lodash.tail": "4.1.1", - "neo-async": "2.5.0", + "neo-async": "2.5.1", "pify": "3.0.0" } }, @@ -11924,9 +11930,9 @@ } }, "stable": { - "version": "0.1.6", - "resolved": "https://registry.npmjs.org/stable/-/stable-0.1.6.tgz", - "integrity": "sha1-kQ9dKu17Ugxud3SZwfMuE5/eyxA=" + "version": "0.1.7", + "resolved": "https://registry.npmjs.org/stable/-/stable-0.1.7.tgz", + "integrity": "sha512-LmxBix+nUtyihSBpxXAhRakYEy49fan2suysdS1fUZcKjI+krXmH8DCZJ3yfngfrOnIFNU8O73EgNTzO2jI53w==" }, "static-extend": { "version": "0.1.2", @@ -12191,7 +12197,7 @@ "micromatch": "3.1.0", "postcss": "5.2.18", "postcss-prefix-selector": "1.6.0", - "posthtml-rename-id": "1.0.3", + "posthtml-rename-id": "1.0.4", "posthtml-svg-mode": "1.0.2", "query-string": "4.3.4", "traverse": "0.6.6" @@ -12330,9 +12336,9 @@ } }, "svg-baker-runtime": { - "version": "1.3.3", - "resolved": "https://registry.npmjs.org/svg-baker-runtime/-/svg-baker-runtime-1.3.3.tgz", - "integrity": "sha512-yDnHhVM+nGxLu+Oj/zG07yUPXmJ7XLmekU2XQqL0jaLUazLjxj61uO8IMyww2roUjdMQo4x+J+KIlAp37qIbZQ==", + "version": "1.3.5", + "resolved": "https://registry.npmjs.org/svg-baker-runtime/-/svg-baker-runtime-1.3.5.tgz", + "integrity": "sha512-BKxJT/Zz9M+K043zXbZf7CA3c10NKWByxobAukO30VLv71OvmpagjG32Z0UIay6ctMaOUmywOKHuceiSDqwUOA==", "requires": { "deepmerge": "1.3.2", "mitt": "1.1.2", @@ -12350,7 +12356,7 @@ "escape-string-regexp": "1.0.5", "loader-utils": "1.1.0", "svg-baker": "1.2.17", - "svg-baker-runtime": "1.3.3", + "svg-baker-runtime": "1.3.5", "url-slug": "2.0.0" } }, @@ -12389,7 +12395,7 @@ "requires": { "ajv": "6.4.0", "ajv-keywords": "3.1.0", - "chalk": "2.3.2", + "chalk": "2.4.0", "lodash": "4.17.5", "slice-ansi": "1.0.0", "string-width": "2.1.1" @@ -12415,13 +12421,13 @@ } }, "chalk": { - "version": "2.3.2", - "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.3.2.tgz", - "integrity": "sha512-ZM4j2/ld/YZDc3Ma8PgN7gyAk+kHMMMyzLNryCPGhWrsfAuDVeuid5bpRFTDgMH9JBK2lA4dyyAkkZYF/WcqDQ==", + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.0.tgz", + "integrity": "sha512-Wr/w0f4o9LuE7K53cD0qmbAMM+2XNLzR29vFn5hqko4sxGlUsyy363NvmyGIyk5tpe9cjTr9SJYbysEyPkRnFw==", "requires": { "ansi-styles": "3.2.1", "escape-string-regexp": "1.0.5", - "supports-color": "5.3.0" + "supports-color": "5.4.0" } }, "has-flag": { @@ -12430,9 +12436,9 @@ "integrity": "sha1-tdRU3CGZriJWmfNGfloH87lVuv0=" }, "supports-color": { - "version": "5.3.0", - "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.3.0.tgz", - "integrity": "sha512-0aP01LLIskjKs3lq52EC0aGBAJhLq7B2Rd8HC/DR/PtNNpcLilNmHC12O+hu0usQpo7wtHNRqtrhBwtDb0+dNg==", + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", "requires": { "has-flag": "3.0.0" } @@ -12522,9 +12528,9 @@ "integrity": "sha1-8y6srFoXW+ol1/q1Zas+2HQe9W8=" }, "timers-browserify": { - "version": "2.0.6", - "resolved": "https://registry.npmjs.org/timers-browserify/-/timers-browserify-2.0.6.tgz", - "integrity": "sha512-HQ3nbYRAowdVd0ckGFvmJPPCOH/CHleFN/Y0YQCX1DVaB7t+KFvisuyN09fuP8Jtp1CpfSh8O8bMkHbdbPe6Pw==", + "version": "2.0.7", + "resolved": "https://registry.npmjs.org/timers-browserify/-/timers-browserify-2.0.7.tgz", + "integrity": "sha512-U7DtjfsHeYjNAyEz4MdCLGZMY3ySyHIgZZp6ba9uxZlMRMiK5yTHUYc2XfGQHKFgxGcmvBF2jafoNtQYvlDpOw==", "requires": { "setimmediate": "1.0.5" } @@ -13205,7 +13211,7 @@ "requires": { "chokidar": "2.0.3", "graceful-fs": "4.1.11", - "neo-async": "2.5.0" + "neo-async": "2.5.1" } }, "wbuf": { @@ -13213,7 +13219,7 @@ "resolved": "https://registry.npmjs.org/wbuf/-/wbuf-1.7.3.tgz", "integrity": "sha512-O84QOnr0icsbFGLS0O3bI5FswxzRr8/gHwWkDlQFskhSPryQXvrTMxjxGP4+iWYoauLoBvfDpkrOauZ+0iZpDA==", "requires": { - "minimalistic-assert": "1.0.0" + "minimalistic-assert": "1.0.1" } }, "webpack": { @@ -13225,7 +13231,7 @@ "acorn-dynamic-import": "3.0.0", "ajv": "6.4.0", "ajv-keywords": "3.1.0", - "chrome-trace-event": "0.1.2", + "chrome-trace-event": "0.1.3", "enhanced-resolve": "4.0.0", "eslint-scope": "3.7.1", "loader-runner": "2.3.0", @@ -13233,7 +13239,7 @@ "memory-fs": "0.4.1", "micromatch": "3.1.10", "mkdirp": "0.5.1", - "neo-async": "2.5.0", + "neo-async": "2.5.1", "node-libs-browser": "2.1.0", "schema-utils": "0.4.5", "tapable": "1.0.0", @@ -13566,7 +13572,7 @@ "babel-register": "6.26.0", "babylon": "6.18.0", "colors": "1.1.2", - "flow-parser": "0.69.0", + "flow-parser": "0.70.0", "lodash": "4.17.5", "micromatch": "2.3.11", "node-dir": "0.1.8", @@ -13600,7 +13606,7 @@ "resolved": "https://registry.npmjs.org/webpack-cli/-/webpack-cli-2.0.14.tgz", "integrity": "sha512-gRoWaxSi2JWiYsn1QgOTb6ENwIeSvN1YExZ+kJ0STsTZK7bWPElW+BBBv1UnTbvcPC3v7E17mK8hlFX8DOYSGw==", "requires": { - "chalk": "2.3.2", + "chalk": "2.4.0", "cross-spawn": "6.0.5", "diff": "3.5.0", "enhanced-resolve": "4.0.0", @@ -13619,13 +13625,13 @@ "mkdirp": "0.5.1", "p-each-series": "1.0.0", "p-lazy": "1.0.0", - "prettier": "1.11.1", - "supports-color": "5.3.0", + "prettier": "1.12.1", + "supports-color": "5.4.0", "v8-compile-cache": "1.1.2", "webpack-addons": "1.1.5", "yargs": "11.1.0", "yeoman-environment": "2.0.6", - "yeoman-generator": "2.0.3" + "yeoman-generator": "2.0.4" }, "dependencies": { "ansi-regex": { @@ -13647,13 +13653,13 @@ "integrity": "sha1-1UVjW+HjPFQmScaRc+Xeas+uNN0=" }, "chalk": { - "version": "2.3.2", - "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.3.2.tgz", - "integrity": "sha512-ZM4j2/ld/YZDc3Ma8PgN7gyAk+kHMMMyzLNryCPGhWrsfAuDVeuid5bpRFTDgMH9JBK2lA4dyyAkkZYF/WcqDQ==", + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.0.tgz", + "integrity": "sha512-Wr/w0f4o9LuE7K53cD0qmbAMM+2XNLzR29vFn5hqko4sxGlUsyy363NvmyGIyk5tpe9cjTr9SJYbysEyPkRnFw==", "requires": { "ansi-styles": "3.2.1", "escape-string-regexp": "1.0.5", - "supports-color": "5.3.0" + "supports-color": "5.4.0" } }, "cliui": { @@ -13689,7 +13695,7 @@ "integrity": "sha512-E9BmnJbAKLPGonz0HeWHtbKf+EeSP93paWO3ZYoUpq/aowXvYGjjCSuashhXPpzbArIjBbji39THkxTz9ZeEUQ==", "requires": { "ansi-escapes": "3.1.0", - "chalk": "2.3.2", + "chalk": "2.4.0", "cli-cursor": "2.1.0", "cli-width": "2.2.0", "external-editor": "2.2.0", @@ -13697,7 +13703,7 @@ "lodash": "4.17.5", "mute-stream": "0.0.7", "run-async": "2.3.0", - "rxjs": "5.5.8", + "rxjs": "5.5.10", "string-width": "2.1.1", "strip-ansi": "4.0.0", "through": "2.3.8" @@ -13714,9 +13720,9 @@ } }, "rxjs": { - "version": "5.5.8", - "resolved": "https://registry.npmjs.org/rxjs/-/rxjs-5.5.8.tgz", - "integrity": "sha512-Bz7qou7VAIoGiglJZbzbXa4vpX5BmTTN2Dj/se6+SwADtw4SihqBIiEa7VmTXJ8pynvq0iFr5Gx9VLyye1rIxQ==", + "version": "5.5.10", + "resolved": "https://registry.npmjs.org/rxjs/-/rxjs-5.5.10.tgz", + "integrity": "sha512-SRjimIDUHJkon+2hFo7xnvNC4ZEHGzCRwh9P7nzX3zPkCGFEg/tuElrNR7L/rZMagnK2JeH2jQwPRpmyXyLB6A==", "requires": { "symbol-observable": "1.0.1" } @@ -13730,9 +13736,9 @@ } }, "supports-color": { - "version": "5.3.0", - "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.3.0.tgz", - "integrity": "sha512-0aP01LLIskjKs3lq52EC0aGBAJhLq7B2Rd8HC/DR/PtNNpcLilNmHC12O+hu0usQpo7wtHNRqtrhBwtDb0+dNg==", + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", "requires": { "has-flag": "3.0.0" } @@ -13788,7 +13794,7 @@ "requires": { "loud-rejection": "1.6.0", "memory-fs": "0.4.1", - "mime": "2.2.2", + "mime": "2.3.1", "path-is-absolute": "1.0.1", "range-parser": "1.2.0", "url-join": "4.0.0", @@ -13796,9 +13802,9 @@ }, "dependencies": { "mime": { - "version": "2.2.2", - "resolved": "https://registry.npmjs.org/mime/-/mime-2.2.2.tgz", - "integrity": "sha512-A7PDg4s48MkqFEcYg2b069m3DXOEq7hx+9q9rIFrSSYfzsh35GX+LOVMQ8Au0ko7d8bSQCIAuzkjp0vCtwENlQ==" + "version": "2.3.1", + "resolved": "https://registry.npmjs.org/mime/-/mime-2.3.1.tgz", + "integrity": "sha512-OEUllcVoydBHGN1z84yfQDimn58pZNNNXgZlHXSboxMlFvgI6MXSWpWKpFRra7H1HxpVhHTkrghfRW49k6yjeg==" } } }, @@ -13831,7 +13837,7 @@ "sockjs-client": "1.1.4", "spdy": "3.4.7", "strip-ansi": "3.0.1", - "supports-color": "5.3.0", + "supports-color": "5.4.0", "webpack-dev-middleware": "3.0.1", "webpack-log": "1.2.0", "yargs": "9.0.1" @@ -13883,9 +13889,9 @@ "integrity": "sha1-tdRU3CGZriJWmfNGfloH87lVuv0=" }, "supports-color": { - "version": "5.3.0", - "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.3.0.tgz", - "integrity": "sha512-0aP01LLIskjKs3lq52EC0aGBAJhLq7B2Rd8HC/DR/PtNNpcLilNmHC12O+hu0usQpo7wtHNRqtrhBwtDb0+dNg==", + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", "requires": { "has-flag": "3.0.0" } @@ -13897,7 +13903,7 @@ "resolved": "https://registry.npmjs.org/webpack-log/-/webpack-log-1.2.0.tgz", "integrity": "sha512-U9AnICnu50HXtiqiDxuli5gLB5PGBo7VvcHx36jRZHwK4vzOYLbImqT4lwWwoMHdQWwEKw736fCHEekokTEKHA==", "requires": { - "chalk": "2.3.2", + "chalk": "2.4.0", "log-symbols": "2.2.0", "loglevelnext": "1.0.4", "uuid": "3.2.1" @@ -13912,13 +13918,13 @@ } }, "chalk": { - "version": "2.3.2", - "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.3.2.tgz", - "integrity": "sha512-ZM4j2/ld/YZDc3Ma8PgN7gyAk+kHMMMyzLNryCPGhWrsfAuDVeuid5bpRFTDgMH9JBK2lA4dyyAkkZYF/WcqDQ==", + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.0.tgz", + "integrity": "sha512-Wr/w0f4o9LuE7K53cD0qmbAMM+2XNLzR29vFn5hqko4sxGlUsyy363NvmyGIyk5tpe9cjTr9SJYbysEyPkRnFw==", "requires": { "ansi-styles": "3.2.1", "escape-string-regexp": "1.0.5", - "supports-color": "5.3.0" + "supports-color": "5.4.0" } }, "has-flag": { @@ -13927,9 +13933,9 @@ "integrity": "sha1-tdRU3CGZriJWmfNGfloH87lVuv0=" }, "supports-color": { - "version": "5.3.0", - "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.3.0.tgz", - "integrity": "sha512-0aP01LLIskjKs3lq52EC0aGBAJhLq7B2Rd8HC/DR/PtNNpcLilNmHC12O+hu0usQpo7wtHNRqtrhBwtDb0+dNg==", + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", "requires": { "has-flag": "3.0.0" } @@ -14307,7 +14313,7 @@ "resolved": "https://registry.npmjs.org/yeoman-environment/-/yeoman-environment-2.0.6.tgz", "integrity": "sha512-jzHBTTy8EPI4ImV8dpUMt+Q5zELkSU5xvGpndHcHudQ4tqN6YgIWaCGmRFl+HDchwRUkcgyjQ+n6/w5zlJBCPg==", "requires": { - "chalk": "2.3.2", + "chalk": "2.4.0", "debug": "3.1.0", "diff": "3.5.0", "escape-string-regexp": "1.0.5", @@ -14331,13 +14337,13 @@ } }, "chalk": { - "version": "2.3.2", - "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.3.2.tgz", - "integrity": "sha512-ZM4j2/ld/YZDc3Ma8PgN7gyAk+kHMMMyzLNryCPGhWrsfAuDVeuid5bpRFTDgMH9JBK2lA4dyyAkkZYF/WcqDQ==", + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.0.tgz", + "integrity": "sha512-Wr/w0f4o9LuE7K53cD0qmbAMM+2XNLzR29vFn5hqko4sxGlUsyy363NvmyGIyk5tpe9cjTr9SJYbysEyPkRnFw==", "requires": { "ansi-styles": "3.2.1", "escape-string-regexp": "1.0.5", - "supports-color": "5.3.0" + "supports-color": "5.4.0" } }, "debug": { @@ -14371,9 +14377,9 @@ "integrity": "sha1-7RQaasBDqEnqWISY59yosVMw6Qw=" }, "supports-color": { - "version": "5.3.0", - "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.3.0.tgz", - "integrity": "sha512-0aP01LLIskjKs3lq52EC0aGBAJhLq7B2Rd8HC/DR/PtNNpcLilNmHC12O+hu0usQpo7wtHNRqtrhBwtDb0+dNg==", + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", "requires": { "has-flag": "3.0.0" } @@ -14381,12 +14387,12 @@ } }, "yeoman-generator": { - "version": "2.0.3", - "resolved": "https://registry.npmjs.org/yeoman-generator/-/yeoman-generator-2.0.3.tgz", - "integrity": "sha512-mODmrZ26a94djmGZZuIiomSGlN4wULdou29ZwcySupb2e9FdvoCl7Ps2FqHFjEHio3kOl/iBeaNqrnx3C3NwWg==", + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/yeoman-generator/-/yeoman-generator-2.0.4.tgz", + "integrity": "sha512-Sgvz3MAkOpEIobcpW3rjEl6bOTNnl8SkibP9z7hYKfIGIlw0QDC2k0MAeXvyE2pLqc2M0Duql+6R7/W9GrJojg==", "requires": { "async": "2.6.0", - "chalk": "2.3.2", + "chalk": "2.4.0", "cli-table": "0.3.1", "cross-spawn": "5.1.0", "dargs": "5.1.0", @@ -14421,13 +14427,13 @@ } }, "chalk": { - "version": "2.3.2", - "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.3.2.tgz", - "integrity": "sha512-ZM4j2/ld/YZDc3Ma8PgN7gyAk+kHMMMyzLNryCPGhWrsfAuDVeuid5bpRFTDgMH9JBK2lA4dyyAkkZYF/WcqDQ==", + "version": "2.4.0", + "resolved": "https://registry.npmjs.org/chalk/-/chalk-2.4.0.tgz", + "integrity": "sha512-Wr/w0f4o9LuE7K53cD0qmbAMM+2XNLzR29vFn5hqko4sxGlUsyy363NvmyGIyk5tpe9cjTr9SJYbysEyPkRnFw==", "requires": { "ansi-styles": "3.2.1", "escape-string-regexp": "1.0.5", - "supports-color": "5.3.0" + "supports-color": "5.4.0" } }, "debug": { @@ -14493,9 +14499,9 @@ "integrity": "sha1-IzTBjpx1n3vdVv3vfprj1YjmjtM=" }, "supports-color": { - "version": "5.3.0", - "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.3.0.tgz", - "integrity": "sha512-0aP01LLIskjKs3lq52EC0aGBAJhLq7B2Rd8HC/DR/PtNNpcLilNmHC12O+hu0usQpo7wtHNRqtrhBwtDb0+dNg==", + "version": "5.4.0", + "resolved": "https://registry.npmjs.org/supports-color/-/supports-color-5.4.0.tgz", + "integrity": "sha512-zjaXglF5nnWpsq470jSv6P9DwPvgLkuapYmfDm3JWOm0vkNTVF2tI4UrN2r6jH1qM/uc/WtxYY1hYoA2dOKj5w==", "requires": { "has-flag": "3.0.0" } diff --git a/modules/web-console/frontend/package.json b/modules/web-console/frontend/package.json index 3b42a61947f8a..87a516f584d78 100644 --- a/modules/web-console/frontend/package.json +++ b/modules/web-console/frontend/package.json @@ -94,6 +94,7 @@ "natural-compare-lite": "^1.4.0", "node-sass": "4.8.3", "nvd3": "1.8.6", + "outdent": "^0.5.0", "pako": "1.0.6", "progress-bar-webpack-plugin": "1.11.0", "pug-html-loader": "1.1.0", From d0997d7740ea1114b4c8236f225d989de98e2f10 Mon Sep 17 00:00:00 2001 From: YuriBabak Date: Tue, 17 Apr 2018 11:22:14 +0300 Subject: [PATCH 0060/1463] IGNITE-8292: Broken yardstick compilation. this closes #3838 (cherry picked from commit e76fcb4) --- ...zyCMeansDistributedClustererBenchmark.java | 130 ------------------ ...iteFuzzyCMeansLocalClustererBenchmark.java | 93 ------------- 2 files changed, 223 deletions(-) delete mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/IgniteFuzzyCMeansDistributedClustererBenchmark.java delete mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/IgniteFuzzyCMeansLocalClustererBenchmark.java diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/IgniteFuzzyCMeansDistributedClustererBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/IgniteFuzzyCMeansDistributedClustererBenchmark.java deleted file mode 100644 index e356746ebf094..0000000000000 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/IgniteFuzzyCMeansDistributedClustererBenchmark.java +++ /dev/null @@ -1,130 +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.yardstick.ml.clustering; - -import java.util.Map; -import org.apache.ignite.Ignite; -import org.apache.ignite.ml.clustering.BaseFuzzyCMeansClusterer; -import org.apache.ignite.ml.clustering.FuzzyCMeansDistributedClusterer; -import org.apache.ignite.ml.clustering.FuzzyCMeansModel; -import org.apache.ignite.ml.math.StorageConstants; -import org.apache.ignite.ml.math.distances.DistanceMeasure; -import org.apache.ignite.ml.math.distances.EuclideanDistance; -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.resources.IgniteInstanceResource; -import org.apache.ignite.thread.IgniteThread; -import org.apache.ignite.yardstick.IgniteAbstractBenchmark; -import org.apache.ignite.yardstick.ml.DataChanger; - -/** - * Ignite benchmark that performs ML Grid operations. - */ -@SuppressWarnings("unused") -public class IgniteFuzzyCMeansDistributedClustererBenchmark extends IgniteAbstractBenchmark { - /** */ - @IgniteInstanceResource - private Ignite ignite; - - /** {@inheritDoc} */ - @Override public boolean test(Map ctx) throws Exception { - // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread - // because we create ignite cache internally. - IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), - this.getClass().getSimpleName(), new Runnable() { - /** {@inheritDoc} */ - @Override public void run() { - // IMPL NOTE originally taken from FuzzyCMeansExample. - // Distance measure that computes distance between two points. - DistanceMeasure distanceMeasure = new EuclideanDistance(); - - // "Fuzziness" - specific constant that is used in membership calculation (1.0+-eps ~ K-Means). - double exponentialWeight = 2.0; - - // Condition that indicated when algorithm must stop. - // In this example algorithm stops if memberships have changed insignificantly. - BaseFuzzyCMeansClusterer.StopCondition stopCond = - BaseFuzzyCMeansClusterer.StopCondition.STABLE_MEMBERSHIPS; - - // Maximum difference between new and old membership values with which algorithm will continue to work. - double maxDelta = 0.01; - - // The maximum number of FCM iterations. - int maxIterations = 50; - - // Number of steps of primary centers selection (more steps more candidates). - int initializationSteps = 2; - - // Number of K-Means iteration that is used to choose required number of primary centers from candidates. - int kMeansMaxIterations = 50; - - // Create new distributed clusterer with parameters described above. - FuzzyCMeansDistributedClusterer clusterer = new FuzzyCMeansDistributedClusterer( - distanceMeasure, exponentialWeight, stopCond, maxDelta, maxIterations, - null, initializationSteps, kMeansMaxIterations); - - // Create sample data. - double[][] points = shuffle((int)(DataChanger.next())); - - // Initialize matrix of data points. Each row contains one point. - int rows = points.length; - int cols = points[0].length; - - // Create the matrix that contains sample points. - SparseDistributedMatrix pntMatrix = new SparseDistributedMatrix(rows, cols, - StorageConstants.ROW_STORAGE_MODE, StorageConstants.RANDOM_ACCESS_MODE); - - // Store points into matrix. - pntMatrix.assign(points); - - // Call clusterization method with some number of centers. - // It returns model that can predict results for new points. - int numCenters = 4; - FuzzyCMeansModel mdl = clusterer.cluster(pntMatrix, numCenters); - - // Get centers of clusters that is computed by Fuzzy C-Means algorithm. - mdl.centers(); - - pntMatrix.destroy(); - } - }); - - igniteThread.start(); - - igniteThread.join(); - - return true; - } - - /** */ - private double[][] shuffle(int off) { - final double[][] points = new double[][] { - {-10, -10}, {-9, -11}, {-10, -9}, {-11, -9}, - {10, 10}, {9, 11}, {10, 9}, {11, 9}, - {-10, 10}, {-9, 11}, {-10, 9}, {-11, 9}, - {10, -10}, {9, -11}, {10, -9}, {11, -9}}; - - final int size = points.length; - - final double[][] res = new double[size][]; - - for (int i = 0; i < size; i++) - res[i] = points[(i + off) % size]; - - return res; - } -} diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/IgniteFuzzyCMeansLocalClustererBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/IgniteFuzzyCMeansLocalClustererBenchmark.java deleted file mode 100644 index 8c4c9ce7529d1..0000000000000 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/IgniteFuzzyCMeansLocalClustererBenchmark.java +++ /dev/null @@ -1,93 +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.yardstick.ml.clustering; - -import java.util.Map; -import org.apache.ignite.ml.clustering.BaseFuzzyCMeansClusterer; -import org.apache.ignite.ml.clustering.FuzzyCMeansLocalClusterer; -import org.apache.ignite.ml.clustering.FuzzyCMeansModel; -import org.apache.ignite.ml.math.distances.DistanceMeasure; -import org.apache.ignite.ml.math.distances.EuclideanDistance; -import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; -import org.apache.ignite.yardstick.IgniteAbstractBenchmark; -import org.apache.ignite.yardstick.ml.DataChanger; - -/** - * Ignite benchmark that performs ML Grid operations. - */ -@SuppressWarnings("unused") -public class IgniteFuzzyCMeansLocalClustererBenchmark extends IgniteAbstractBenchmark { - /** {@inheritDoc} */ - @Override public boolean test(Map ctx) throws Exception { - // IMPL NOTE originally taken from FuzzyLocalCMeansExample. - // Distance measure that computes distance between two points. - DistanceMeasure distanceMeasure = new EuclideanDistance(); - - // "Fuzziness" - specific constant that is used in membership calculation (1.0+-eps ~ K-Means). - double exponentialWeight = 2.0; - - // Condition that indicated when algorithm must stop. - // In this example algorithm stops if memberships have changed insignificantly. - BaseFuzzyCMeansClusterer.StopCondition stopCond = - BaseFuzzyCMeansClusterer.StopCondition.STABLE_MEMBERSHIPS; - - // Maximum difference between new and old membership values with which algorithm will continue to work. - double maxDelta = 0.01; - - // The maximum number of FCM iterations. - int maxIterations = 50; - - // Create new local clusterer with parameters described above. - FuzzyCMeansLocalClusterer clusterer = new FuzzyCMeansLocalClusterer(distanceMeasure, - exponentialWeight, stopCond, maxDelta, maxIterations, null); - - // Create sample data. - double[][] points = shuffle((int)(DataChanger.next())); - - // Create the matrix that contains sample points. - DenseLocalOnHeapMatrix pntMatrix = new DenseLocalOnHeapMatrix(points); - - // Call clusterization method with some number of centers. - // It returns model that can predict results for new points. - int numCenters = 4; - FuzzyCMeansModel mdl = clusterer.cluster(pntMatrix, numCenters); - - // Get centers of clusters that is computed by Fuzzy C-Means algorithm. - mdl.centers(); - - return true; - } - - /** */ - private double[][] shuffle(int off) { - final double[][] points = new double[][] { - {-10, -10}, {-9, -11}, {-10, -9}, {-11, -9}, - {10, 10}, {9, 11}, {10, 9}, {11, 9}, - {-10, 10}, {-9, 11}, {-10, 9}, {-11, 9}, - {10, -10}, {9, -11}, {10, -9}, {11, -9}}; - - final int size = points.length; - - final double[][] res = new double[size][]; - - for (int i = 0; i < size; i++) - res[i] = points[(i + off) % size]; - - return res; - } -} From 3d2556bc73eff6c5ccd52af1bea88b6016358db8 Mon Sep 17 00:00:00 2001 From: Ilya Borisov Date: Tue, 17 Apr 2018 15:46:10 +0700 Subject: [PATCH 0061/1463] IGNITE-8285 Web console: Removed debug output. (cherry picked from commit 8c80dce) --- .../web-console/frontend/app/components/page-configure/index.js | 1 - 1 file changed, 1 deletion(-) diff --git a/modules/web-console/frontend/app/components/page-configure/index.js b/modules/web-console/frontend/app/components/page-configure/index.js index 3209edeb4094f..34b8cfe3f0f6f 100644 --- a/modules/web-console/frontend/app/components/page-configure/index.js +++ b/modules/web-console/frontend/app/components/page-configure/index.js @@ -164,7 +164,6 @@ export default angular state: actionsWindow.filter((a) => !actions.includes(a)).reduce(ConfigureState._combinedReducer, {}) }; }) - .debug('UNDOED') .do((a) => ConfigureState.dispatchAction(a)) .subscribe(); ConfigEffects.connect(); From 4846e967e4cb7a174880a2956e807505a78fd441 Mon Sep 17 00:00:00 2001 From: YuriBabak Date: Tue, 17 Apr 2018 11:54:41 +0300 Subject: [PATCH 0062/1463] IGNITE-8292: Broken yardstick compilation. this closes #3840 (cherry picked from commit 3cebf91) --- ...teKMeansDistributedClustererBenchmark.java | 75 ------------------- .../IgniteKMeansLocalClustererBenchmark.java | 50 ------------- .../yardstick/ml/clustering/package-info.java | 22 ------ 3 files changed, 147 deletions(-) delete mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/IgniteKMeansDistributedClustererBenchmark.java delete mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/IgniteKMeansLocalClustererBenchmark.java delete mode 100644 modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/package-info.java diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/IgniteKMeansDistributedClustererBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/IgniteKMeansDistributedClustererBenchmark.java deleted file mode 100644 index de928e88e7733..0000000000000 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/IgniteKMeansDistributedClustererBenchmark.java +++ /dev/null @@ -1,75 +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.yardstick.ml.clustering; - -import java.util.Map; -import org.apache.ignite.Ignite; -import org.apache.ignite.ml.clustering.KMeansDistributedClusterer; -import org.apache.ignite.ml.math.StorageConstants; -import org.apache.ignite.ml.math.distances.EuclideanDistance; -import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix; -import org.apache.ignite.resources.IgniteInstanceResource; -import org.apache.ignite.thread.IgniteThread; -import org.apache.ignite.yardstick.IgniteAbstractBenchmark; -import org.apache.ignite.yardstick.ml.DataChanger; - -/** - * Ignite benchmark that performs ML Grid operations. - */ -@SuppressWarnings("unused") -public class IgniteKMeansDistributedClustererBenchmark extends IgniteAbstractBenchmark { - /** */ - @IgniteInstanceResource - private Ignite ignite; - - /** {@inheritDoc} */ - @Override public boolean test(Map ctx) throws Exception { - final DataChanger.Scale scale = new DataChanger.Scale(); - - // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread - // because we create ignite cache internally. - IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(), - this.getClass().getSimpleName(), new Runnable() { - /** {@inheritDoc} */ - @Override public void run() { - // IMPL NOTE originally taken from KMeansDistributedClustererTest - KMeansDistributedClusterer clusterer = new KMeansDistributedClusterer( - new EuclideanDistance(), 1, 1, 1L); - - double[] v1 = scale.mutate(new double[] {1959, 325100}); - double[] v2 = scale.mutate(new double[] {1960, 373200}); - - SparseDistributedMatrix points = new SparseDistributedMatrix( - 2, 2, StorageConstants.ROW_STORAGE_MODE, StorageConstants.RANDOM_ACCESS_MODE); - - points.setRow(0, v1); - points.setRow(1, v2); - - clusterer.cluster(points, 1); - - points.destroy(); - } - }); - - igniteThread.start(); - - igniteThread.join(); - - return true; - } -} diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/IgniteKMeansLocalClustererBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/IgniteKMeansLocalClustererBenchmark.java deleted file mode 100644 index d68fc6d19f342..0000000000000 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/IgniteKMeansLocalClustererBenchmark.java +++ /dev/null @@ -1,50 +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.yardstick.ml.clustering; - -import java.util.Map; -import org.apache.ignite.ml.clustering.KMeansLocalClusterer; -import org.apache.ignite.ml.math.distances.EuclideanDistance; -import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix; -import org.apache.ignite.yardstick.IgniteAbstractBenchmark; -import org.apache.ignite.yardstick.ml.DataChanger; - -/** - * Ignite benchmark that performs ML Grid operations. - */ -@SuppressWarnings("unused") -public class IgniteKMeansLocalClustererBenchmark extends IgniteAbstractBenchmark { - /** {@inheritDoc} */ - @Override public boolean test(Map ctx) throws Exception { - final DataChanger.Scale scale = new DataChanger.Scale(); - - // IMPL NOTE originally taken from KMeansLocalClustererTest - KMeansLocalClusterer clusterer = new KMeansLocalClusterer(new EuclideanDistance(), 1, 1L); - - double[] v1 = scale.mutate(new double[] {1959, 325100}); - double[] v2 = scale.mutate(new double[] {1960, 373200}); - - DenseLocalOnHeapMatrix points = new DenseLocalOnHeapMatrix(new double[][] { - v1, - v2}); - - clusterer.cluster(points, 1); - - return true; - } -} diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/package-info.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/package-info.java deleted file mode 100644 index af217d24644a2..0000000000000 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/clustering/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. - */ - -/** - * - * ML Grid clustering benchmarks. - */ -package org.apache.ignite.yardstick.ml.clustering; \ No newline at end of file From 733a62bcb6c0d9381a496f07417c10c7edea6d7c Mon Sep 17 00:00:00 2001 From: Ilya Borisov Date: Tue, 17 Apr 2018 14:12:39 +0700 Subject: [PATCH 0063/1463] IGNITE-8287 Change position on signup inputs on page-sign-in. (cherry picked from commit e5c3f89) --- .../app/components/page-signin/style.scss | 10 +++++ .../app/components/page-signin/template.pug | 39 +++++++++---------- 2 files changed, 29 insertions(+), 20 deletions(-) diff --git a/modules/web-console/frontend/app/components/page-signin/style.scss b/modules/web-console/frontend/app/components/page-signin/style.scss index 7e13ffe502efd..8ea143af440cd 100644 --- a/modules/web-console/frontend/app/components/page-signin/style.scss +++ b/modules/web-console/frontend/app/components/page-signin/style.scss @@ -35,4 +35,14 @@ page-sign-in { background-color: #ffffff; color: #444444; } + + .ps-grid { + display: grid; + grid-gap: 10px; + grid-template-columns: 1fr 1fr; + + .ps-grid-full-width { + grid-column: 1 / 3; + } + } } \ No newline at end of file diff --git a/modules/web-console/frontend/app/components/page-signin/template.pug b/modules/web-console/frontend/app/components/page-signin/template.pug index 9a8b3badd6f08..58d85715dc0a5 100644 --- a/modules/web-console/frontend/app/components/page-signin/template.pug +++ b/modules/web-console/frontend/app/components/page-signin/template.pug @@ -27,10 +27,9 @@ section .row .col-xs-12.col-md-11 -var form = '$ctrl.form_signup' - form(name=form novalidate) - .settings-row - h3 Don't Have An Account? - .settings-row + h3 Don't Have An Account? + form.ps-grid(name=form novalidate) + .ps-grid-full-width +form-field__email({ label: 'Email:', model: '$ctrl.data.signup.email', @@ -42,7 +41,7 @@ section ng-model-options='{allowInvalid: true}' ) +form-field__error({error: 'server', message: `{{$ctrl.serverErrors.signup}}`}) - .settings-row + div +form-field__password({ label: 'Password:', model: '$ctrl.data.signup.password', @@ -52,7 +51,7 @@ section })( ignite-on-enter-focus-move='confirmInput' ) - .settings-row + div +form-field__password({ label: 'Confirm:', model: 'confirm', @@ -63,7 +62,7 @@ section ignite-on-enter-focus-move='firstNameInput' ignite-match='$ctrl.data.signup.password' ) - .settings-row + div +form-field__text({ label: 'First name:', model: '$ctrl.data.signup.firstName', @@ -73,7 +72,7 @@ section })( ignite-on-enter-focus-move='lastNameInput' ) - .settings-row + div +form-field__text({ label: 'Last name:', model: '$ctrl.data.signup.lastName', @@ -83,17 +82,7 @@ section })( ignite-on-enter-focus-move='companyInput' ) - .settings-row - +form-field__text({ - label: 'Company:', - model: '$ctrl.data.signup.company', - name: '"company"', - placeholder: 'Input company name', - required: true - })( - ignite-on-enter-focus-move='countryInput' - ) - .settings-row + .ps-grid-full-width +form-field__dropdown({ label: 'Country:', model: '$ctrl.data.signup.country', @@ -104,7 +93,17 @@ section })( ignite-on-enter-focus-move='signup_submit' ) - .login-footer + .ps-grid-full-width + +form-field__text({ + label: 'Company:', + model: '$ctrl.data.signup.company', + name: '"company"', + placeholder: 'Input company name', + required: true + })( + ignite-on-enter-focus-move='countryInput' + ) + .login-footer.ps-grid-full-width button#signup_submit.btn-ignite.btn-ignite--primary( ng-click='$ctrl.signup()' ng-disabled=`!$ctrl.canSubmitForm(${form})` From 83e54311fce1d46279c6ddd687ced6f7c9f17ff6 Mon Sep 17 00:00:00 2001 From: Ilya Borisov Date: Tue, 17 Apr 2018 17:15:57 +0700 Subject: [PATCH 0064/1463] IGNITE-8200 Web Console: Override clonedCluster in cluster-edit-form if caches or models have changed. This improves interop with "import from DB" feature, which might update caches/models of cluster currently opened for editing. The import dialog works as a separate state, so the form change detection mechanism ensures that any changes to the original cluster are safe and won't interfere with changes made by user in cluster edit form. (cherry picked from commit 7731669) --- .../cluster-edit-form/controller.js | 24 +++++- .../cluster-edit-form/controller.spec.js | 81 +++++++++++++++++++ 2 files changed, 102 insertions(+), 3 deletions(-) create mode 100644 modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/controller.spec.js diff --git a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/controller.js b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/controller.js index 35b43e0727440..0207729821379 100644 --- a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/controller.js +++ b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/controller.js @@ -17,6 +17,7 @@ import cloneDeep from 'lodash/cloneDeep'; import get from 'lodash/get'; +import isEqual from 'lodash/isEqual'; import _ from 'lodash'; export default class ClusterEditFormController { @@ -29,9 +30,11 @@ export default class ClusterEditFormController { constructor(IgniteLegacyUtils, IgniteEventGroups, IgniteConfirm, IgniteVersion, $scope, Clusters, IgniteFormUtils) { Object.assign(this, {IgniteLegacyUtils, IgniteEventGroups, IgniteConfirm, IgniteVersion, $scope, Clusters, IgniteFormUtils}); } + $onDestroy() { this.subscription.unsubscribe(); } + $onInit() { this.available = this.IgniteVersion.available.bind(this.IgniteVersion); @@ -87,10 +90,9 @@ export default class ClusterEditFormController { this.$scope.ui = this.IgniteFormUtils.formUI(); this.$scope.ui.loadedPanels = ['checkpoint', 'serviceConfiguration', 'odbcConfiguration']; } + $onChanges(changes) { - if ( - 'cluster' in changes && get(this.clonedCluster, '_id') !== get(this.cluster, '_id') - ) { + if ('cluster' in changes && this.shouldOverwriteValue(this.cluster, this.clonedCluster)) { this.clonedCluster = cloneDeep(changes.cluster.currentValue); if (this.$scope.ui && this.$scope.ui.inputForm) { this.$scope.ui.inputForm.$setPristine(); @@ -100,14 +102,30 @@ export default class ClusterEditFormController { if ('caches' in changes) this.cachesMenu = (changes.caches.currentValue || []).map((c) => ({label: c.name, value: c._id})); } + + /** + * The form should accept incoming cluster value if: + * 1. It has different _id ("new" to real id). + * 2. Different caches or models (imported from DB). + * @param {Object} a Incoming value. + * @param {Object} b Current value. + */ + shouldOverwriteValue(a, b) { + return get(a, '_id') !== get(b, '_id') || + !isEqual(get(a, 'caches'), get(b, 'caches')) || + !isEqual(get(a, 'models'), get(b, 'models')); + } + getValuesToCompare() { return [this.cluster, this.clonedCluster].map(this.Clusters.normalize); } + save() { if (this.$scope.ui.inputForm.$invalid) return this.IgniteFormUtils.triggerValidation(this.$scope.ui.inputForm, this.$scope); this.onSave({$event: cloneDeep(this.clonedCluster)}); } + reset = () => this.clonedCluster = cloneDeep(this.cluster); confirmAndReset() { return this.IgniteConfirm.confirm('Are you sure you want to undo all changes for current cluster?') diff --git a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/controller.spec.js b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/controller.spec.js new file mode 100644 index 0000000000000..cac888f769120 --- /dev/null +++ b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/controller.spec.js @@ -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. + */ + +import 'mocha'; +import {assert} from 'chai'; +import {spy} from 'sinon'; +import Controller from './controller'; + +suite('cluster-edit-form controller', () => { + test('cluster binding changes', () => { + const $scope = { + ui: { + inputForm: { + $setPristine: spy(), + $setUntouched: spy() + } + } + }; + + const mocks = Controller.$inject.map((token) => { + switch (token) { + case '$scope': return $scope; + default: return null; + } + }); + + const changeBoundCluster = ($ctrl, cluster) => { + $ctrl.cluster = cluster; + $ctrl.$onChanges({ + cluster: { + currentValue: cluster + } + }); + }; + + const $ctrl = new Controller(...mocks); + + const cluster1 = {_id: 1, caches: [1, 2, 3]}; + const cluster2 = {_id: 1, caches: [1, 2, 3, 4, 5, 6], models: [1, 2, 3]}; + const cluster3 = {_id: 1, caches: [1, 2, 3, 4, 5, 6], models: [1, 2, 3], name: 'Foo'}; + + changeBoundCluster($ctrl, cluster1); + + assert.notEqual($ctrl.clonedCluster, cluster1, 'Cloned cluster is really cloned'); + assert.deepEqual($ctrl.clonedCluster, cluster1, 'Cloned cluster is really a clone of incloming value'); + assert.equal(1, $scope.ui.inputForm.$setPristine.callCount, 'Sets form pristine when cluster value changes'); + assert.equal(1, $scope.ui.inputForm.$setUntouched.callCount, 'Sets form untouched when cluster value changes'); + + changeBoundCluster($ctrl, cluster2); + + assert.deepEqual( + $ctrl.clonedCluster, + cluster2, + 'Overrides clonedCluster if incoming cluster has same id but different caches or models' + ); + assert.equal(2, $scope.ui.inputForm.$setPristine.callCount, 'Sets form pristine when bound cluster caches/models change'); + assert.equal(2, $scope.ui.inputForm.$setUntouched.callCount, 'Sets form untouched when bound cluster caches/models change'); + + changeBoundCluster($ctrl, cluster3); + + assert.deepEqual( + $ctrl.clonedCluster, + cluster2, + 'Does not change cloned cluster value if fields other than id, chaches and models change' + ); + }); +}); From 86d3f196e436095f277bb9b3e2c32293185db634 Mon Sep 17 00:00:00 2001 From: Sergey Chugunov Date: Tue, 17 Apr 2018 14:28:47 +0300 Subject: [PATCH 0065/1463] IGNITE-8210 Fixed custom event handling for baseline topology change - Fixes #3814. Signed-off-by: Alexey Goncharuk --- .../affinity/GridAffinityAssignmentCache.java | 2 +- .../CacheBaselineTopologyTest.java | 94 +++++++++++++++++++ 2 files changed, 95 insertions(+), 1 deletion(-) 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 18edd028a2eef..427d60328b7fb 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 @@ -315,7 +315,7 @@ public List> calculate( for (DiscoveryEvent event : events.events()) { boolean affinityNode = CU.affinityNode(event.eventNode(), nodeFilter); - if (affinityNode) { + if (affinityNode || event.type() == EVT_DISCOVERY_CUSTOM_EVT) { skipCalculation = false; break; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java index 26502ed62391e..0d59a2d79a13d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java @@ -32,6 +32,7 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cache.affinity.AffinityFunction; import org.apache.ignite.cache.affinity.AffinityFunctionContext; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; @@ -54,6 +55,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; @@ -80,6 +82,12 @@ public class CacheBaselineTopologyTest extends GridCommonAbstractTest { /** */ private boolean delayRebalance; + /** */ + private Map userAttrs; + + /** */ + private static final String DATA_NODE = "dataNodeUserAttr"; + /** */ private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); @@ -129,6 +137,9 @@ public class CacheBaselineTopologyTest extends GridCommonAbstractTest { .setWalMode(WALMode.LOG_ONLY) ); + if (userAttrs != null) + cfg.setUserAttributes(userAttrs); + if (client) cfg.setClientMode(true); @@ -138,6 +149,89 @@ public class CacheBaselineTopologyTest extends GridCommonAbstractTest { return cfg; } + /** + * Verifies that rebalance on cache with Node Filter happens when BaselineTopology changes. + * + * @throws Exception + */ + public void testRebalanceForCacheWithNodeFilter() throws Exception { + try { + final int EMPTY_NODE_IDX = 2; + + userAttrs = U.newHashMap(1); + userAttrs.put(DATA_NODE, true); + + startGrids(2); + + userAttrs.put(DATA_NODE, false); + + IgniteEx ignite = startGrid(2); + + ignite.cluster().active(true); + + awaitPartitionMapExchange(); + + IgniteCache cache = + ignite.createCache( + new CacheConfiguration() + .setName(CACHE_NAME) + .setCacheMode(PARTITIONED) + .setBackups(1) + .setPartitionLossPolicy(READ_ONLY_SAFE) + .setAffinity(new RendezvousAffinityFunction(32, null)) + .setNodeFilter(new DataNodeFilter()) + ); + + for (int k = 0; k < 10_000; k++) + cache.put(k, k); + + Thread.sleep(500); + + printSizesDataNodes(NODE_COUNT - 1, EMPTY_NODE_IDX); + + userAttrs.put(DATA_NODE, true); + + startGrid(3); + + ignite.cluster().setBaselineTopology(ignite.cluster().topologyVersion()); + + awaitPartitionMapExchange(); + + Thread.sleep(500); + + printSizesDataNodes(NODE_COUNT, EMPTY_NODE_IDX); + } + finally { + userAttrs = null; + } + } + + /** */ + private void printSizesDataNodes(int nodesCnt, int emptyNodeIdx) { + for (int i = 0; i < nodesCnt; i++) { + IgniteEx ig = grid(i); + + int locSize = ig.cache(CACHE_NAME).localSize(CachePeekMode.PRIMARY); + + if (i == emptyNodeIdx) + assertEquals("Cache local size on " + + i + + " node is expected to be zero", 0, locSize); + else + assertTrue("Cache local size on " + + i + + " node is expected to be non zero", locSize > 0); + } + } + + /** */ + private static class DataNodeFilter implements IgnitePredicate { + + @Override public boolean apply(ClusterNode clusterNode) { + return clusterNode.attribute(DATA_NODE); + } + } + /** * @throws Exception If failed. */ From a7dbea16064bbd52907a770bb40c3a2445313db2 Mon Sep 17 00:00:00 2001 From: Dmitriy Sorokin Date: Tue, 17 Apr 2018 14:48:44 +0300 Subject: [PATCH 0066/1463] IGNITE-8255 Possible name collisions in WorkersRegistry. Signed-off-by: Andrey Gura --- .../org/apache/ignite/internal/worker/WorkersRegistry.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersRegistry.java b/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersRegistry.java index e8d46fb2ebddb..16676c8ffa93a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersRegistry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersRegistry.java @@ -36,7 +36,7 @@ public class WorkersRegistry implements GridWorkerListener { * @param w Worker. */ public void register(GridWorker w) { - if (registeredWorkers.putIfAbsent(w.name(), w) != null) + if (registeredWorkers.putIfAbsent(w.runner().getName(), w) != null) throw new IllegalStateException("Worker is already registered [worker=" + w + ']'); } @@ -75,6 +75,6 @@ public GridWorker worker(String name) { /** {@inheritDoc} */ @Override public void onStopped(GridWorker w) { - unregister(w.name()); + unregister(w.runner().getName()); } } From b762d681b97ea121a8321eb66bf02f89a1d177cd Mon Sep 17 00:00:00 2001 From: Aleksey Plekhanov Date: Tue, 17 Apr 2018 15:56:36 +0300 Subject: [PATCH 0067/1463] IGNITE-8166 PME hangs when error occurs during checkpoint Signed-off-by: Andrey Gura --- .../persistence/GridCacheDatabaseSharedManager.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) 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 5beaafc585c19..16d32924da847 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 @@ -3606,7 +3606,14 @@ private static class CheckpointProgress { private GridFutureAdapter cpBeginFut = new GridFutureAdapter<>(); /** */ - private GridFutureAdapter cpFinishFut = new GridFutureAdapter<>(); + private GridFutureAdapter cpFinishFut = new GridFutureAdapter() { + @Override protected boolean onDone(@Nullable Void res, @Nullable Throwable err, boolean cancel) { + if (err != null && !cpBeginFut.isDone()) + cpBeginFut.onDone(err); + + return super.onDone(res, err, cancel); + } + }; /** */ private volatile boolean nextSnapshot; From 8428b0e63e97c10277f6d9e1640e16528a772270 Mon Sep 17 00:00:00 2001 From: Ivan Daschinskiy Date: Tue, 17 Apr 2018 18:05:42 +0300 Subject: [PATCH 0068/1463] IGNITE-8021 Delete cache config files when cache is destroyed - Fixes #3697. Signed-off-by: Alexey Goncharuk --- .../pagemem/store/IgnitePageStoreManager.java | 9 + .../processors/cache/GridCacheProcessor.java | 11 + .../file/FilePageStoreManager.java | 47 +++ ...onfigurationDataAfterDestroyCacheTest.java | 326 ++++++++++++++++++ .../pagemem/NoOpPageStoreManager.java | 5 + .../ignite/testsuites/IgnitePdsTestSuite.java | 2 + 6 files changed, 400 insertions(+) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDeleteCacheConfigurationDataAfterDestroyCacheTest.java 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 1b46bf990c540..0fc9f94b41978 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 @@ -193,6 +193,15 @@ public void initializeForCache(CacheGroupDescriptor grpDesc, StoredCacheData cac * @throws IgniteCheckedException If failed. */ public void storeCacheData(StoredCacheData cacheData, boolean overwrite) throws IgniteCheckedException; + + /** + * Remove cache configuration data file. + * + * @param cacheData Cache configuration. + * @throws IgniteCheckedException If failed. + */ + public void removeCacheData(StoredCacheData cacheData) throws IgniteCheckedException; + /** * @param grpId Cache group ID. * @return {@code True} if index store for given cache group existed before node started. 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 36edd72dd952f..bceb8c70dd235 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 @@ -1284,6 +1284,17 @@ private void stopCache(GridCacheAdapter cache, boolean cancel, boolean des U.stopLifecycleAware(log, lifecycleAwares(ctx.group(), cache.configuration(), ctx.store().configuredStore())); + IgnitePageStoreManager pageStore; + + if (destroy && (pageStore = sharedCtx.pageStore()) != null) { + try { + pageStore.removeCacheData(new StoredCacheData(ctx.config())); + } catch (IgniteCheckedException e) { + U.error(log, "Failed to delete cache configuration data while destroying cache" + + "[cache=" + ctx.name() + "]", e); + } + } + if (log.isInfoEnabled()) { if (ctx.group().sharedGroup()) log.info("Stopped cache [cacheName=" + cache.name() + ", group=" + ctx.group().name() + ']'); 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 6313eac008195..837f3d01b2b2c 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 @@ -26,6 +26,7 @@ import java.io.InputStream; import java.io.OutputStream; import java.nio.ByteBuffer; +import java.nio.file.DirectoryStream; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardCopyOption; @@ -281,6 +282,9 @@ public FilePageStoreManager(GridKernalContext ctx) { IgniteCheckedException ex = shutdown(old, /*clean files if destroy*/destroy, null); + if (destroy) + removeCacheGroupConfigurationData(grp); + if (ex != null) throw ex; } @@ -745,6 +749,49 @@ private IgniteCheckedException shutdown(CacheStoreHolder holder, boolean cleanFi return aggr; } + /** + * Delete caches' configuration data files of cache group. + * + * @param ctx Cache group context. + * @throws IgniteCheckedException If fails. + */ + private void removeCacheGroupConfigurationData(CacheGroupContext ctx) throws IgniteCheckedException { + File cacheGrpDir = cacheWorkDir(ctx.sharedGroup(), ctx.cacheOrGroupName()); + + if (cacheGrpDir != null && cacheGrpDir.exists()) { + DirectoryStream.Filter cacheCfgFileFilter = new DirectoryStream.Filter() { + @Override public boolean accept(Path path) { + return Files.isRegularFile(path) && path.getFileName().toString().endsWith(CACHE_DATA_FILENAME); + } + }; + + try (DirectoryStream dirStream = Files.newDirectoryStream(cacheGrpDir.toPath(), cacheCfgFileFilter)) { + for(Path path: dirStream) + Files.deleteIfExists(path); + } + catch (IOException e) { + throw new IgniteCheckedException("Failed to delete cache configurations of group: " + ctx.toString(), e); + } + } + } + + /** {@inheritDoc} */ + @Override public void removeCacheData(StoredCacheData cacheData) throws IgniteCheckedException { + CacheConfiguration cacheCfg = cacheData.config(); + File cacheWorkDir = cacheWorkDir(cacheCfg); + File file; + + if (cacheData.config().getGroupName() != null) + file = new File(cacheWorkDir, cacheCfg.getName() + CACHE_DATA_FILENAME); + else + file = new File(cacheWorkDir, CACHE_DATA_FILENAME); + + if (file.exists()) { + if (!file.delete()) + throw new IgniteCheckedException("Failed to delete cache configuration:" + cacheCfg.getName()); + } + } + /** * @param store Store to shutdown. * @param cleanFile {@code True} if files should be cleaned. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDeleteCacheConfigurationDataAfterDestroyCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDeleteCacheConfigurationDataAfterDestroyCacheTest.java new file mode 100644 index 0000000000000..d2767d465506b --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDeleteCacheConfigurationDataAfterDestroyCacheTest.java @@ -0,0 +1,326 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.util.ArrayList; +import java.util.List; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.cache.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.processors.cache.GatewayProtectedCacheProxy; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Test correct clean up cache configuration data after destroying cache. + */ +public class IgnitePdsDeleteCacheConfigurationDataAfterDestroyCacheTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final int CACHES = 3; + + /** */ + private static final int NODES = 3; + + /** */ + private static final int NUM_OF_KEYS = 100; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + return cfg.setDiscoverySpi(new TcpDiscoverySpi() + .setIpFinder(IP_FINDER)) + .setDataStorageConfiguration(new DataStorageConfiguration() + .setDefaultDataRegionConfiguration(new DataRegionConfiguration() + .setMaxSize(200 * 1024 * 1024) + .setPersistenceEnabled(true))); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + + super.afterTest(); + } + + /** + * {@inheritDoc} + * @returns always {@code true} in order to be able to kill nodes when checkpointer thread hangs. + */ + @Override protected boolean isMultiJvm() { + return true; + } + + /** + * Test destroy non grouped caches. + * + * @throws Exception If failed. + */ + public void testDestroyCaches() throws Exception { + Ignite ignite = startGrids(NODES); + + ignite.cluster().active(true); + + startCachesDynamically(ignite); + + checkDestroyCaches(ignite); + } + + /** + * Test destroy grouped caches. + * + * @throws Exception If failed. + */ + public void testDestroyGroupCaches() throws Exception { + Ignite ignite = startGrids(NODES); + + ignite.cluster().active(true); + + startGroupCachesDynamically(ignite); + + checkDestroyCaches(ignite); + } + + /** + * Test destroy caches with disabled checkpoints. + * + * @throws Exception If failed. + */ + public void testDestroyCachesAbruptlyWithoutCheckpoints() throws Exception { + Ignite ignite = startGrids(NODES); + + ignite.cluster().active(true); + + startCachesDynamically(ignite); + + enableCheckpoints(false); + + checkDestroyCachesAbruptly(ignite); + } + + /** + * Test destroy group caches with disabled checkpoints. + * + * @throws Exception If failed. + */ + public void testDestroyGroupCachesAbruptlyWithoutCheckpoints() throws Exception { + Ignite ignite = startGrids(NODES); + + ignite.cluster().active(true); + + startGroupCachesDynamically(ignite); + + enableCheckpoints(false); + + checkDestroyCachesAbruptly(ignite); + } + + /** + * Test destroy caches abruptly with checkpoints. + * + * @throws Exception If failed. + */ + public void testDestroyCachesAbruptly() throws Exception { + Ignite ignite = startGrids(NODES); + + ignite.cluster().active(true); + + startCachesDynamically(ignite); + + checkDestroyCachesAbruptly(ignite); + } + + + /** + * Test destroy group caches abruptly with checkpoints. + * + * @throws Exception If failed. + */ + public void testDestroyGroupCachesAbruptly() throws Exception { + Ignite ignite = startGrids(NODES); + + ignite.cluster().active(true); + + startGroupCachesDynamically(ignite); + + checkDestroyCachesAbruptly(ignite); + } + + /** + * @param ignite Ignite. + */ + private void loadCaches(Ignite ignite) { + for (int i = 0; i < CACHES; i++) { + try (IgniteDataStreamer s = ignite.dataStreamer(cacheName(i))) { + s.allowOverwrite(true); + + for (int j = 0; j < NUM_OF_KEYS; j++) + s.addData(j, "cache: " + i + " data: " + j); + + s.flush(); + } + } + } + + /** + * @param ignite Ignite. + */ + private void checkDestroyCaches(Ignite ignite) throws Exception { + loadCaches(ignite); + + log.warning("destroying caches...."); + + ignite.cache(cacheName(0)).destroy(); + ignite.cache(cacheName(1)).destroy(); + + assertEquals(CACHES - 2, ignite.cacheNames().size()); + + log.warning("Stopping grid"); + + stopAllGrids(); + + log.warning("Grid stopped"); + + log.warning("Starting grid"); + + ignite = startGrids(NODES); + + log.warning("Grid started"); + + assertEquals("Check that caches don't survived", CACHES - 2, ignite.cacheNames().size()); + + for(Ignite ig: G.allGrids()) { + IgniteCache cache = ig.cache(cacheName(2)); + + for (int j = 0; j < NUM_OF_KEYS; j++) + assertNotNull("Check that cache2 contains key: " + j + " node: " + ignite.name(), cache.get(j)); + } + } + + + /** + * @param ignite Ignite instance. + */ + private void checkDestroyCachesAbruptly(Ignite ignite) throws Exception { + loadCaches(ignite); + + log.warning("Destroying caches"); + + ((GatewayProtectedCacheProxy)ignite.cache(cacheName(0))).destroyAsync(); + ((GatewayProtectedCacheProxy)ignite.cache(cacheName(1))).destroyAsync(); + + log.warning("Stopping grid"); + + stopAllGrids(); + + log.warning("Grid stopped"); + + log.warning("Starting grid"); + + startGrids(NODES); + + log.warning("Grid started"); + + for(Ignite ig: G.allGrids()) { + assertTrue(ig.cacheNames().contains(cacheName(2))); + + IgniteCache cache = ig.cache(cacheName(2)); + + for (int j = 0; j < NUM_OF_KEYS; j++) + assertNotNull("Check that survived cache cache2 contains key: " + j + " node: " + ig.name(), cache.get(j)); + } + } + + /** + * @param ignite Ignite. + */ + private void startCachesDynamically(Ignite ignite) { + List ccfg = new ArrayList<>(CACHES); + + for (int i = 0; i < CACHES; i++) + ccfg.add(new CacheConfiguration<>(cacheName(i)) + .setBackups(1) + .setAffinity(new RendezvousAffinityFunction(false, 32))); + + ignite.createCaches(ccfg); + } + + /** + * @param ignite Ignite instance. + */ + private void startGroupCachesDynamically(Ignite ignite) { + List ccfg = new ArrayList<>(CACHES); + + for (int i = 0; i < CACHES; i++) + ccfg.add(new CacheConfiguration<>(cacheName(i)) + .setGroupName(i % 2 == 0 ? "grp-even" : "grp-odd") + .setBackups(1) + .setAffinity(new RendezvousAffinityFunction(false, 32))); + + ignite.createCaches(ccfg); + } + + + /** + * Generate cache name from idx. + * + * @param idx Index. + */ + private String cacheName(int idx) { + return "cache" + idx; + } + + /** + * Enable/disable checkpoints on multi JVM nodes only. + * + * @param enabled Enabled flag. + * @throws IgniteCheckedException If failed. + */ + private void enableCheckpoints(boolean enabled) throws IgniteCheckedException { + for (Ignite ignite : G.allGrids()) { + assert !ignite.cluster().localNode().isClient(); + + GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager)((IgniteEx)ignite).context() + .cache().context().database(); + + dbMgr.enableCheckpoints(enabled).get(); + } + } +} 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 64acf0211ddcf..be40c90f7c15c 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 @@ -186,6 +186,11 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager { // No-op. } + /** {@inheritDoc} */ + @Override public void removeCacheData(StoredCacheData cacheData) throws IgniteCheckedException { + // No-op. + } + /** {@inheritDoc} */ @Override public boolean hasIndexStore(int grpId) { return false; 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 a9668e7fc5852..af0b7adc1e68b 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 @@ -19,6 +19,7 @@ import junit.framework.TestSuite; import org.apache.ignite.internal.processors.cache.IgniteClusterActivateDeactivateTestWithPersistence; +import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsDeleteCacheConfigurationDataAfterDestroyCacheTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsDynamicCacheTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsSingleNodePutGetPersistenceTest; import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsCacheRestoreTest; @@ -113,6 +114,7 @@ public static void addRealPageStoreTests(TestSuite suite) { suite.addTestSuite(IgnitePdsCacheRestoreTest.class); suite.addTestSuite(IgnitePdsDataRegionMetricsTest.class); + suite.addTestSuite(IgnitePdsDeleteCacheConfigurationDataAfterDestroyCacheTest.class); suite.addTestSuite(DefaultPageSizeBackwardsCompatibilityTest.class); From cd59c8e64f05ca03c7da8dc35d027a14fcebf250 Mon Sep 17 00:00:00 2001 From: Aleksey Plekhanov Date: Tue, 17 Apr 2018 18:27:53 +0300 Subject: [PATCH 0069/1463] IGNITE-8033 Fixed flaky failure of TxOptimisticDeadlockDetectionCrossCacheTest Signed-off-by: Andrey Gura --- ...misticDeadlockDetectionCrossCacheTest.java | 147 ++++++------------ 1 file changed, 50 insertions(+), 97 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionCrossCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionCrossCacheTest.java index 5d1374c0918d8..056b093f7fba6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionCrossCacheTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionCrossCacheTest.java @@ -18,30 +18,21 @@ package org.apache.ignite.internal.processors.cache.transactions; import java.util.Collection; -import java.util.Set; -import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.CountDownLatch; 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.IgniteException; import org.apache.ignite.cache.CacheMode; -import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.events.Event; +import org.apache.ignite.events.EventType; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.IgniteKernal; -import org.apache.ignite.internal.managers.communication.GridIoMessage; -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.version.GridCacheVersion; -import org.apache.ignite.internal.util.GridConcurrentHashSet; -import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.lang.IgniteInClosure; -import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.spi.IgniteSpiException; -import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -57,9 +48,6 @@ * */ public class TxOptimisticDeadlockDetectionCrossCacheTest extends GridCommonAbstractTest { - /** Nodes count. */ - private static final int NODES_CNT = 2; - /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { @@ -73,10 +61,6 @@ public class TxOptimisticDeadlockDetectionCrossCacheTest extends GridCommonAbstr cfg.setDiscoverySpi(discoSpi); } - TcpCommunicationSpi commSpi = new TestCommunicationSpi(); - - cfg.setCommunicationSpi(commSpi); - CacheConfiguration ccfg0 = defaultCacheConfiguration(); ccfg0.setName("cache0"); @@ -96,42 +80,46 @@ public class TxOptimisticDeadlockDetectionCrossCacheTest extends GridCommonAbstr return cfg; } - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - super.beforeTestsStarted(); - - startGrids(NODES_CNT); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() throws Exception { - super.afterTestsStopped(); - - stopAllGrids(); - } - /** * @throws Exception If failed. */ public void testDeadlock() throws Exception { - // Sometimes boh transactions perform commit, so we repeat attempt. - while (!doTestDeadlock()) {} + startGrids(2); + + try { + doTestDeadlock(); + } + finally { + stopAllGrids(); + } } /** * @throws Exception If failed. */ private boolean doTestDeadlock() throws Exception { - TestCommunicationSpi.init(2); - - final CyclicBarrier barrier = new CyclicBarrier(2); - final AtomicInteger threadCnt = new AtomicInteger(); final AtomicBoolean deadlock = new AtomicBoolean(); final AtomicInteger commitCnt = new AtomicInteger(); + grid(0).events().localListen(new CacheLocksListener(), EventType.EVT_CACHE_OBJECT_LOCKED); + + AffinityTopologyVersion waitTopVer = new AffinityTopologyVersion(2, 1); + + IgniteInternalFuture exchFut = grid(0).context().cache().context().exchange().affinityReadyFuture(waitTopVer); + + if (exchFut != null && !exchFut.isDone()) { + log.info("Waiting for topology exchange future [waitTopVer=" + waitTopVer + ", curTopVer=" + + grid(0).context().cache().context().exchange().readyAffinityVersion() + ']'); + + exchFut.get(); + } + + log.info("Finished topology exchange future [curTopVer=" + + grid(0).context().cache().context().exchange().readyAffinityVersion() + ']'); + IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Runnable() { @Override public void run() { int threadNum = threadCnt.getAndIncrement(); @@ -152,8 +140,6 @@ private boolean doTestDeadlock() throws Exception { cache1.put(key1, 0); - barrier.await(); - int key2 = primaryKey(cache2); log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() + @@ -171,23 +157,23 @@ private boolean doTestDeadlock() throws Exception { hasCause(e, TransactionDeadlockException.class) ) { if (deadlock.compareAndSet(false, true)) - U.error(log, "At least one stack trace should contain " + - TransactionDeadlockException.class.getSimpleName(), e); + log.info("Successfully set deadlock flag"); + else + log.info("Deadlock flag was already set"); } + else + log.warning("Got not deadlock exception", e); } } }, 2, "tx-thread"); fut.get(); - if (commitCnt.get() == 2) - return false; + assertFalse("Commits must fail", commitCnt.get() == 2); assertTrue(deadlock.get()); - for (int i = 0; i < NODES_CNT ; i++) { - Ignite ignite = ignite(i); - + for (Ignite ignite : G.allGrids()) { IgniteTxManager txMgr = ((IgniteKernal)ignite).context().cache().context().tm(); Collection> futs = txMgr.deadlockDetectionFutures(); @@ -199,59 +185,26 @@ private boolean doTestDeadlock() throws Exception { } /** + * Listener for cache lock events. * + * To ensure deadlock this listener blocks transaction thread until both threads acquire first lock. */ - private static class TestCommunicationSpi extends TcpCommunicationSpi { - /** Tx count. */ - private static volatile int TX_CNT; - - /** Tx ids. */ - private static final Set TX_IDS = new GridConcurrentHashSet<>(); - - /** - * @param txCnt Tx count. - */ - private static void init(int txCnt) { - TX_CNT = txCnt; - TX_IDS.clear(); - } + private static class CacheLocksListener implements IgnitePredicate { + /** Latch. */ + private final CountDownLatch latch = new CountDownLatch(2); /** {@inheritDoc} */ - @Override public void sendMessage( - final ClusterNode node, - final Message msg, - final IgniteInClosure ackC - ) throws IgniteSpiException { - if (msg instanceof GridIoMessage) { - Message msg0 = ((GridIoMessage)msg).message(); - - if (msg0 instanceof GridNearTxPrepareRequest) { - final GridNearTxPrepareRequest req = (GridNearTxPrepareRequest)msg0; - - GridCacheVersion txId = req.version(); - - if (TX_IDS.contains(txId)) { - while (TX_IDS.size() < TX_CNT) { - try { - U.sleep(50); - } - catch (IgniteInterruptedCheckedException e) { - e.printStackTrace(); - } - } - } - } - else if (msg0 instanceof GridNearTxPrepareResponse) { - GridNearTxPrepareResponse res = (GridNearTxPrepareResponse)msg0; + @Override public boolean apply(Event evt) { + latch.countDown(); - GridCacheVersion txId = res.version(); - - TX_IDS.add(txId); - } + try { + latch.await(); + } + catch (InterruptedException e) { + e.printStackTrace(); } - super.sendMessage(node, msg, ackC); + return true; } } - } From acfef907db8204ac93fc235770f36bf7f61269c3 Mon Sep 17 00:00:00 2001 From: Ilya Kasnacheev Date: Tue, 17 Apr 2018 19:50:51 +0300 Subject: [PATCH 0070/1463] IGNITE-2766 Fix .net test. - Fixes #3853. Signed-off-by: dpavlov (cherry picked from commit 96cb795) --- .../dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs index 7e6222f026f6c..274439e4e3be6 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs @@ -98,10 +98,8 @@ public void TestClusterRestart() cache1[1] = 2; Assert.AreEqual(2, cache1[1]); - // Check that old cache instance does not work. - var cacheEx1 = Assert.Throws(() => cache.Get(1)); - Assert.IsTrue(cacheEx1.Message.EndsWith("Failed to perform cache operation (cache is stopped): cache"), - cacheEx1.Message); + // Check that old cache instance still works. + Assert.AreEqual(2, cache.Get(1)); } /// From 6cea78e4e13fe43555b78dcd683366f54c6816ff Mon Sep 17 00:00:00 2001 From: Andrey Kuznetsov Date: Tue, 17 Apr 2018 19:58:43 +0300 Subject: [PATCH 0071/1463] IGNITE-7770 Test testRandomMixedTxConfigurations partialy fixed Signed-off-by: Andrey Gura --- .../processors/cache/GridCacheAdapter.java | 3 +- ...OptimisticSerializableTxPrepareFuture.java | 2 +- .../GridNearOptimisticTxPrepareFuture.java | 2 +- .../GridNearPessimisticTxPrepareFuture.java | 2 +- .../near/GridNearTxFastFinishFuture.java | 10 +++- .../distributed/near/GridNearTxLocal.java | 57 ++++++++++++------- .../cache/transactions/IgniteTxManager.java | 5 +- .../transactions/TxRollbackOnTimeoutTest.java | 2 + 8 files changed, 55 insertions(+), 28 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java index c2d0f427fc917..bd613a191f407 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 @@ -104,6 +104,7 @@ import org.apache.ignite.internal.processors.task.GridInternal; 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.future.GridEmbeddedFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -4283,7 +4284,7 @@ public T applyx(IgniteInternalFuture tFut) throws IgniteCheckedException { } catch (IgniteCheckedException e1) { try { - tx0.rollbackNearTxLocalAsync(); + tx0.rollbackNearTxLocalAsync(e1 instanceof IgniteTxTimeoutCheckedException); } catch (Throwable e2) { if (e2 != e1) 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 beb1e160c7ad6..3e2c84a740af5 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 @@ -294,7 +294,7 @@ private boolean onComplete() { boolean txStateCheck = remap ? tx.state() == PREPARING : tx.state(PREPARING); if (!txStateCheck) { - if (tx.setRollbackOnly()) { + if (tx.isRollbackOnly() || tx.setRollbackOnly()) { if (tx.timedOut()) onError(null, new IgniteTxTimeoutCheckedException("Transaction timed out and " + "was rolled back: " + this)); 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 8d8c0b28b8b5f..2afb09601ebac 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 @@ -327,7 +327,7 @@ private boolean onComplete() { boolean txStateCheck = remap ? tx.state() == PREPARING : tx.state(PREPARING); if (!txStateCheck) { - if (tx.setRollbackOnly()) { + if (tx.isRollbackOnly() || tx.setRollbackOnly()) { if (tx.remainingTime() == -1) onError(new IgniteTxTimeoutCheckedException("Transaction timed out and " + "was rolled back: " + this), false); 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 4ab6863ad03b2..54ae85c539cc7 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 @@ -155,7 +155,7 @@ private MiniFuture miniFuture(int miniId) { /** {@inheritDoc} */ @Override public void prepare() { if (!tx.state(PREPARING)) { - if (tx.setRollbackOnly()) { + if (tx.isRollbackOnly() || tx.setRollbackOnly()) { if (tx.remainingTime() == -1) onDone(new IgniteTxTimeoutCheckedException("Transaction timed out and was rolled back: " + tx)); else 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 72226973009c5..95e4deded214f 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 @@ -37,13 +37,17 @@ public class GridNearTxFastFinishFuture extends GridFutureAdapter 0 && !implicit()) - trackTimeout = cctx.time().addTimeoutObject(this); + trackTimeout = timeout() > 0 && !implicit() && cctx.time().addTimeoutObject(this); } /** {@inheritDoc} */ @@ -3155,8 +3155,13 @@ public IgniteInternalFuture prepareNearTxLocal() { if (!PREP_FUT_UPD.compareAndSet(this, null, fut)) return prepFut; - if (trackTimeout) - removeTimeoutHandler(); + if (trackTimeout) { + prepFut.listen(new IgniteInClosure>() { + @Override public void apply(IgniteInternalFuture f) { + GridNearTxLocal.this.removeTimeoutHandler(); + } + }); + } if (timeout == -1) { fut.onDone(this, timeoutException()); @@ -3215,7 +3220,7 @@ public IgniteInternalFuture commitNearTxLocalAsync() { if (fastFinish()) { GridNearTxFastFinishFuture fut0; - if (!FINISH_FUT_UPD.compareAndSet(this, null, fut0 = new GridNearTxFastFinishFuture(this, true))) + if (!FINISH_FUT_UPD.compareAndSet(this, null, fut0 = new GridNearTxFastFinishFuture(this, true, false))) return chainFinishFuture(finishFut, true); fut0.finish(); @@ -3238,7 +3243,9 @@ public IgniteInternalFuture commitNearTxLocalAsync() { // Make sure that here are no exceptions. prepareFut.get(); - fut0.finish(true, true); + TransactionState state = state(); + + fut0.finish(state == PREPARED || state == COMMITTING || state == COMMITTED, true); } catch (Error | RuntimeException e) { COMMIT_ERR_UPD.compareAndSet(GridNearTxLocal.this, null, e); @@ -3282,7 +3289,7 @@ public IgniteInternalFuture rollbackNearTxLocalAsync() { * @param onTimeout {@code True} if rolled back asynchronously on timeout. * @return Rollback future. */ - private IgniteInternalFuture rollbackNearTxLocalAsync(final boolean onTimeout) { + public IgniteInternalFuture rollbackNearTxLocalAsync(final boolean onTimeout) { if (log.isDebugEnabled()) log.debug("Rolling back near tx: " + this); @@ -3292,13 +3299,13 @@ private IgniteInternalFuture rollbackNearTxLocalAsync(final bo NearTxFinishFuture fut = finishFut; if (fut != null) - return chainFinishFuture(finishFut, false); + return chainFinishFuture(finishFut, false, !onTimeout); if (fastFinish()) { GridNearTxFastFinishFuture fut0; - if (!FINISH_FUT_UPD.compareAndSet(this, null, fut0 = new GridNearTxFastFinishFuture(this, false))) - return chainFinishFuture(finishFut, false); + if (!FINISH_FUT_UPD.compareAndSet(this, null, fut0 = new GridNearTxFastFinishFuture(this, false, onTimeout))) + return chainFinishFuture(finishFut, false, !onTimeout); fut0.finish(); @@ -3308,7 +3315,7 @@ private IgniteInternalFuture rollbackNearTxLocalAsync(final bo final GridNearTxFinishFuture fut0; if (!FINISH_FUT_UPD.compareAndSet(this, null, fut0 = new GridNearTxFinishFuture<>(cctx, this, false))) - return chainFinishFuture(finishFut, false); + return chainFinishFuture(finishFut, false, !onTimeout); cctx.mvcc().addFuture(fut0, fut0.futureId()); @@ -3327,7 +3334,7 @@ private IgniteInternalFuture rollbackNearTxLocalAsync(final bo fut0.finish(false, !onTimeout); } - else { + else if (!onTimeout) { prepFut.listen(new CI1>() { @Override public void apply(IgniteInternalFuture f) { try { @@ -3339,10 +3346,11 @@ private IgniteInternalFuture rollbackNearTxLocalAsync(final bo log.debug("Got optimistic tx failure [tx=" + this + ", err=" + e + ']'); } - fut0.finish(false, !onTimeout); + fut0.finish(false, true); } }); - } + } else + fut0.finish(false, false); return fut0; } @@ -3352,12 +3360,17 @@ private IgniteInternalFuture rollbackNearTxLocalAsync(final bo return rollbackNearTxLocalAsync(); } + /** */ + private IgniteInternalFuture chainFinishFuture(final NearTxFinishFuture fut, final boolean commit) { + return chainFinishFuture(fut, commit, true); + } + /** * @param fut Already started finish future. * @param commit Commit flag. * @return Finish future. */ - private IgniteInternalFuture chainFinishFuture(final NearTxFinishFuture fut, final boolean commit) { + private IgniteInternalFuture chainFinishFuture(final NearTxFinishFuture fut, final boolean commit, final boolean clearThreadMap) { assert fut != null; if (fut.commit() != commit) { @@ -3381,7 +3394,7 @@ private IgniteInternalFuture chainFinishFuture(final NearTxFin if (!cctx.mvcc().addFuture(rollbackFut, rollbackFut.futureId())) return; - rollbackFut.finish(false, true); + rollbackFut.finish(false, clearThreadMap); } } } @@ -4161,7 +4174,13 @@ public boolean addTimeoutHandler() { } } - if (state(MARKED_ROLLBACK, true) || (state() == MARKED_ROLLBACK)) { + boolean proceed; + + synchronized (this) { + proceed = state() != PREPARED && state(MARKED_ROLLBACK, true); + } + + if (proceed || (state() == MARKED_ROLLBACK)) { cctx.kernalContext().closure().runLocalSafe(new Runnable() { @Override public void run() { // Note: if rollback asynchronously on timeout should not clear thread map 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 9fb87770df64d..7fa31bf027cbc 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 @@ -1356,7 +1356,7 @@ else if (log.isDebugEnabled()) * @param tx Transaction to finish. * @param commit {@code True} if transaction is committed, {@code false} if rolled back. */ - public void fastFinishTx(GridNearTxLocal tx, boolean commit) { + public void fastFinishTx(GridNearTxLocal tx, boolean commit, boolean clearThreadMap) { assert tx != null; assert tx.writeMap().isEmpty(); assert tx.optimistic() || tx.readMap().isEmpty(); @@ -1377,7 +1377,8 @@ public void fastFinishTx(GridNearTxLocal tx, boolean commit) { removeObsolete(tx); // 4. Remove from per-thread storage. - clearThreadMap(tx); + if (clearThreadMap) + clearThreadMap(tx); // 5. Clear context. resetContext(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackOnTimeoutTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackOnTimeoutTest.java index c5be40ee414ee..97de81edc781d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackOnTimeoutTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxRollbackOnTimeoutTest.java @@ -439,6 +439,8 @@ public void testRandomMixedTxConfigurations() throws Exception { if (delay > 0) sleep(delay); + assert v != null; + node.cache(CACHE_NAME).put(k, v + 1); tx.commit(); From e394693a7389b4daff328827abdb1dcd28783f66 Mon Sep 17 00:00:00 2001 From: Maxim Muzafarov Date: Tue, 17 Apr 2018 21:18:36 +0300 Subject: [PATCH 0072/1463] IGNITE-8301 testReconnectCacheDestroyedAndCreated should excpect recreated client cache - Fixes #3856. Signed-off-by: dpavlov (cherry picked from commit 56be24b) --- .../ignite/internal/IgniteClientReconnectCacheTest.java | 6 ------ 1 file changed, 6 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 3cb82e07cab14..ec5eab2561b63 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 @@ -894,12 +894,6 @@ public void testReconnectCacheDestroyedAndCreated() throws Exception { } }); - GridTestUtils.assertThrows(log, new Callable() { - @Override public Object call() throws Exception { - return clientCache.get(1); - } - }, IllegalStateException.class, null); - checkCacheDiscoveryData(srv, client, DEFAULT_CACHE_NAME, true, false, false); IgniteCache clientCache0 = client.cache(DEFAULT_CACHE_NAME); From 4685ebe5f5dda4023980398806e222fada895e26 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Wed, 18 Apr 2018 10:44:44 +0700 Subject: [PATCH 0073/1463] IGNITE-8140 Web Console: Fixed code generation for large numbers in configuration params. (cherry picked from commit eda5fe7) --- .../generator/ConfigurationGenerator.js | 106 +++++++++--------- 1 file changed, 53 insertions(+), 53 deletions(-) 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 45d9ad10aacdf..c5f82d3b80841 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js @@ -720,8 +720,8 @@ export default class IgniteConfigurationGenerator { .intProperty('connectionTimeout') .intProperty('requestTimeout') .stringProperty('signerOverride') - .intProperty('connectionTTL') - .intProperty('connectionMaxIdleMillis') + .longProperty('connectionTTL') + .longProperty('connectionMaxIdleMillis') .emptyBeanProperty('dnsResolver') .intProperty('responseMetadataCacheSize') .emptyBeanProperty('secureRandom') @@ -790,7 +790,7 @@ export default class IgniteConfigurationGenerator { if (!available('2.3.0')) return cfg; - cfg.intProperty('longQueryWarningTimeout'); + cfg.longProperty('longQueryWarningTimeout'); if (_.get(cluster, 'clientConnectorConfiguration.enabled') !== true) return cfg; @@ -837,7 +837,7 @@ export default class IgniteConfigurationGenerator { colSpi.intProperty('activeJobsThreshold') .intProperty('waitJobsThreshold') - .intProperty('messageExpireTime') + .longProperty('messageExpireTime') .intProperty('maximumStealingAttempts') .boolProperty('stealingEnabled') .emptyBeanProperty('externalCollisionListener') @@ -892,9 +892,9 @@ export default class IgniteConfigurationGenerator { .intProperty('sharedMemoryPort') .intProperty('directBuffer') .intProperty('directSendBuffer') - .intProperty('idleConnectionTimeout') - .intProperty('connectTimeout') - .intProperty('maxConnectTimeout') + .longProperty('idleConnectionTimeout') + .longProperty('connectTimeout') + .longProperty('maxConnectTimeout') .intProperty('reconnectCount') .intProperty('socketSendBuffer') .intProperty('socketReceiveBuffer') @@ -903,19 +903,19 @@ export default class IgniteConfigurationGenerator { .intProperty('tcpNoDelay') .intProperty('ackSendThreshold') .intProperty('unacknowledgedMessagesBufferSize') - .intProperty('socketWriteTimeout') + .longProperty('socketWriteTimeout') .intProperty('selectorsCount') .emptyBeanProperty('addressResolver'); if (commSpi.nonEmpty()) cfg.beanProperty('communicationSpi', commSpi); - cfg.intProperty('networkTimeout') - .intProperty('networkSendRetryDelay') + cfg.longProperty('networkTimeout') + .longProperty('networkSendRetryDelay') .intProperty('networkSendRetryCount'); if (available(['1.0.0', '2.3.0'])) - cfg.intProperty('discoveryStartupDelay'); + cfg.longProperty('discoveryStartupDelay'); return cfg; } @@ -930,9 +930,9 @@ export default class IgniteConfigurationGenerator { .stringProperty('host') .intProperty('port') .intProperty('portRange') - .intProperty('idleTimeout') - .intProperty('idleQueryCursorTimeout') - .intProperty('idleQueryCursorCheckFrequency') + .longProperty('idleTimeout') + .longProperty('idleQueryCursorTimeout') + .longProperty('idleQueryCursorCheckFrequency') .intProperty('receiveBufferSize') .intProperty('sendBufferSize') .intProperty('sendQueueLimit') @@ -1020,11 +1020,11 @@ export default class IgniteConfigurationGenerator { .intProperty('localPort') .intProperty('localPortRange') .emptyBeanProperty('addressResolver') - .intProperty('socketTimeout') - .intProperty('ackTimeout') - .intProperty('maxAckTimeout') - .intProperty('networkTimeout') - .intProperty('joinTimeout') + .longProperty('socketTimeout') + .longProperty('ackTimeout') + .longProperty('maxAckTimeout') + .longProperty('networkTimeout') + .longProperty('joinTimeout') .intProperty('threadPriority'); // Removed in ignite 2.0 @@ -1034,13 +1034,13 @@ export default class IgniteConfigurationGenerator { .intProperty('maxMissedClientHeartbeats'); } - discoSpi.intProperty('topHistorySize') + discoSpi.longProperty('topHistorySize') .emptyBeanProperty('listener') .emptyBeanProperty('dataExchange') .emptyBeanProperty('metricsProvider') .intProperty('reconnectCount') - .intProperty('statisticsPrintFrequency') - .intProperty('ipFinderCleanFrequency') + .longProperty('statisticsPrintFrequency') + .longProperty('ipFinderCleanFrequency') .emptyBeanProperty('authenticator') .intProperty('forceServerMode') .intProperty('clientReconnectDisabled'); @@ -1090,8 +1090,8 @@ export default class IgniteConfigurationGenerator { case 'Memory': eventStorageBean = new Bean('org.apache.ignite.spi.eventstorage.memory.MemoryEventStorageSpi', 'eventStorage', eventStorage.Memory, clusterDflts.eventStorage.Memory); - eventStorageBean.intProperty('expireAgeMs') - .intProperty('expireCount') + eventStorageBean.longProperty('expireAgeMs') + .longProperty('expireCount') .emptyBeanProperty('filter'); break; @@ -1128,8 +1128,8 @@ export default class IgniteConfigurationGenerator { // Since ignite 2.0 if (available('2.0.0')) { - cfg.intProperty('failureDetectionTimeout') - .intProperty('clientFailureDetectionTimeout'); + cfg.longProperty('failureDetectionTimeout') + .longProperty('clientFailureDetectionTimeout'); } _.forEach(cluster.failoverSpi, (spi) => { @@ -1210,7 +1210,7 @@ export default class IgniteConfigurationGenerator { if (plannerBean) hadoopBean.beanProperty('mapReducePlanner', plannerBean); - hadoopBean.intProperty('finishedJobInfoTtl') + hadoopBean.longProperty('finishedJobInfoTtl') .intProperty('maxParallelTasks') .intProperty('maxTaskQueueSize') .arrayProperty('nativeLibraryNames', 'nativeLibraryNames', _.get(hadoop, 'nativeLibraryNames')); @@ -1368,12 +1368,12 @@ export default class IgniteConfigurationGenerator { memoryBean.intProperty('pageSize') .intProperty('concurrencyLevel') - .intProperty('systemCacheInitialSize') - .intProperty('systemCacheMaxSize') + .longProperty('systemCacheInitialSize') + .longProperty('systemCacheMaxSize') .stringProperty('defaultMemoryPolicyName'); if (memoryBean.valueOf('defaultMemoryPolicyName') === 'default') - memoryBean.intProperty('defaultMemoryPolicySize'); + memoryBean.longProperty('defaultMemoryPolicySize'); const policies = []; @@ -1438,8 +1438,8 @@ export default class IgniteConfigurationGenerator { storageBean.intProperty('pageSize') .intProperty('concurrencyLevel') - .intProperty('systemRegionInitialSize') - .intProperty('systemRegionMaxSize'); + .longProperty('systemRegionInitialSize') + .longProperty('systemRegionMaxSize'); const dfltDataRegionCfg = this.dataRegionConfiguration(_.get(dataStorageCfg, 'defaultDataRegionConfiguration')); @@ -1461,7 +1461,7 @@ export default class IgniteConfigurationGenerator { storageBean.varArgProperty('dataRegionConfigurations', 'dataRegionConfigurations', dataRegionCfgs, 'org.apache.ignite.configuration.DataRegionConfiguration'); storageBean.stringProperty('storagePath') - .intProperty('checkpointFrequency') + .longProperty('checkpointFrequency') .intProperty('checkpointThreads') .enumProperty('checkpointWriteOrder') .enumProperty('walMode') @@ -1480,7 +1480,7 @@ export default class IgniteConfigurationGenerator { .longProperty('lockWaitTime') .intProperty('walThreadLocalBufferSize') .intProperty('metricsSubIntervalCount') - .intProperty('metricsRateTimeInterval') + .longProperty('metricsRateTimeInterval') .longProperty('walAutoArchiveAfterInactivity') .boolProperty('metricsEnabled') .boolProperty('alwaysWriteFullPages') @@ -1594,13 +1594,13 @@ export default class IgniteConfigurationGenerator { // Generate metrics group. static clusterMetrics(cluster, available, cfg = this.igniteConfigurationBean(cluster)) { - cfg.intProperty('metricsExpireTime') + cfg.longProperty('metricsExpireTime') .intProperty('metricsHistorySize') - .intProperty('metricsLogFrequency'); + .longProperty('metricsLogFrequency'); // Since ignite 2.0 if (available('2.0.0')) - cfg.intProperty('metricsUpdateFrequency'); + cfg.longProperty('metricsUpdateFrequency'); return cfg; } @@ -1630,7 +1630,7 @@ export default class IgniteConfigurationGenerator { if (!available('2.1.0')) return cfg; - cfg.intProperty('longQueryWarningTimeout'); + cfg.longProperty('longQueryWarningTimeout'); if (_.get(cluster, 'sqlConnectorConfiguration.enabled') !== true) return cfg; @@ -1663,7 +1663,7 @@ export default class IgniteConfigurationGenerator { bean.stringProperty('persistentStorePath') .boolProperty('metricsEnabled') .boolProperty('alwaysWriteFullPages') - .intProperty('checkpointingFrequency') + .longProperty('checkpointingFrequency') .longProperty('checkpointingPageBufferSize') .intProperty('checkpointingThreads') .stringProperty('walStorePath') @@ -1675,7 +1675,7 @@ export default class IgniteConfigurationGenerator { .longProperty('walFsyncDelayNanos') .intProperty('walRecordIteratorBufferSize') .longProperty('lockWaitTime') - .intProperty('rateTimeInterval') + .longProperty('rateTimeInterval') .intProperty('tlbSize') .intProperty('subIntervals'); @@ -1782,7 +1782,7 @@ export default class IgniteConfigurationGenerator { .intProperty('igfsThreadPoolSize') .intProperty('rebalanceThreadPoolSize') .intProperty('utilityCacheThreadPoolSize', 'utilityCachePoolSize') - .intProperty('utilityCacheKeepAliveTime') + .longProperty('utilityCacheKeepAliveTime') .intProperty('asyncCallbackPoolSize') .intProperty('stripedPoolSize'); @@ -1817,7 +1817,7 @@ export default class IgniteConfigurationGenerator { bean.enumProperty('defaultTxConcurrency') .enumProperty('defaultTxIsolation') - .intProperty('defaultTxTimeout') + .longProperty('defaultTxTimeout') .intProperty('pessimisticTxLogLinger') .intProperty('pessimisticTxLogSize') .boolProperty('txSerializableEnabled') @@ -2070,7 +2070,7 @@ export default class IgniteConfigurationGenerator { ccfg.enumProperty('memoryMode'); if (ccfg.valueOf('memoryMode') !== 'OFFHEAP_VALUES') - ccfg.intProperty('offHeapMaxMemory'); + ccfg.longProperty('offHeapMaxMemory'); } // Since ignite 2.0 @@ -2105,7 +2105,7 @@ export default class IgniteConfigurationGenerator { if (available(['1.0.0', '2.0.0'])) ccfg.intProperty('sqlOnheapRowCacheSize'); - ccfg.intProperty('longQueryWarningTimeout') + ccfg.longProperty('longQueryWarningTimeout') .arrayProperty('indexedTypes', 'indexedTypes', indexedTypes, 'java.lang.Class') .intProperty('queryDetailMetricsSize') .arrayProperty('sqlFunctionClasses', 'sqlFunctionClasses', cache.sqlFunctionClasses, 'java.lang.Class'); @@ -2225,7 +2225,7 @@ export default class IgniteConfigurationGenerator { ccfg.boolProperty('writeBehindEnabled') .intProperty('writeBehindBatchSize') .intProperty('writeBehindFlushSize') - .intProperty('writeBehindFlushFrequency') + .longProperty('writeBehindFlushFrequency') .intProperty('writeBehindFlushThreadCount'); // Since ignite 2.0 @@ -2239,7 +2239,7 @@ export default class IgniteConfigurationGenerator { // Generate cache concurrency control group. static cacheConcurrency(cache, available, ccfg = this.cacheConfigurationBean(cache)) { ccfg.intProperty('maxConcurrentAsyncOperations') - .intProperty('defaultLockTimeout'); + .longProperty('defaultLockTimeout'); // Removed in ignite 2.0 if (available(['1.0.0', '2.0.0'])) @@ -2297,11 +2297,11 @@ export default class IgniteConfigurationGenerator { ccfg.enumProperty('rebalanceMode') .intProperty('rebalanceThreadPoolSize') .intProperty('rebalanceBatchSize') - .intProperty('rebalanceBatchesPrefetchCount') + .longProperty('rebalanceBatchesPrefetchCount') .intProperty('rebalanceOrder') - .intProperty('rebalanceDelay') - .intProperty('rebalanceTimeout') - .intProperty('rebalanceThrottle'); + .longProperty('rebalanceDelay') + .longProperty('rebalanceTimeout') + .longProperty('rebalanceThrottle'); } if (ccfg.includes('igfsAffinnityGroupSize')) { @@ -2456,8 +2456,8 @@ export default class IgniteConfigurationGenerator { static igfsFragmentizer(igfs, cfg = this.igfsConfigurationBean(igfs)) { if (igfs.fragmentizerEnabled) { cfg.intProperty('fragmentizerConcurrentFiles') - .intProperty('fragmentizerThrottlingBlockLength') - .intProperty('fragmentizerThrottlingDelay'); + .longProperty('fragmentizerThrottlingBlockLength') + .longProperty('fragmentizerThrottlingDelay'); } else cfg.boolProperty('fragmentizerEnabled'); @@ -2490,7 +2490,7 @@ export default class IgniteConfigurationGenerator { if (available(['1.0.0', '2.0.0'])) cfg.intProperty('maxSpaceSize'); - cfg.intProperty('maximumTaskRangeLength') + cfg.longProperty('maximumTaskRangeLength') .intProperty('managementPort') .intProperty('perNodeBatchSize') .intProperty('perNodeParallelBatchCount') From 5efc589fcabffdb29cd6dfe0e7323bc91db47703 Mon Sep 17 00:00:00 2001 From: Ilya Borisov Date: Wed, 18 Apr 2018 11:39:41 +0700 Subject: [PATCH 0074/1463] IGNITE-8294 Web Console: Move "Beta" ribbon to the left. (cherry picked from commit 69606e4) --- .../web-console/frontend/public/stylesheets/style.scss | 10 ++++++++++ modules/web-console/frontend/views/index.pug | 2 +- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/modules/web-console/frontend/public/stylesheets/style.scss b/modules/web-console/frontend/public/stylesheets/style.scss index ae1e58c41a769..0c978e9ca43aa 100644 --- a/modules/web-console/frontend/public/stylesheets/style.scss +++ b/modules/web-console/frontend/public/stylesheets/style.scss @@ -1938,6 +1938,16 @@ treecontrol.tree-classic { } } +.ribbon-wrapper.left { + overflow: visible; + + .ribbon { + transform: rotate(-45deg); + left: -75px; + top: 10px; + } +} + html, body { width: 100%; min-height: 100vh; diff --git a/modules/web-console/frontend/views/index.pug b/modules/web-console/frontend/views/index.pug index e91af9bda889f..6384592156c77 100644 --- a/modules/web-console/frontend/views/index.pug +++ b/modules/web-console/frontend/views/index.pug @@ -39,7 +39,7 @@ html(ng-app='ignite-console' id='app' ng-strict-di) .splash-wellcome Loading... - .ribbon-wrapper.right(ng-if='!IgniteDemoMode') + .ribbon-wrapper.left(ng-if='!IgniteDemoMode') .ribbon label Beta From af46856d7f7ec88aa663865a108900abb8314ffe Mon Sep 17 00:00:00 2001 From: oleg-ostanin Date: Wed, 18 Apr 2018 00:58:53 +0700 Subject: [PATCH 0075/1463] IGNITE-8274 sqlline.sh script uses JAVA_HOME now Signed-off-by: Andrey Gura (cherry picked from commit c3ff274) --- modules/sqlline/bin/sqlline.sh | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/modules/sqlline/bin/sqlline.sh b/modules/sqlline/bin/sqlline.sh index 5745aea489a1c..552440de5d185 100644 --- a/modules/sqlline/bin/sqlline.sh +++ b/modules/sqlline/bin/sqlline.sh @@ -35,6 +35,11 @@ SCRIPTS_HOME="${IGNITE_HOME_TMP}/bin" source "${SCRIPTS_HOME}"/include/functions.sh +# +# Discover path to Java executable and check it's version. +# +checkJava + # # Discover IGNITE_HOME environment variable. # @@ -51,4 +56,4 @@ CP="${IGNITE_LIBS}" CP="${CP}${SEP}${IGNITE_HOME_TMP}/bin/include/sqlline/*" -java -cp ${CP} sqlline.SqlLine -d org.apache.ignite.IgniteJdbcThinDriver $@ \ No newline at end of file +"$JAVA" -cp ${CP} sqlline.SqlLine -d org.apache.ignite.IgniteJdbcThinDriver $@ \ No newline at end of file From ba4e337068ebfcc4bbb93509166f118225fe0cb4 Mon Sep 17 00:00:00 2001 From: Dmitriy Shabalin Date: Wed, 18 Apr 2018 18:43:13 +0700 Subject: [PATCH 0076/1463] IGNITE-8298 Web Console: Fixed tables UI issues. (cherry picked from commit a050436) --- .../app/primitives/ui-grid-header/index.scss | 33 +++++++- .../app/primitives/ui-grid/index.scss | 11 ++- .../web-console/frontend/package-lock.json | 80 +++++++++---------- 3 files changed, 80 insertions(+), 44 deletions(-) diff --git a/modules/web-console/frontend/app/primitives/ui-grid-header/index.scss b/modules/web-console/frontend/app/primitives/ui-grid-header/index.scss index ac58707233bd2..1ff27b29c05dd 100644 --- a/modules/web-console/frontend/app/primitives/ui-grid-header/index.scss +++ b/modules/web-console/frontend/app/primitives/ui-grid-header/index.scss @@ -33,6 +33,9 @@ } .ui-grid-header-cell { + // Workaround: Fixed cell header offset in IE11. + vertical-align: top; + .ui-grid-cell-contents > span:not(.ui-grid-header-cell-label) { right: 3px; } @@ -44,7 +47,7 @@ .ui-grid-header-cell [role="columnheader"] { display: flex; - + flex-wrap: wrap; align-items: center; justify-content: center; @@ -77,15 +80,29 @@ .ui-grid-column-resizer.right { top: -100px; } + .ng-hide + .ui-grid-header-cell-row .ui-grid-column-resizer.right { bottom: 0; } + &.ui-grid-header-cell:not(:first-child) { + left: 0; + box-shadow: -1px 0px 0 0 #d4d4d4; + } + &.ui-grid-header-cell .ui-grid-header-cell .ui-grid-column-resizer.right { border-right-width: 0; } + &.ui-grid-header-cell .ui-grid-header-cell:last-child .ui-grid-column-resizer.right { - border-right-width: 1px; + // Hide all right borders, and fix cell offset. + right: -1px; + border: none; + } + + &.ui-grid-header-cell [ng-show] .ui-grid-cell-contents { + text-indent: -20px; + margin-right: -20px; } & > div > .ui-grid-cell-contents { @@ -97,3 +114,15 @@ line-height: 21px; } } + +.ui-grid[ui-grid-selection][ui-grid-grouping] { + .ui-grid-pinned-container-left { + .ui-grid-header--subcategories { + .ui-grid-header-span { + &.ui-grid-header-cell { + box-shadow: none; + } + } + } + } +} diff --git a/modules/web-console/frontend/app/primitives/ui-grid/index.scss b/modules/web-console/frontend/app/primitives/ui-grid/index.scss index b19d5f7af3bbf..2a5c5872a5917 100644 --- a/modules/web-console/frontend/app/primitives/ui-grid/index.scss +++ b/modules/web-console/frontend/app/primitives/ui-grid/index.scss @@ -148,12 +148,12 @@ .ui-grid-header--subcategories { .ui-grid-header-span.ui-grid-header-cell { + background: initial; + .ui-grid-cell-contents { padding: 8px 20px; } - background: initial; - [ng-show] .ui-grid-cell-contents { text-align: center; } @@ -464,6 +464,7 @@ color: #393939; text-align: left; font-size: 14px; + font-weight: normal; line-height: 1.42857; box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); @@ -538,3 +539,9 @@ font-style: italic; line-height: 16px; } + +.ui-grid { + input[type="text"].ui-grid-filter-input { + font-weight: normal; + } +} diff --git a/modules/web-console/frontend/package-lock.json b/modules/web-console/frontend/package-lock.json index e28ef4d01c123..1fec909e3860e 100644 --- a/modules/web-console/frontend/package-lock.json +++ b/modules/web-console/frontend/package-lock.json @@ -401,9 +401,9 @@ "integrity": "sha512-/2xvG6vDC+Us8h0baSa1siDKwPj5R2A7LldxxhK2339HInc09bq9shMVCUy9zqnuvwnDUJ/DSgkSaBoSHSZrqg==" }, "angular-mocks": { - "version": "1.6.9", - "resolved": "https://registry.npmjs.org/angular-mocks/-/angular-mocks-1.6.9.tgz", - "integrity": "sha512-5aEwXmfd5DQvb64pOgP2W2D3ozAQSARkB6q+6NQfUJvJs9bD2YcExrUc1P4EbiIuyWag2OQM+pIKUNojVi3SBg==", + "version": "1.6.10", + "resolved": "https://registry.npmjs.org/angular-mocks/-/angular-mocks-1.6.10.tgz", + "integrity": "sha512-1865/NmqHNogibNoglY1MGBjx882iu2hI46BBhYDWyz0C4TDM5ER8H8SnYwQKUUG4RXMDsJizszEQ2BEoYKV9w==", "dev": true }, "angular-motion": { @@ -1859,9 +1859,9 @@ "integrity": "sha1-c5JncZI7Whl0etZmqlzUv5xunOg=" }, "base64-js": { - "version": "1.2.3", - "resolved": "https://registry.npmjs.org/base64-js/-/base64-js-1.2.3.tgz", - "integrity": "sha512-MsAhsUW1GxCdgYSO6tAfZrNapmUKk7mWx/k5mFY/A1gBtkaCaNapTg+FExCw1r9yeaZhqx/xPg43xgTFH6KL5w==" + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/base64-js/-/base64-js-1.3.0.tgz", + "integrity": "sha512-ccav/yGvoa80BQDljCxsmmQ3Xvx60/UpBIij5QN21W3wBi/hhIC9OoO+KLpu9IJTS9j4DRVJ3aDDF9cMSoa2lw==" }, "base64id": { "version": "1.0.0", @@ -2120,7 +2120,7 @@ "integrity": "sha512-YkIRgwsZwJWTnyQrsBTWefizHh+8GYj3kbL1BTiAQ/9pwpino0G7B2gp5tx/FUBqUlvtxV85KNR3mwfAtv15Yw==", "dev": true, "requires": { - "base64-js": "1.2.3", + "base64-js": "1.3.0", "ieee754": "1.1.11" } }, @@ -2278,7 +2278,7 @@ "resolved": "https://registry.npmjs.org/buffer/-/buffer-4.9.1.tgz", "integrity": "sha1-bRu2AbB6TvztlwlBMgkwJ8lbwpg=", "requires": { - "base64-js": "1.2.3", + "base64-js": "1.3.0", "ieee754": "1.1.11", "isarray": "1.0.0" } @@ -3006,9 +3006,9 @@ } }, "commander": { - "version": "2.15.1", - "resolved": "https://registry.npmjs.org/commander/-/commander-2.15.1.tgz", - "integrity": "sha512-VlfT9F3V0v+jr4yxPc5gg9s62/fIVWsd2Bk2iD435um1NlGMYdVCq+MjcXnhYq2icNOizHr1kK+5TI6H0Hy0ag==" + "version": "2.13.0", + "resolved": "https://registry.npmjs.org/commander/-/commander-2.13.0.tgz", + "integrity": "sha512-MVuS359B+YzaWqjCL/c+22gfryv+mCBPHAv3zyVI2GN8EY6IRP8VwtasXn8jyyhvvq84R4ImN1OKRtcbIasjYA==" }, "commondir": { "version": "1.0.1", @@ -3192,7 +3192,7 @@ "loader-utils": "1.1.0", "minimatch": "3.0.4", "p-limit": "1.2.0", - "serialize-javascript": "1.4.0" + "serialize-javascript": "1.5.0" }, "dependencies": { "globby": { @@ -3517,9 +3517,9 @@ "integrity": "sha1-vEZ0gAQ3iyGjYMn8fPUjF5B2L7g=" }, "d3-color": { - "version": "1.0.3", - "resolved": "https://registry.npmjs.org/d3-color/-/d3-color-1.0.3.tgz", - "integrity": "sha1-vHZD/KjlOoNH4vva/6I2eWtYUJs=" + "version": "1.1.0", + "resolved": "https://registry.npmjs.org/d3-color/-/d3-color-1.1.0.tgz", + "integrity": "sha512-IZVcqX5yYFvR2NUBbSfIfbgNcSgAtZ7JbgQWqDXf4CywtN7agvI7Kw6+Q1ETvlHOHWJT55Kyuzt0C3I0GVtRHQ==" }, "d3-hierarchy": { "version": "1.1.6", @@ -3531,7 +3531,7 @@ "resolved": "https://registry.npmjs.org/d3-interpolate/-/d3-interpolate-1.1.6.tgz", "integrity": "sha512-mOnv5a+pZzkNIHtw/V6I+w9Lqm9L5bG3OTXPM5A+QO0yyVMQ4W1uZhR+VOJmazaOZXri2ppbiZ5BUNWT0pFM9A==", "requires": { - "d3-color": "1.0.3" + "d3-color": "1.1.0" } }, "dargs": { @@ -5568,7 +5568,7 @@ "integrity": "sha1-zcvAgYgmWtEZtqWnyKtw7s+10n0=", "requires": { "chalk": "1.1.3", - "commander": "2.15.1", + "commander": "2.13.0", "is-my-json-valid": "2.17.2", "pinkie-promise": "2.0.1" }, @@ -5825,7 +5825,7 @@ "integrity": "sha512-KcuaIRWTU0kFjOJCs32a3JsGNCWkeOak0/F/uvJNp3x/N4McXdqHpcK64cYTozK7QLPKKtUqb9h7wR9K9rYRkg==", "requires": { "@posthtml/esm": "1.0.0", - "htmlnano": "0.1.7", + "htmlnano": "0.1.8", "loader-utils": "1.1.0", "posthtml": "0.11.3", "schema-utils": "0.4.5" @@ -5845,6 +5845,11 @@ "uglify-js": "3.3.21" }, "dependencies": { + "commander": { + "version": "2.15.1", + "resolved": "https://registry.npmjs.org/commander/-/commander-2.15.1.tgz", + "integrity": "sha512-VlfT9F3V0v+jr4yxPc5gg9s62/fIVWsd2Bk2iD435um1NlGMYdVCq+MjcXnhYq2icNOizHr1kK+5TI6H0Hy0ag==" + }, "source-map": { "version": "0.6.1", "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", @@ -5895,16 +5900,16 @@ "dev": true }, "htmlnano": { - "version": "0.1.7", - "resolved": "https://registry.npmjs.org/htmlnano/-/htmlnano-0.1.7.tgz", - "integrity": "sha512-t8Gy+r/loFP2VXAJl6ClaNIomGI609oyQcT7O3IoJE6VcDCLR6PYWXaSh+hfd/dnoZ6KPbpgPek/Crm3havqig==", + "version": "0.1.8", + "resolved": "https://registry.npmjs.org/htmlnano/-/htmlnano-0.1.8.tgz", + "integrity": "sha512-wwmDRJn5OQ9BqFYy5vWaufUQTKj7Ct6xTv+od7QNNJzJM7K3yqR4lJ8SHSOTcBahlXMO5EzddUdsS+fmdGvXpw==", "requires": { "cssnano": "3.10.0", "object-assign": "4.1.1", "posthtml": "0.11.3", "posthtml-render": "1.1.3", "svgo": "1.0.5", - "uglify-js": "3.3.21" + "uglify-es": "3.3.9" }, "dependencies": { "coa": { @@ -5969,12 +5974,12 @@ "util.promisify": "1.0.0" } }, - "uglify-js": { - "version": "3.3.21", - "resolved": "https://registry.npmjs.org/uglify-js/-/uglify-js-3.3.21.tgz", - "integrity": "sha512-uy82472lH8tshK3jS3c5IFb5MmNKd/5qyBd0ih8sM42L3jWvxnE339U9gZU1zufnLVs98Stib9twq8dLm2XYCA==", + "uglify-es": { + "version": "3.3.9", + "resolved": "https://registry.npmjs.org/uglify-es/-/uglify-es-3.3.9.tgz", + "integrity": "sha512-r+MU0rfv4L/0eeW3xZrd16t4NZfK8Ld4SWVglYBb7ez5uXFWHuVRs6xCTrf1yirs9a4j4Y27nn7SRfO6v67XsQ==", "requires": { - "commander": "2.15.1", + "commander": "2.13.0", "source-map": "0.6.1" }, "dependencies": { @@ -8538,7 +8543,7 @@ "stream-browserify": "2.0.1", "stream-http": "2.8.1", "string_decoder": "1.1.1", - "timers-browserify": "2.0.7", + "timers-browserify": "2.0.9", "tty-browserify": "0.0.0", "url": "0.11.0", "util": "0.10.3", @@ -11273,9 +11278,9 @@ } }, "serialize-javascript": { - "version": "1.4.0", - "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-1.4.0.tgz", - "integrity": "sha1-fJWFFNtqwkQ6irwGLcn3iGp/YAU=" + "version": "1.5.0", + "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-1.5.0.tgz", + "integrity": "sha512-Ga8c8NjAAp46Br4+0oZ2WxJCwIzwP60Gq1YPgU+39PiTVxyed/iKE/zyZI6+UlVYH5Q4PaQdHhcegIFPZTUfoQ==" }, "serve-index": { "version": "1.9.1", @@ -12528,9 +12533,9 @@ "integrity": "sha1-8y6srFoXW+ol1/q1Zas+2HQe9W8=" }, "timers-browserify": { - "version": "2.0.7", - "resolved": "https://registry.npmjs.org/timers-browserify/-/timers-browserify-2.0.7.tgz", - "integrity": "sha512-U7DtjfsHeYjNAyEz4MdCLGZMY3ySyHIgZZp6ba9uxZlMRMiK5yTHUYc2XfGQHKFgxGcmvBF2jafoNtQYvlDpOw==", + "version": "2.0.9", + "resolved": "https://registry.npmjs.org/timers-browserify/-/timers-browserify-2.0.9.tgz", + "integrity": "sha512-2DhyvVpCWwY7gk8UmKhYvgHQl9XTlO0Dg0/2UZcLgPnpulhdm2aGIlFy5rU5igmOCA51w6jPHqLRA4UH1YmhcA==", "requires": { "setimmediate": "1.0.5" } @@ -12790,18 +12795,13 @@ "cacache": "10.0.4", "find-cache-dir": "1.0.0", "schema-utils": "0.4.5", - "serialize-javascript": "1.4.0", + "serialize-javascript": "1.5.0", "source-map": "0.6.1", "uglify-es": "3.3.9", "webpack-sources": "1.1.0", "worker-farm": "1.6.0" }, "dependencies": { - "commander": { - "version": "2.13.0", - "resolved": "https://registry.npmjs.org/commander/-/commander-2.13.0.tgz", - "integrity": "sha512-MVuS359B+YzaWqjCL/c+22gfryv+mCBPHAv3zyVI2GN8EY6IRP8VwtasXn8jyyhvvq84R4ImN1OKRtcbIasjYA==" - }, "source-map": { "version": "0.6.1", "resolved": "https://registry.npmjs.org/source-map/-/source-map-0.6.1.tgz", From 25b25f271bc89d77013e1cda2bad30d941e1c8ad Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 18 Apr 2018 15:57:45 +0300 Subject: [PATCH 0077/1463] IGNITE-8122 Restore partition state from WAL if no checkpoints are done - Fixes #3745. Signed-off-by: Alexey Goncharuk --- .../affinity/GridAffinityAssignment.java | 4 +- .../processors/cache/ExchangeActions.java | 2 +- .../dht/GridDhtPartitionTopology.java | 3 +- .../dht/GridDhtPartitionTopologyImpl.java | 220 ++++++++---------- .../dht/GridDhtPartitionsStateValidator.java | 61 ++++- .../GridDhtPartitionsExchangeFuture.java | 6 +- .../GridDhtPartitionsSingleMessage.java | 18 +- .../dht/preloader/GridDhtPreloader.java | 4 +- .../GridCacheDatabaseSharedManager.java | 89 +++---- .../CacheBaselineTopologyTest.java | 16 +- ...cheLoadingConcurrentGridStartSelfTest.java | 3 + ...ridCachePartitionsStateValidationTest.java | 4 + ...CachePartitionsStateValidatorSelfTest.java | 47 ++-- .../testsuites/IgniteStandByClusterSuite.java | 2 +- .../IgniteCacheQueryNodeRestartSelfTest.java | 2 + ...BaselineCacheQueryNodeRestartSelfTest.java | 4 +- 16 files changed, 258 insertions(+), 227 deletions(-) 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 6da6aaa5c2d6d..cbec1a1852eba 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 @@ -149,7 +149,7 @@ public AffinityTopologyVersion topologyVersion() { * @param part Partition. * @return Affinity nodes. */ - public List get(int part) { + @Override public List get(int part) { assert part >= 0 && part < assignment.size() : "Affinity partition is out of range" + " [part=" + part + ", partitions=" + assignment.size() + ']'; @@ -162,7 +162,7 @@ public List get(int part) { * @param part Partition. * @return Affinity nodes IDs. */ - public HashSet getIds(int part) { + @Override public HashSet getIds(int part) { assert part >= 0 && part < assignment.size() : "Affinity partition is out of range" + " [part=" + part + ", partitions=" + assignment.size() + ']'; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java index bcf3f408c222f..c289b6e31e7f4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java @@ -409,7 +409,7 @@ public DynamicCacheDescriptor descriptor() { /** * */ - static class CacheGroupActionData { + public static class CacheGroupActionData { /** */ private final CacheGroupDescriptor desc; 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 6f68dbbaeb511..d586a94d53203 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 @@ -386,8 +386,9 @@ public boolean update(@Nullable GridDhtPartitionExchangeId exchId, * State of all current owners that aren't contained in the set will be reset to MOVING. * * @param p Partition ID. - * @param updateSeq If should increment sequence when updated. * @param owners Set of new owners. + * @param haveHistory {@code True} if there is WAL history to rebalance given partition. + * @param updateSeq If should increment sequence when updated. * @return Set of node IDs that should reload partitions. */ public Set setOwners(int p, Set owners, boolean haveHistory, boolean updateSeq); 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 740903e415baf..164f0bf451286 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 @@ -17,7 +17,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht; -import java.nio.file.Files; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -31,8 +30,6 @@ import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReferenceArray; -import java.util.stream.Collectors; -import java.util.stream.IntStream; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; @@ -43,7 +40,6 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.managers.discovery.DiscoCache; -import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager; import org.apache.ignite.internal.processors.affinity.AffinityAssignment; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheGroupContext; @@ -55,8 +51,6 @@ 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.persistence.GridCacheDatabaseSharedManager; -import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridAtomicLong; import org.apache.ignite.internal.util.GridPartitionStateMap; @@ -324,7 +318,7 @@ private String mapString(GridDhtPartitionMap map) { long updateSeq = this.updateSeq.incrementAndGet(); - needRefresh = initPartitions0(affVer, exchFut, updateSeq); + needRefresh = initPartitions(affVer, grp.affinity().readyAssignments(affVer), exchFut, updateSeq); consistencyCheck(); } @@ -340,14 +334,15 @@ private String mapString(GridDhtPartitionMap map) { } /** + * Creates and initializes partitions using given {@code affVer} and {@code affAssignment}. + * * @param affVer Affinity version to use. + * @param affAssignment Affinity assignment to use. * @param exchFut Exchange future. * @param updateSeq Update sequence. * @return {@code True} if partitions must be refreshed. */ - private boolean initPartitions0(AffinityTopologyVersion affVer, GridDhtPartitionsExchangeFuture exchFut, long updateSeq) { - List> aff = grp.affinity().readyAssignments(affVer); - + private boolean initPartitions(AffinityTopologyVersion affVer, List> affAssignment, GridDhtPartitionsExchangeFuture exchFut, long updateSeq) { boolean needRefresh = false; if (grp.affinityNode()) { @@ -357,32 +352,24 @@ private boolean initPartitions0(AffinityTopologyVersion affVer, GridDhtPartition GridDhtPartitionExchangeId exchId = exchFut.exchangeId(); - assert grp.affinity().lastVersion().equals(affVer) : - "Invalid affinity [topVer=" + grp.affinity().lastVersion() + - ", grp=" + grp.cacheOrGroupName() + - ", affVer=" + affVer + - ", fut=" + exchFut + ']'; - int num = grp.affinity().partitions(); if (grp.rebalanceEnabled()) { boolean added = exchFut.cacheGroupAddedOnExchange(grp.groupId(), grp.receivedFrom()); - boolean first = added || (loc.equals(oldest) && loc.id().equals(exchId.nodeId()) && exchId.isJoined()); + boolean first = added || (loc.equals(oldest) && loc.id().equals(exchId.nodeId()) && exchId.isJoined()) || exchFut.activateCluster(); if (first) { - assert exchId.isJoined() || added; + assert exchId.isJoined() || added || exchFut.activateCluster(); for (int p = 0; p < num; p++) { - if (localNode(p, aff) || initLocalPartition(p, discoCache)) { - GridDhtLocalPartition locPart = createPartition(p); + if (localNode(p, affAssignment)) { + // Partition is created first time, so it's safe to own it. + boolean shouldOwn = locParts.get(p) == null; - if (grp.persistenceEnabled()) { - GridCacheDatabaseSharedManager db = (GridCacheDatabaseSharedManager)grp.shared().database(); + GridDhtLocalPartition locPart = getOrCreatePartition(p); - locPart.restoreState(db.readPartitionState(grp, locPart.id())); - } - else { + if (shouldOwn) { boolean owned = locPart.own(); assert owned : "Failed to own partition for oldest node [grp=" + grp.cacheOrGroupName() + @@ -390,7 +377,7 @@ private boolean initPartitions0(AffinityTopologyVersion affVer, GridDhtPartition if (log.isDebugEnabled()) log.debug("Owned partition for oldest node [grp=" + grp.cacheOrGroupName() + - ", part=" + locPart + ']'); + ", part=" + locPart + ']'); } needRefresh = true; @@ -400,15 +387,15 @@ private boolean initPartitions0(AffinityTopologyVersion affVer, GridDhtPartition } } else - createPartitions(affVer, aff, updateSeq); + createPartitions(affVer, affAssignment, updateSeq); } else { // If preloader is disabled, then we simply clear out // the partitions this node is not responsible for. for (int p = 0; p < num; p++) { - GridDhtLocalPartition locPart = localPartition0(p, affVer, false, true, false); + GridDhtLocalPartition locPart = localPartition0(p, affVer, false, true); - boolean belongs = localNode(p, aff); + boolean belongs = localNode(p, affAssignment); if (locPart != null) { if (!belongs) { @@ -429,7 +416,7 @@ private boolean initPartitions0(AffinityTopologyVersion affVer, GridDhtPartition locPart.own(); } else if (belongs) { - locPart = createPartition(p); + locPart = getOrCreatePartition(p); locPart.own(); @@ -439,27 +426,14 @@ else if (belongs) { } } - updateRebalanceVersion(aff); + updateRebalanceVersion(affAssignment); return needRefresh; } /** - * @param p Partition ID to restore. - * @param discoCache Disco cache to use. - * @return {@code True} if should restore local partition. - */ - private boolean initLocalPartition(int p, DiscoCache discoCache) { - IgnitePageStoreManager storeMgr = ctx.pageStore(); - - return - grp.persistenceEnabled() && - storeMgr instanceof FilePageStoreManager && - discoCache.baselineNode(ctx.localNodeId()) && - Files.exists(((FilePageStoreManager)storeMgr).getPath(grp.sharedGroup(), grp.cacheOrGroupName(), p)); - } - - /** + * Creates non-existing partitions belong to given affinity {@code aff}. + * * @param affVer Affinity version. * @param aff Affinity assignments. * @param updateSeq Update sequence. @@ -475,7 +449,7 @@ private void createPartitions(AffinityTopologyVersion affVer, List> affAssignment; + if (affReady) { - assert grp.affinity().lastVersion().equals(evts.topologyVersion()); + affVer = evts.topologyVersion(); - initPartitions0(evts.topologyVersion(), exchFut, updateSeq); + assert grp.affinity().lastVersion().equals(affVer) : + "Invalid affinity [topVer=" + grp.affinity().lastVersion() + + ", grp=" + grp.cacheOrGroupName() + + ", affVer=" + affVer + + ", fut=" + exchFut + ']'; + + affAssignment = grp.affinity().readyAssignments(affVer); } else { assert !exchFut.context().mergeExchanges(); - List> aff = grp.affinity().idealAssignment(); - - createPartitions(exchFut.initialVersion(), aff, updateSeq); + affVer = exchFut.initialVersion(); + affAssignment = grp.affinity().idealAssignment(); } + + initPartitions(affVer, affAssignment, exchFut, updateSeq); } } @@ -695,19 +680,12 @@ private boolean partitionLocalNode(int p, AffinityTopologyVersion topVer) { long updateSeq = this.updateSeq.incrementAndGet(); for (int p = 0; p < num; p++) { - GridDhtLocalPartition locPart = localPartition0(p, topVer, false, false, false); + GridDhtLocalPartition locPart = localPartition0(p, topVer, false, true); if (partitionLocalNode(p, topVer)) { - // This partition will be created during next topology event, - // which obviously has not happened at this point. - if (locPart == null) { - if (log.isDebugEnabled()) { - log.debug("Skipping local partition afterExchange (will not create) [" + - "grp=" + grp.cacheOrGroupName() + ", p=" + p + ']'); - } - - continue; - } + // Prepare partition to rebalance if it's not happened on full map update phase. + if (locPart == null || locPart.state() == RENTING || locPart.state() == EVICTED) + locPart = rebalancePartition(p, false); GridDhtPartitionState state = locPart.state(); @@ -793,20 +771,23 @@ else if (log.isDebugEnabled()) @Nullable @Override public GridDhtLocalPartition localPartition(int p, AffinityTopologyVersion topVer, boolean create) throws GridDhtInvalidPartitionException { - return localPartition0(p, topVer, create, false, true); + return localPartition0(p, topVer, create, false); } /** {@inheritDoc} */ @Nullable @Override public GridDhtLocalPartition localPartition(int p, AffinityTopologyVersion topVer, boolean create, boolean showRenting) throws GridDhtInvalidPartitionException { - return localPartition0(p, topVer, create, showRenting, true); + return localPartition0(p, topVer, create, showRenting); } /** + * Creates partition with id {@code p} if it doesn't exist or evicted. + * In other case returns existing partition. + * * @param p Partition number. * @return Partition. */ - private GridDhtLocalPartition createPartition(int p) { + private GridDhtLocalPartition getOrCreatePartition(int p) { assert lock.isWriteLockedByCurrentThread(); assert ctx.database().checkpointLockIsHeldByThread(); @@ -865,16 +846,15 @@ private GridDhtLocalPartition createPartition(int p) { /** * @param p Partition number. * @param topVer Topology version. - * @param create Create flag. - * @param updateSeq Update sequence. + * @param create If {@code true} create partition if it doesn't exists or evicted. + * @param showRenting If {@code true} return partition in RENTING state if exists. * @return Local partition. */ @SuppressWarnings("TooBroadScope") private GridDhtLocalPartition localPartition0(int p, AffinityTopologyVersion topVer, boolean create, - boolean showRenting, - boolean updateSeq) { + boolean showRenting) { GridDhtLocalPartition loc; loc = locParts.get(p); @@ -1345,6 +1325,8 @@ private boolean shouldOverridePartitionMap(GridDhtPartitionMap currentMap, GridD if (incomeCntrMap != null) { // update local counters in partitions for (int i = 0; i < locParts.length(); i++) { + cntrMap.updateCounter(i, incomeCntrMap.updateCounter(i)); + GridDhtLocalPartition part = locParts.get(i); if (part == null) @@ -1511,51 +1493,11 @@ private boolean shouldOverridePartitionMap(GridDhtPartitionMap currentMap, GridD } } else if (state == MOVING) { - GridDhtLocalPartition locPart = locParts.get(p); - - if (!partsToReload.contains(p)) { - if (locPart == null || locPart.state() == EVICTED) - locPart = createPartition(p); - - if (locPart.state() == OWNING) { - locPart.moving(); + boolean haveHistory = !partsToReload.contains(p); - changed = true; - } - } - else { - if (locPart == null || locPart.state() == EVICTED) { - createPartition(p); + rebalancePartition(p, haveHistory); - changed = true; - } - else if (locPart.state() == OWNING || locPart.state() == MOVING) { - if (locPart.state() == OWNING) - locPart.moving(); - locPart.clearAsync(); - - changed = true; - } - else if (locPart.state() == RENTING) { - // Try to prevent partition eviction. - if (locPart.reserve()) { - try { - locPart.moving(); - locPart.clearAsync(); - } finally { - locPart.release(); - } - } - // In other case just recreate it. - else { - assert locPart.state() == EVICTED; - - createPartition(p); - } - - changed = true; - } - } + changed = true; } } } @@ -2113,7 +2055,7 @@ else if (plc != PartitionLossPolicy.IGNORE) { } /** {@inheritDoc} */ - @Override public Set setOwners(int p, Set owners, boolean haveHistory, boolean updateSeq) { + @Override public Set setOwners(int p, Set ownersByUpdCounters, boolean haveHistory, boolean updateSeq) { Set result = haveHistory ? Collections.emptySet() : new HashSet(); ctx.database().checkpointReadLock(); @@ -2125,17 +2067,15 @@ else if (plc != PartitionLossPolicy.IGNORE) { GridDhtLocalPartition locPart = locParts.get(p); if (locPart != null) { - if (locPart.state() == OWNING && !owners.contains(ctx.localNodeId())) { - locPart.moving(); + if (locPart.state() == OWNING && !ownersByUpdCounters.contains(ctx.localNodeId())) { + rebalancePartition(p, haveHistory); - if (!haveHistory) { - locPart.clearAsync(); + if (!haveHistory) result.add(ctx.localNodeId()); - } U.warn(log, "Partition has been scheduled for rebalancing due to outdated update counter " + "[nodeId=" + ctx.localNodeId() + ", grp=" + grp.cacheOrGroupName() + - ", partId=" + locPart.id() + ", haveHistory=" + haveHistory + "]"); + ", partId=" + p + ", haveHistory=" + haveHistory + "]"); } } @@ -2146,7 +2086,7 @@ else if (plc != PartitionLossPolicy.IGNORE) { if (!partMap.containsKey(p)) continue; - if (partMap.get(p) == OWNING && !owners.contains(remoteNodeId)) { + if (partMap.get(p) == OWNING && !ownersByUpdCounters.contains(remoteNodeId)) { partMap.put(p, MOVING); if (!haveHistory) @@ -2179,6 +2119,42 @@ else if (plc != PartitionLossPolicy.IGNORE) { return result; } + /** + * Prepares given partition {@code p} for rebalance. + * Changes partition state to MOVING and starts clearing if needed. + * Prevents ongoing renting if required. + * + * @param p Partition id. + * @param haveHistory If {@code true} there is WAL history to rebalance partition, + * in other case partition will be cleared for full rebalance. + */ + private GridDhtLocalPartition rebalancePartition(int p, boolean haveHistory) { + GridDhtLocalPartition part = getOrCreatePartition(p); + + // Prevent renting. + if (part.state() == RENTING) { + if (part.reserve()) { + part.moving(); + part.release(); + } + else { + assert part.state() == EVICTED : part; + + part = getOrCreatePartition(p); + } + } + + if (part.state() != MOVING) + part.moving(); + + if (!haveHistory) + part.clearAsync(); + + assert part.state() == MOVING : part; + + return part; + } + /** * Finds local partitions which don't belong to affinity and runs eviction process for such partitions. * 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 92a05848e3d00..cc0542c5d0e5b 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 @@ -35,6 +35,7 @@ import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.SB; import org.apache.ignite.lang.IgniteProductVersion; +import org.jetbrains.annotations.Nullable; import static org.apache.ignite.events.EventType.EVT_NODE_JOINED; @@ -71,9 +72,9 @@ public GridDhtPartitionsStateValidator(GridCacheSharedContext cctx) { public void validatePartitionCountersAndSizes(GridDhtPartitionsExchangeFuture fut, GridDhtPartitionTopology top, Map messages) throws IgniteCheckedException { - // Ignore just joined nodes. final Set ignoringNodes = new HashSet<>(); + // Ignore just joined nodes. for (DiscoveryEvent evt : fut.events().events()) if (evt.type() == EVT_NODE_JOINED) ignoringNodes.add(evt.eventNode().id()); @@ -98,6 +99,46 @@ public void validatePartitionCountersAndSizes(GridDhtPartitionsExchangeFuture fu throw new IgniteCheckedException("Partitions cache sizes are inconsistent for " + fold(topVer, result)); } + /** + * Checks what partitions from given {@code singleMsg} message should be excluded from validation. + * + * @param top Topology to validate. + * @param nodeId Node which sent single message. + * @param singleMsg Single message. + * @return Set of partition ids should be excluded from validation. + */ + @Nullable private Set shouldIgnore(GridDhtPartitionTopology top, UUID nodeId, GridDhtPartitionsSingleMessage singleMsg) { + CachePartitionPartialCountersMap countersMap = singleMsg.partitionUpdateCounters(top.groupId(), top.partitions()); + Map sizesMap = singleMsg.partitionSizes(top.groupId()); + + Set ignore = null; + + for (int p = 0; p < top.partitions(); p++) { + if (top.partitionState(nodeId, p) != GridDhtPartitionState.OWNING) { + if (ignore == null) + ignore = new HashSet<>(); + + ignore.add(p); + + continue; + } + + int partIdx = countersMap.partitionIndex(p); + long updateCounter = partIdx >= 0 ? countersMap.updateCounterAt(partIdx) : 0; + long size = sizesMap.containsKey(p) ? sizesMap.get(p) : 0; + + // Do not validate partitions with zero update counter and size. + if (updateCounter == 0 && size == 0) { + if (ignore == null) + ignore = new HashSet<>(); + + ignore.add(p); + } + } + + return ignore; + } + /** * Validate partitions update counters for given {@code top}. * @@ -117,7 +158,10 @@ Map> validatePartitionsUpdateCounters( // Populate counters statistics from local node partitions. for (GridDhtLocalPartition part : top.currentLocalPartitions()) { - if (top.partitionState(cctx.localNodeId(), part.id()) != GridDhtPartitionState.OWNING) + if (part.state() != GridDhtPartitionState.OWNING) + continue; + + if (part.updateCounter() == 0 && part.fullSize() == 0) continue; updateCountersAndNodesByPartitions.put(part.id(), new T2<>(cctx.localNodeId(), part.updateCounter())); @@ -133,8 +177,10 @@ Map> validatePartitionsUpdateCounters( CachePartitionPartialCountersMap countersMap = e.getValue().partitionUpdateCounters(top.groupId(), partitions); + Set ignorePartitions = shouldIgnore(top, nodeId, e.getValue()); + for (int part = 0; part < partitions; part++) { - if (top.partitionState(nodeId, part) != GridDhtPartitionState.OWNING) + if (ignorePartitions != null && ignorePartitions.contains(part)) continue; int partIdx = countersMap.partitionIndex(part); @@ -166,7 +212,10 @@ Map> validatePartitionsSizes( // Populate sizes statistics from local node partitions. for (GridDhtLocalPartition part : top.currentLocalPartitions()) { - if (top.partitionState(cctx.localNodeId(), part.id()) != GridDhtPartitionState.OWNING) + if (part.state() != GridDhtPartitionState.OWNING) + continue; + + if (part.updateCounter() == 0 && part.fullSize() == 0) continue; sizesAndNodesByPartitions.put(part.id(), new T2<>(cctx.localNodeId(), part.fullSize())); @@ -182,8 +231,10 @@ Map> validatePartitionsSizes( Map sizesMap = e.getValue().partitionSizes(top.groupId()); + Set ignorePartitions = shouldIgnore(top, nodeId, e.getValue()); + for (int part = 0; part < partitions; part++) { - if (top.partitionState(nodeId, part) != GridDhtPartitionState.OWNING) + if (ignorePartitions != null && ignorePartitions.contains(part)) continue; long currentSize = sizesMap.containsKey(part) ? sizesMap.get(part) : 0L; 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 af5acd64257e7..0d57d483f71a9 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 @@ -58,6 +58,7 @@ import org.apache.ignite.internal.managers.discovery.DiscoCache; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.pagemem.wal.record.ExchangeRecord; +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.CacheAffinityChangeMessage; @@ -2244,6 +2245,8 @@ private void onAffinityInitialized(IgniteInternalFuture> partSizes; /** Serialized partitions counters. */ - private byte[] partSizesBytes; + private byte[] partsSizesBytes; /** Partitions history reservation counters. */ @GridToStringInclude @@ -324,7 +324,7 @@ public void setError(Exception ex) { boolean marshal = (parts != null && partsBytes == null) || (partCntrs != null && partCntrsBytes == null) || (partHistCntrs != null && partHistCntrsBytes == null) || - (partSizes != null && partSizesBytes == null) || + (partSizes != null && partsSizesBytes == null) || (err != null && errBytes == null); if (marshal) { @@ -343,7 +343,7 @@ public void setError(Exception ex) { if (partHistCntrs != null && partHistCntrsBytes == null) partHistCntrsBytes0 = U.marshal(ctx, partHistCntrs); - if (partSizes != null && partSizesBytes == null) + if (partSizes != null && partsSizesBytes == null) partSizesBytes0 = U.marshal(ctx, partSizes); if (err != null && errBytes == null) @@ -375,7 +375,7 @@ public void setError(Exception ex) { partsBytes = partsBytes0; partCntrsBytes = partCntrsBytes0; partHistCntrsBytes = partHistCntrsBytes0; - partSizesBytes = partSizesBytes0; + partsSizesBytes = partSizesBytes0; errBytes = errBytes0; } } @@ -405,11 +405,11 @@ public void setError(Exception ex) { partHistCntrs = U.unmarshal(ctx, partHistCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); } - if (partSizesBytes != null && partSizes == null) { + if (partsSizesBytes != null && partSizes == null) { if (compressed()) - partSizes = U.unmarshalZip(ctx.marshaller(), partSizesBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + partSizes = U.unmarshalZip(ctx.marshaller(), partsSizesBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); else - partSizes = U.unmarshal(ctx, partSizesBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + partSizes = U.unmarshal(ctx, partsSizesBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); } if (errBytes != null && err == null) { @@ -504,7 +504,7 @@ public void setError(Exception ex) { writer.incrementState(); case 13: - if (!writer.writeByteArray("partsSizesBytes", partSizesBytes)) + if (!writer.writeByteArray("partsSizesBytes", partsSizesBytes)) return false; writer.incrementState(); @@ -589,7 +589,7 @@ public void setError(Exception ex) { reader.incrementState(); case 13: - partSizesBytes = reader.readByteArray("partsSizesBytes"); + partsSizesBytes = reader.readByteArray("partsSizesBytes"); if (!reader.isLastRead()) return false; 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 6ec6ad33991da..ddcb81e238965 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 @@ -202,7 +202,7 @@ private IgniteCheckedException stopError() { // If partition belongs to local node. if (aff.get(p).contains(ctx.localNode())) { - GridDhtLocalPartition part = top.localPartition(p, topVer, true, true); + GridDhtLocalPartition part = top.localPartition(p); assert part != null; assert part.id() == p; @@ -226,7 +226,7 @@ private IgniteCheckedException stopError() { part = top.localPartition(p, topVer, true); } - assert part != null && part.state() == MOVING : "Partition has invalid state for rebalance " + part; + assert part.state() == MOVING : "Partition has invalid state for rebalance " + aff.topologyVersion() + " " + part; ClusterNode histSupplier = null; 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 16d32924da847..3009febae9b44 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 @@ -59,6 +59,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.regex.Matcher; import java.util.regex.Pattern; +import java.util.stream.Collectors; import javax.management.ObjectName; import org.apache.ignite.DataStorageMetrics; import org.apache.ignite.IgniteCheckedException; @@ -1151,9 +1152,18 @@ private void shutdownCheckpointer(boolean cancel) { boolean clusterInTransitionStateToActive = fut.activateCluster(); - // Before local node join event. - if (clusterInTransitionStateToActive || (joinEvt && locNode && isSrvNode)) + // In case of cluster activation or local join restore, restore whole manager state. + if (clusterInTransitionStateToActive || (joinEvt && locNode && isSrvNode)) { restoreState(); + } + // In case of starting groups, restore partition states only for these groups. + else if (fut.exchangeActions() != null && !F.isEmpty(fut.exchangeActions().cacheGroupsToStart())) { + Set restoreGroups = fut.exchangeActions().cacheGroupsToStart().stream() + .map(actionData -> actionData.descriptor().groupId()) + .collect(Collectors.toSet()); + + restorePartitionStates(Collections.emptyMap(), restoreGroups); + } if (cctx.kernalContext().query().moduleEnabled()) { ExchangeActions acts = fut.exchangeActions(); @@ -1393,6 +1403,8 @@ private boolean safeToUpdatePageMemories() { } /** + * Restores from last checkpoint and applies WAL changes since this checkpoint. + * * @throws IgniteCheckedException If failed to restore database status from WAL. */ private void restoreState() throws IgniteCheckedException { @@ -2154,7 +2166,7 @@ else if (log != null) checkpointReadLock(); try { - restorePartitionState(partStates, Collections.emptySet()); + restorePartitionStates(partStates, null); } finally { checkpointReadUnlock(); @@ -2264,7 +2276,7 @@ private void applyLastUpdates(CheckpointStatus status, boolean metastoreOnly) th } if (!metastoreOnly) - restorePartitionState(partStates, ignoreGrps); + restorePartitionStates(partStates, null); } finally { if (!metastoreOnly) @@ -2277,15 +2289,17 @@ private void applyLastUpdates(CheckpointStatus status, boolean metastoreOnly) th } /** - * @param partStates Partition states. - * @throws IgniteCheckedException If failed to restore. + * Initializes not empty partitions and restores their state from page memory or WAL. + * Partition states presented in page memory may be overriden by states restored from WAL {@code partStates}. + * + * @param partStates Partition states restored from WAL. + * @param onlyForGroups If not {@code null} restore states only for specified cache groups. + * @throws IgniteCheckedException If failed to restore partition states. */ - private void restorePartitionState( - Map, T2> partStates, - Collection ignoreGrps - ) throws IgniteCheckedException { + private void restorePartitionStates(Map, T2> partStates, + @Nullable Set onlyForGroups) throws IgniteCheckedException { for (CacheGroupContext grp : cctx.cache().cacheGroups()) { - if (grp.isLocal() || !grp.affinityNode() || ignoreGrps.contains(grp.groupId())) { + if (grp.isLocal() || !grp.affinityNode()) { // Local cache has no partitions and its states. continue; } @@ -2293,6 +2307,9 @@ private void restorePartitionState( if (!grp.dataRegion().config().isPersistenceEnabled()) continue; + if (onlyForGroups != null && !onlyForGroups.contains(grp.groupId())) + continue; + int grpId = grp.groupId(); PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory(); @@ -2375,56 +2392,6 @@ else if (restore != null) { } } - /** - * @param grpCtx Group context. - * @param partId Partition ID. - * @return Partition state. - */ - public GridDhtPartitionState readPartitionState(CacheGroupContext grpCtx, int partId) { - int grpId = grpCtx.groupId(); - PageMemoryEx pageMem = (PageMemoryEx)grpCtx.dataRegion().pageMemory(); - - try { - if (storeMgr.exists(grpId, partId)) { - storeMgr.ensure(grpId, partId); - - if (storeMgr.pages(grpId, partId) > 1) { - long partMetaId = pageMem.partitionMetaPageId(grpId, partId); - long partMetaPage = pageMem.acquirePage(grpId, partMetaId); - - try { - long pageAddr = pageMem.readLock(grpId, partMetaId, partMetaPage); - - try { - if (PageIO.getType(pageAddr) == PageIO.T_PART_META) { - PagePartitionMetaIO io = PagePartitionMetaIO.VERSIONS.forPage(pageAddr); - - GridDhtPartitionState state = GridDhtPartitionState.fromOrdinal((int)io.getPartitionState(pageAddr)); - - if (state == null) - state = GridDhtPartitionState.MOVING; - - return state; - } - } - finally { - pageMem.readUnlock(grpId, partMetaId, partMetaPage); - } - } - finally { - pageMem.releasePage(grpId, partMetaId, partMetaPage); - } - } - } - } - catch (IgniteCheckedException e) { - U.error(log, "Failed to read partition state (will default to MOVING) [grp=" + grpCtx + - ", partId=" + partId + "]", e); - } - - return GridDhtPartitionState.MOVING; - } - /** * Wal truncate callBack. * diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java index 0d59a2d79a13d..c3d404b50fd7c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheBaselineTopologyTest.java @@ -82,6 +82,9 @@ public class CacheBaselineTopologyTest extends GridCommonAbstractTest { /** */ private boolean delayRebalance; + /** */ + private boolean disableAutoActivation; + /** */ private Map userAttrs; @@ -107,6 +110,8 @@ public class CacheBaselineTopologyTest extends GridCommonAbstractTest { cleanPersistenceDir(); client = false; + + disableAutoActivation = false; } /** {@inheritDoc} */ @@ -120,6 +125,9 @@ public class CacheBaselineTopologyTest extends GridCommonAbstractTest { cfg.setConsistentId(igniteInstanceName); + if (disableAutoActivation) + cfg.setAutoActivationEnabled(false); + cfg.setDataStorageConfiguration( new DataStorageConfiguration().setDefaultDataRegionConfiguration( new DataRegionConfiguration() @@ -884,11 +892,15 @@ public void testAffinityAssignmentChangedAfterRestart() throws Exception { delayRebalance = true; + /* There is a problem with handling simultaneous auto activation after restart and manual activation. + To properly catch the moment when cluster activation has finished we temporary disable auto activation. */ + disableAutoActivation = true; + startGrids(4); ig = grid(0); - ig.active(true); + ig.cluster().active(true); cache = ig.cache(cacheName); @@ -990,7 +1002,7 @@ public TestAffinityFunction(AffinityFunction delegate) { /** {@inheritDoc} */ @Override public void reset() { - delegate.reset();; + delegate.reset(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java index 1e046d41f4b05..ebc804f1c885d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java @@ -31,6 +31,7 @@ import org.apache.ignite.IgniteDataStreamer; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CachePeekMode; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.cache.store.CacheStoreAdapter; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; @@ -93,6 +94,8 @@ public class CacheLoadingConcurrentGridStartSelfTest extends GridCommonAbstractT ccfg.setCacheStoreFactory(new FactoryBuilder.SingletonFactory(new TestCacheStoreAdapter())); + ccfg.setAffinity(new RendezvousAffinityFunction(false, 64)); + if (getTestIgniteInstanceName(0).equals(igniteInstanceName)) { if (client) cfg.setClientMode(true); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidationTest.java index 63d772a0077c0..fc617bb8f97e2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidationTest.java @@ -106,6 +106,10 @@ public void testValidationIfPartitionCountersAreInconsistent() throws Exception awaitPartitionMapExchange(); + // Populate cache to increment update counters. + for (int i = 0; i < 1000; i++) + ignite.cache(CACHE_NAME).put(i, i); + // Modify update counter for some partition. for (GridDhtLocalPartition partition : ignite.cachex(CACHE_NAME).context().topology().localPartitions()) { partition.updateCounter(100500L); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidatorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidatorSelfTest.java index 9ed8d54080a37..43a23031dd99a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidatorSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidatorSelfTest.java @@ -27,6 +27,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.jetbrains.annotations.Nullable; import org.junit.Assert; import org.mockito.Matchers; import org.mockito.Mockito; @@ -70,24 +71,21 @@ private GridDhtLocalPartition partitionMock(int id, long updateCounter, long siz Mockito.when(partitionMock.id()).thenReturn(id); Mockito.when(partitionMock.updateCounter()).thenReturn(updateCounter); Mockito.when(partitionMock.fullSize()).thenReturn(size); + Mockito.when(partitionMock.state()).thenReturn(GridDhtPartitionState.OWNING); return partitionMock; } /** - * @return Message containing specified {@code countersMap}. + * @param countersMap Update counters map. + * @param sizesMap Sizes map. + * @return Message with specified {@code countersMap} and {@code sizeMap}. */ - private GridDhtPartitionsSingleMessage fromUpdateCounters(Map> countersMap) { + private GridDhtPartitionsSingleMessage from(@Nullable Map> countersMap, @Nullable Map sizesMap) { GridDhtPartitionsSingleMessage msg = new GridDhtPartitionsSingleMessage(); - msg.addPartitionUpdateCounters(0, countersMap); - return msg; - } - - /** - * @return Message containing specified {@code sizesMap}. - */ - private GridDhtPartitionsSingleMessage fromCacheSizes(Map sizesMap) { - GridDhtPartitionsSingleMessage msg = new GridDhtPartitionsSingleMessage(); - msg.addPartitionSizes(0, sizesMap); + if (countersMap != null) + msg.addPartitionUpdateCounters(0, countersMap); + if (sizesMap != null) + msg.addPartitionSizes(0, sizesMap); return msg; } @@ -98,15 +96,22 @@ public void testPartitionCountersValidation() { UUID remoteNode = UUID.randomUUID(); UUID ignoreNode = UUID.randomUUID(); - // For partitions 0 and 2 (zero counter) we have inconsistent update counters. + // For partitions 0 and 2 we have inconsistent update counters. Map> updateCountersMap = new HashMap<>(); updateCountersMap.put(0, new T2<>(2L, 2L)); updateCountersMap.put(1, new T2<>(2L, 2L)); + updateCountersMap.put(2, new T2<>(5L, 5L)); + + // For partitions 0 and 2 we have inconsistent cache sizes. + Map cacheSizesMap = new HashMap<>(); + cacheSizesMap.put(0, 2L); + cacheSizesMap.put(1, 2L); + cacheSizesMap.put(2, 2L); // Form single messages map. Map messages = new HashMap<>(); - messages.put(remoteNode, fromUpdateCounters(updateCountersMap)); - messages.put(ignoreNode, fromUpdateCounters(updateCountersMap)); + messages.put(remoteNode, from(updateCountersMap, cacheSizesMap)); + messages.put(ignoreNode, from(updateCountersMap, cacheSizesMap)); GridDhtPartitionsStateValidator validator = new GridDhtPartitionsStateValidator(cctxMock); @@ -120,7 +125,7 @@ public void testPartitionCountersValidation() { Assert.assertTrue(result.get(0).get(localNodeId) == 1L); Assert.assertTrue(result.get(0).get(remoteNode) == 2L); Assert.assertTrue(result.get(2).get(localNodeId) == 3L); - Assert.assertTrue(result.get(2).get(remoteNode) == 0L); + Assert.assertTrue(result.get(2).get(remoteNode) == 5L); } /** @@ -130,6 +135,12 @@ public void testPartitionCacheSizesValidation() { UUID remoteNode = UUID.randomUUID(); UUID ignoreNode = UUID.randomUUID(); + // For partitions 0 and 2 we have inconsistent update counters. + Map> updateCountersMap = new HashMap<>(); + updateCountersMap.put(0, new T2<>(2L, 2L)); + updateCountersMap.put(1, new T2<>(2L, 2L)); + updateCountersMap.put(2, new T2<>(5L, 5L)); + // For partitions 0 and 2 we have inconsistent cache sizes. Map cacheSizesMap = new HashMap<>(); cacheSizesMap.put(0, 2L); @@ -138,8 +149,8 @@ public void testPartitionCacheSizesValidation() { // Form single messages map. Map messages = new HashMap<>(); - messages.put(remoteNode, fromCacheSizes(cacheSizesMap)); - messages.put(ignoreNode, fromCacheSizes(cacheSizesMap)); + messages.put(remoteNode, from(updateCountersMap, cacheSizesMap)); + messages.put(ignoreNode, from(updateCountersMap, cacheSizesMap)); GridDhtPartitionsStateValidator validator = new GridDhtPartitionsStateValidator(cctxMock); diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStandByClusterSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStandByClusterSuite.java index 6039ae37a1539..fd124b72b1796 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStandByClusterSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStandByClusterSuite.java @@ -47,7 +47,7 @@ public class IgniteStandByClusterSuite extends TestSuite { * @return Test suite. */ public static TestSuite suite() { - TestSuite suite = new TestSuite("Ignite Activate/DeActivate Cluster Test Suit"); + TestSuite suite = new TestSuite("Ignite Activate/DeActivate Cluster Test Suite"); suite.addTestSuite(IgniteClusterActivateDeactivateTest.class); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java index fc1cea69653b4..dd495cfb216a9 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java @@ -73,6 +73,8 @@ public class IgniteCacheQueryNodeRestartSelfTest extends GridCacheAbstractSelfTe @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { IgniteConfiguration c = super.getConfiguration(igniteInstanceName); + c.setConsistentId(igniteInstanceName); + TcpDiscoverySpi disco = new TcpDiscoverySpi(); disco.setIpFinder(ipFinder); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteChangingBaselineCacheQueryNodeRestartSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteChangingBaselineCacheQueryNodeRestartSelfTest.java index 8e049acc7d4fd..3ee19d5904478 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteChangingBaselineCacheQueryNodeRestartSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteChangingBaselineCacheQueryNodeRestartSelfTest.java @@ -58,7 +58,7 @@ public class IgniteChangingBaselineCacheQueryNodeRestartSelfTest extends IgniteC initStoreStrategy(); - grid(0).active(true); + grid(0).cluster().active(true); awaitPartitionMapExchange(); } @@ -74,7 +74,7 @@ public class IgniteChangingBaselineCacheQueryNodeRestartSelfTest extends IgniteC @Override protected IgniteInternalFuture createRestartAction(final AtomicBoolean done, final AtomicInteger restartCnt) throws Exception { return multithreadedAsync(new Callable() { /** */ - private final long baselineTopChangeInterval = 30 * 1000; + private final long baselineTopChangeInterval = 10 * 1000; /** */ private final int logFreq = 50; From 4ab86f13cbc7fef68f5fd39354e242fe6cca285a Mon Sep 17 00:00:00 2001 From: skalashnikov Date: Wed, 18 Apr 2018 16:37:20 +0300 Subject: [PATCH 0078/1463] IGNITE-7512 Check for null before validateKeyAndValue in GridDhtAtomicCache.updateWithBatch - Fixes #3429. Signed-off-by: Alexey Goncharuk --- .../dht/atomic/GridDhtAtomicCache.java | 3 +- .../local/atomic/GridLocalAtomicCache.java | 2 +- .../transactions/IgniteTxLocalAdapter.java | 5 +- .../query/IgniteSqlNotNullConstraintTest.java | 71 ++++++++++++++++++- 4 files changed, 77 insertions(+), 4 deletions(-) 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 e3ad38205783c..44f2b153db3ad 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 @@ -2071,7 +2071,8 @@ private DhtAtomicUpdateResult updateWithBatch( validation = true; - ctx.validateKeyAndValue(entry.key(), updated); + if (updated != null) + ctx.validateKeyAndValue(entry.key(), updated); } } catch (Exception e) { 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 6cb50f20733c5..dad105231e36e 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 @@ -1074,7 +1074,7 @@ private Map updateWithBatch( if (computed != null) invokeRes = CacheInvokeResult.fromResult(ctx.unwrapTemporary(computed)); - if (invokeEntry.modified()) { + if (invokeEntry.modified() && updated != null) { validation = true; ctx.validateKeyAndValue(entry.key(), updated); 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 c2a284239ee38..7a7f65f7fb9e7 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 @@ -1256,7 +1256,10 @@ protected final void addInvokeResult(IgniteTxEntry txEntry, key0 = invokeEntry.key(); } - ctx.validateKeyAndValue(txEntry.key(), ctx.toCacheObject(val0)); + val0 = ctx.toCacheObject(val0); + + if (val0 != null) + ctx.validateKeyAndValue(txEntry.key(), (CacheObject)val0); if (res != null) ret.addEntryProcessResult(ctx, txEntry.key(), key0, res, null, txEntry.keepBinary()); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java index 1f4e018889a3b..183138b82dc60 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java @@ -680,6 +680,72 @@ key1, new TestEntryProcessor(okValue), }); } + /** */ + public void testAtomicOrImplicitTxInvokeDelete() throws Exception { + executeWithAllCaches(new TestClosure() { + @Override public void run() throws Exception { + cache.put(key1, okValue); + + cache.invoke(key1, new TestEntryProcessor(null)); + + assertEquals(0, cache.size()); + } + }); + } + + /** */ + public void testAtomicOrImplicitTxInvokeAllDelete() throws Exception { + executeWithAllCaches(new TestClosure() { + @Override public void run() throws Exception { + cache.put(key1, okValue); + cache.put(key2, okValue); + + cache.invokeAll(F.asMap( + key1, new TestEntryProcessor(null), + key2, new TestEntryProcessor(null))); + + assertEquals(0, cache.size()); + } + }); + } + + /** */ + public void testTxInvokeDelete() throws Exception { + executeWithAllTxCaches(new TestClosure() { + @Override public void run() throws Exception { + cache.put(key1, okValue); + + try (Transaction tx = ignite.transactions().txStart(concurrency, isolation)) { + cache.invoke(key1, new TestEntryProcessor(null)); + + tx.commit(); + } + + assertEquals(0, cache.size()); + } + }); + } + + /** */ + public void testTxInvokeAllDelete() throws Exception { + executeWithAllTxCaches(new TestClosure() { + @Override public void run() throws Exception { + cache.put(key1, okValue); + cache.put(key2, okValue); + + try (Transaction tx = ignite.transactions().txStart(concurrency, isolation)) { + cache.invokeAll(F.asMap( + key1, new TestEntryProcessor(null), + key2, new TestEntryProcessor(null))); + + tx.commit(); + } + + assertEquals(0, cache.size()); + } + }); + } + /** */ public void testDynamicTableCreateNotNullFieldsAllowed() throws Exception { executeSql("CREATE TABLE test(id INT PRIMARY KEY, field INT NOT NULL)"); @@ -1137,7 +1203,10 @@ public TestEntryProcessor(Person value) { @Override public Object process(MutableEntry entry, Object... objects) throws EntryProcessorException { - entry.setValue(value); + if (value == null) + entry.remove(); + else + entry.setValue(value); return null; } From 7f318da5b357cca1a7e483a37d2556fbd56fc2a7 Mon Sep 17 00:00:00 2001 From: Ilya Lantukh Date: Wed, 18 Apr 2018 17:56:56 +0300 Subject: [PATCH 0079/1463] IGNITE-8017 Disable WAL during initial rebalancing --- .../apache/ignite/IgniteSystemProperties.java | 7 + .../processors/cache/CacheGroupContext.java | 54 +- .../processors/cache/WalStateManager.java | 160 ++++- .../dht/GridClientPartitionTopology.java | 5 + .../dht/GridDhtPartitionTopology.java | 7 + .../dht/GridDhtPartitionTopologyImpl.java | 27 + .../preloader/GridDhtPartitionDemander.java | 24 +- .../GridDhtPartitionsExchangeFuture.java | 2 + .../GridCacheDatabaseSharedManager.java | 63 +- .../IgniteCacheDatabaseSharedManager.java | 4 +- .../file/FilePageStoreManager.java | 18 +- ...alModeChangeDuringRebalancingSelfTest.java | 566 ++++++++++++++++++ .../testsuites/IgnitePdsTestSuite2.java | 3 + 13 files changed, 890 insertions(+), 50 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.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 437f49f6a0a9c..32fed05b369ab 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -877,6 +877,13 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD = "IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD"; + /** + * When set to {@code true}, WAL will be automatically disabled during rebalancing if there is no partition in + * OWNING state. + * Default is {@code false}. + */ + public static final String IGNITE_DISABLE_WAL_DURING_REBALANCING = "IGNITE_DISABLE_WAL_DURING_REBALANCING"; + /** * Enforces singleton. */ 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 12636f3bd0ef5..849ecc8196eeb 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 @@ -152,7 +152,10 @@ public class CacheGroupContext { private CacheGroupMetricsMXBean mxBean; /** */ - private volatile boolean walEnabled; + private volatile boolean localWalEnabled; + + /** */ + private volatile boolean globalWalEnabled; /** * @param grpId Group ID. @@ -196,9 +199,10 @@ public class CacheGroupContext { this.reuseList = reuseList; this.locStartVer = locStartVer; this.cacheType = cacheType; - this.walEnabled = walEnabled; + this.globalWalEnabled = walEnabled; + this.localWalEnabled = true; - persistWalState(walEnabled); + persistGlobalWalState(walEnabled); ioPlc = cacheType.ioPolicy(); @@ -1021,22 +1025,52 @@ public CacheGroupMetricsMXBean mxBean() { * WAL enabled flag. */ public boolean walEnabled() { - return walEnabled; + return localWalEnabled && globalWalEnabled; + } + + /** + * Local WAL enabled flag. + */ + public boolean localWalEnabled() { + return localWalEnabled; + } + + /** + * @Global WAL enabled flag. + */ + public boolean globalWalEnabled() { + return globalWalEnabled; } /** - * @param enabled WAL enabled flag. + * @param enabled Global WAL enabled flag. */ - public void walEnabled(boolean enabled) { - persistWalState(enabled); + public void globalWalEnabled(boolean enabled) { + persistGlobalWalState(enabled); - this.walEnabled = enabled; + this.globalWalEnabled = enabled; + } + + /** + * @param enabled Local WAL enabled flag. + */ + public void localWalEnabled(boolean enabled) { + persistLocalWalState(enabled); + + this.localWalEnabled = enabled; + } + + /** + * @param enabled Enabled flag.. + */ + private void persistGlobalWalState(boolean enabled) { + shared().database().walEnabled(grpId, enabled, false); } /** * @param enabled Enabled flag.. */ - private void persistWalState(boolean enabled) { - shared().database().walEnabled(grpId, enabled); + private void persistLocalWalState(boolean enabled) { + shared().database().walEnabled(grpId, enabled, true); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java index 64a6819826357..4a14730964197 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java @@ -18,7 +18,9 @@ package org.apache.ignite.internal.processors.cache; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.IgniteConfiguration; @@ -26,10 +28,13 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.managers.communication.GridMessageListener; +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.persistence.CheckpointFuture; import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashSet; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.lang.IgniteInClosureX; 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; @@ -44,15 +49,18 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.LinkedList; import java.util.Map; +import java.util.Set; import java.util.UUID; import static org.apache.ignite.internal.GridTopic.TOPIC_WAL; import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL; +import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.OWNING; /** * Write-ahead log state manager. Manages WAL enable and disable. @@ -102,6 +110,9 @@ public class WalStateManager extends GridCacheSharedManagerAdapter { /** Disconnected flag. */ private boolean disconnected; + /** Holder for groups with temporary disabled WAL. */ + private volatile TemporaryDisabledWal tmpDisabledWal; + /** * Constructor. * @@ -327,6 +338,126 @@ else if (!F.eq(grpDesc.deploymentId(), curGrpDesc.deploymentId())) { } } + /** + * Change local WAL state before exchange is done. This method will disable WAL for groups without partitions + * in OWNING state if such feature is enabled. + * + * @param topVer Topology version. + */ + public void changeLocalStatesOnExchangeDone(AffinityTopologyVersion topVer) { + if (!IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_DISABLE_WAL_DURING_REBALANCING, false)) + return; + + Set grpsToEnableWal = new HashSet<>(); + Set grpsToDisableWal = new HashSet<>(); + Set grpsWithWalDisabled = new HashSet<>(); + + boolean hasNonEmptyOwning = false; + + for (CacheGroupContext grp : cctx.cache().cacheGroups()) { + if (grp.isLocal() || !grp.affinityNode() || !grp.persistenceEnabled()) + continue; + + boolean hasOwning = false; + + for (GridDhtLocalPartition locPart : grp.topology().currentLocalPartitions()) { + if (locPart.state() == OWNING) { + hasOwning = true; + + if (hasNonEmptyOwning) + break; + + if (locPart.updateCounter() > 0) { + hasNonEmptyOwning = true; + + break; + } + } + } + + if (hasOwning && !grp.localWalEnabled()) { + grpsToEnableWal.add(grp.groupId()); + } + else if (!hasOwning && grp.localWalEnabled()) { + grpsToDisableWal.add(grp.groupId()); + + grpsWithWalDisabled.add(grp.groupId()); + } + else if (!grp.localWalEnabled()) + grpsWithWalDisabled.add(grp.groupId()); + } + + tmpDisabledWal = new TemporaryDisabledWal(grpsWithWalDisabled, topVer); + + if (grpsToEnableWal.isEmpty() && grpsToDisableWal.isEmpty()) + return; + + try { + if (hasNonEmptyOwning && !grpsToEnableWal.isEmpty()) + triggerCheckpoint(0).finishFuture().get(); + } + catch (IgniteCheckedException ex) { + throw new IgniteException(ex); + } + + for (Integer grpId : grpsToEnableWal) + cctx.cache().cacheGroup(grpId).localWalEnabled(true); + + for (Integer grpId : grpsToDisableWal) + cctx.cache().cacheGroup(grpId).localWalEnabled(false); + } + + /** + * Callback when group rebalancing is finished. If there are no pending groups, it should trigger checkpoint and + * change partition states. + * @param grpId Group ID. + * @param topVer Topology version. + */ + public void onGroupRebalanceFinished(int grpId, AffinityTopologyVersion topVer) { + TemporaryDisabledWal session0 = tmpDisabledWal; + + if (session0 == null || !session0.topVer.equals(topVer)) + return; + + session0.remainingGrps.remove(grpId); + + if (session0.remainingGrps.isEmpty()) { + synchronized (mux) { + if (tmpDisabledWal != session0) + return; + + for (Integer grpId0 : session0.disabledGrps) { + CacheGroupContext grp = cctx.cache().cacheGroup(grpId0); + + assert grp != null; + + if (!grp.localWalEnabled()) + grp.localWalEnabled(true); + } + + tmpDisabledWal = null; + } + + CheckpointFuture cpFut = triggerCheckpoint(0); + + assert cpFut != null; + + cpFut.finishFuture().listen(new IgniteInClosureX() { + @Override public void applyx(IgniteInternalFuture future) { + for (Integer grpId0 : session0.disabledGrps) { + CacheGroupContext grp = cctx.cache().cacheGroup(grpId0); + + assert grp != null; + + grp.topology().ownMoving(session0.topVer); + } + + cctx.exchange().refreshPartitions(); + } + }); + } + } + /** * Handle propose message in discovery thread. * @@ -455,7 +586,7 @@ public void onProposeExchange(WalStateProposeMessage msg) { "no longer exist: " + msg.caches().keySet()); } else { - if (F.eq(msg.enable(), grpCtx.walEnabled())) + if (F.eq(msg.enable(), grpCtx.globalWalEnabled())) // Nothing changed -> no-op. res = new WalStateResult(msg, false); else { @@ -468,7 +599,7 @@ public void onProposeExchange(WalStateProposeMessage msg) { cpFut.beginFuture().get(); if (msg.enable()) { - grpCtx.walEnabled(true); + grpCtx.globalWalEnabled(true); // Enable: it is enough to release cache operations once mark is finished because // not-yet-flushed dirty pages have been logged. @@ -489,7 +620,7 @@ public void onProposeExchange(WalStateProposeMessage msg) { // WAL state is persisted after checkpoint if finished. Otherwise in case of crash // and restart we will think that WAL is enabled, but data might be corrupted. - grpCtx.walEnabled(false); + grpCtx.globalWalEnabled(false); } } catch (Exception e) { @@ -917,4 +1048,27 @@ private WalStateChangeWorker(WalStateProposeMessage msg, CheckpointFuture cpFut) onCompletedLocally(res); } } + + /** + * + */ + private static class TemporaryDisabledWal { + /** Groups with disabled WAL. */ + private final Set disabledGrps; + + /** Remaining groups. */ + private final Set remainingGrps; + + /** Topology version*/ + private final AffinityTopologyVersion topVer; + + /** */ + public TemporaryDisabledWal( + Set disabledGrps, + AffinityTopologyVersion topVer) { + this.disabledGrps = Collections.unmodifiableSet(disabledGrps); + this.remainingGrps = new HashSet<>(disabledGrps); + this.topVer = topVer; + } + } } 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 3e3bb0db9b9bb..dcb8b96db26ce 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 @@ -1103,6 +1103,11 @@ private void removeNode(UUID nodeId) { return false; } + /** {@inheritDoc} */ + @Override public void ownMoving(AffinityTopologyVersion topVer) { + // No-op + } + /** {@inheritDoc} */ @Override public void onEvicted(GridDhtLocalPartition part, boolean updateSeq) { assert updateSeq || lock.isWriteLockedByCurrentThread(); 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 d586a94d53203..2df2e8960afc9 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 @@ -356,6 +356,13 @@ public boolean update(@Nullable GridDhtPartitionExchangeId exchId, */ public boolean own(GridDhtLocalPartition part); + /** + * Owns all moving partitions for the given topology version. + * + * @param topVer Topology version. + */ + public void ownMoving(AffinityTopologyVersion topVer); + /** * @param part Evicted partition. * @param updateSeq Update sequence increment flag. 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 164f0bf451286..68104a5349283 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 @@ -2409,6 +2409,33 @@ private void removeNode(UUID nodeId) { } } + /** {@inheritDoc} */ + @Override public void ownMoving(AffinityTopologyVersion topVer) { + lock.writeLock().lock(); + + try { + for (GridDhtLocalPartition locPart : grp.topology().currentLocalPartitions()) { + if (locPart.state() == MOVING) { + boolean reserved = locPart.reserve(); + + try { + if (reserved && locPart.state() == MOVING && lastTopChangeVer.equals(topVer)) + grp.topology().own(locPart); + else // topology changed, rebalancing must be restarted + return; + } + finally { + if (reserved) + locPart.release(); + } + } + } + } + finally { + lock.writeLock().unlock(); + } + } + /** {@inheritDoc} */ @Override public void onEvicted(GridDhtLocalPartition part, boolean updateSeq) { ctx.database().checkpointReadLock(); 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 337553b28cfdc..dc4bfe9bc1720 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 @@ -57,6 +57,7 @@ import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.lang.IgniteInClosureX; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.CI1; @@ -282,6 +283,14 @@ Runnable addAssignments( final RebalanceFuture fut = new RebalanceFuture(grp, assignments, log, rebalanceId); + if (!grp.localWalEnabled()) + fut.listen(new IgniteInClosureX>() { + @Override public void applyx(IgniteInternalFuture future) throws IgniteCheckedException { + if (future.get()) + ctx.walState().onGroupRebalanceFinished(grp.groupId(), assignments.topologyVersion()); + } + }); + if (!oldFut.isInitial()) oldFut.cancel(); else @@ -722,9 +731,7 @@ public void handleSupplyMessage( // If message was last for this partition, // then we take ownership. if (last) { - top.own(part); - - fut.partitionDone(nodeId, p); + fut.partitionDone(nodeId, p, true); if (log.isDebugEnabled()) log.debug("Finished rebalancing partition: " + part); @@ -737,14 +744,14 @@ public void handleSupplyMessage( } else { if (last) - fut.partitionDone(nodeId, p); + fut.partitionDone(nodeId, p, false); if (log.isDebugEnabled()) log.debug("Skipping rebalancing partition (state is not MOVING): " + part); } } else { - fut.partitionDone(nodeId, p); + fut.partitionDone(nodeId, p, false); if (log.isDebugEnabled()) log.debug("Skipping rebalancing partition (it does not belong on current node): " + p); @@ -762,7 +769,7 @@ public void handleSupplyMessage( } for (Integer miss : supply.missed()) - fut.partitionDone(nodeId, miss); + fut.partitionDone(nodeId, miss, false); GridDhtPartitionDemandMessage d = new GridDhtPartitionDemandMessage( supply.rebalanceId(), @@ -1064,8 +1071,11 @@ private void cleanupRemoteContexts(UUID nodeId) { * @param nodeId Node id. * @param p Partition number. */ - private void partitionDone(UUID nodeId, int p) { + private void partitionDone(UUID nodeId, int p, boolean updateState) { synchronized (this) { + if (updateState && grp.localWalEnabled()) + grp.topology().own(grp.topology().localPartition(p)); + if (isDone()) return; 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 0d57d483f71a9..a21d98e67d320 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 @@ -1702,6 +1702,8 @@ public void finishMerged() { if (!grp.isLocal()) grp.topology().onExchangeDone(this, grp.affinity().readyAffinity(res), false); } + + cctx.walState().changeLocalStatesOnExchangeDone(exchId.topologyVersion()); } if (super.onDone(res, err)) { 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 3009febae9b44..59aad5f83ade0 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 @@ -21,6 +21,7 @@ import java.io.FileFilter; import java.io.IOException; import java.io.RandomAccessFile; +import java.io.Serializable; import java.lang.ref.SoftReference; import java.nio.ByteBuffer; import java.nio.ByteOrder; @@ -279,7 +280,13 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan private static final String MBEAN_GROUP = "Persistent Store"; /** WAL marker prefix for meta store. */ - private static final String WAL_KEY_PREFIX = "grp-wal-disabled-"; + private static final String WAL_KEY_PREFIX = "grp-wal-"; + + /** WAL marker prefix for meta store. */ + private static final String WAL_GLOBAL_KEY_PREFIX = WAL_KEY_PREFIX + "disabled-"; + + /** WAL marker prefix for meta store. */ + private static final String WAL_LOCAL_KEY_PREFIX = WAL_KEY_PREFIX + "local-disabled-"; /** WAL marker predicate for meta store. */ private static final IgnitePredicate WAL_KEY_PREFIX_PRED = new IgnitePredicate() { @@ -379,7 +386,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan private List metastorageLifecycleLsnrs; /** Initially disabled cache groups. */ - private Collection initiallyWalDisabledGrps; + private Collection initiallyGlobalWalDisabledGrps = new HashSet<>(); + + private Collection initiallyLocalWalDisabledGrps = new HashSet<>(); /** * @param ctx Kernal context. @@ -563,7 +572,7 @@ private void readMetastore() throws IgniteCheckedException { applyLastUpdates(status, true); - initiallyWalDisabledGrps = walDisabledGroups(); + fillWalDisabledGroups(); notifyMetastorageReadyForRead(); } @@ -1935,7 +1944,8 @@ private WALPointer restoreMemory(CheckpointStatus status, boolean storeOnly, int applied = 0; WALPointer lastRead = null; - Collection ignoreGrps = storeOnly ? Collections.emptySet() : initiallyWalDisabledGrps; + Collection ignoreGrps = storeOnly ? Collections.emptySet() : + F.concat(false, initiallyGlobalWalDisabledGrps, initiallyLocalWalDisabledGrps); try (WALIterator it = cctx.wal().replay(status.endPtr)) { while (it.hasNextX()) { @@ -2189,7 +2199,8 @@ private void applyLastUpdates(CheckpointStatus status, boolean metastoreOnly) th long start = U.currentTimeMillis(); int applied = 0; - Collection ignoreGrps = metastoreOnly ? Collections.emptySet() : initiallyWalDisabledGrps; + Collection ignoreGrps = metastoreOnly ? Collections.emptySet() : + F.concat(false, initiallyGlobalWalDisabledGrps, initiallyLocalWalDisabledGrps); try (WALIterator it = cctx.wal().replay(status.startPtr)) { Map, T2> partStates = new HashMap<>(); @@ -4340,13 +4351,16 @@ public DataStorageMetricsImpl persistentStoreMetricsImpl() { } /** {@inheritDoc} */ - @Override public boolean walEnabled(int grpId) { - return !initiallyWalDisabledGrps.contains(grpId); + @Override public boolean walEnabled(int grpId, boolean local) { + if (local) + return !initiallyLocalWalDisabledGrps.contains(grpId); + else + return !initiallyGlobalWalDisabledGrps.contains(grpId); } /** {@inheritDoc} */ - @Override public void walEnabled(int grpId, boolean enabled) { - String key = walGroupIdToKey(grpId); + @Override public void walEnabled(int grpId, boolean enabled, boolean local) { + String key = walGroupIdToKey(grpId, local); checkpointReadLock(); @@ -4366,27 +4380,26 @@ public DataStorageMetricsImpl persistentStoreMetricsImpl() { } /** - * @return List of initially WAL-disabled groups. + * */ - private Collection walDisabledGroups() { + private void fillWalDisabledGroups() { MetaStorage meta = cctx.database().metaStorage(); try { Set keys = meta.readForPredicate(WAL_KEY_PREFIX_PRED).keySet(); if (keys.isEmpty()) - return Collections.emptySet(); - - HashSet res = new HashSet<>(keys.size()); + return; for (String key : keys) { - int grpId = walKeyToGroupId(key); + T2 t2 = walKeyToGroupIdAndLocalFlag(key); - res.add(grpId); + if (t2.get2()) + initiallyLocalWalDisabledGrps.add(t2.get1()); + else + initiallyGlobalWalDisabledGrps.add(t2.get1()); } - return res; - } catch (IgniteCheckedException e) { throw new IgniteException("Failed to read cache groups WAL state.", e); @@ -4399,8 +4412,11 @@ private Collection walDisabledGroups() { * @param grpId Group ID. * @return Key. */ - private static String walGroupIdToKey(int grpId) { - return WAL_KEY_PREFIX + grpId; + private static String walGroupIdToKey(int grpId, boolean local) { + if (local) + return WAL_LOCAL_KEY_PREFIX + grpId; + else + return WAL_GLOBAL_KEY_PREFIX + grpId; } /** @@ -4409,7 +4425,10 @@ private static String walGroupIdToKey(int grpId) { * @param key Key. * @return Group ID. */ - private static int walKeyToGroupId(String key) { - return Integer.parseInt(key.substring(WAL_KEY_PREFIX.length())); + private static T2 walKeyToGroupIdAndLocalFlag(String key) { + if (key.startsWith(WAL_LOCAL_KEY_PREFIX)) + return new T2<>(Integer.parseInt(key.substring(WAL_LOCAL_KEY_PREFIX.length())), true); + else + return new T2<>(Integer.parseInt(key.substring(WAL_GLOBAL_KEY_PREFIX.length())), 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 8746dca71977b..bf080b6736d0d 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 @@ -1035,7 +1035,7 @@ public MetaStorage metaStorage() { * @param grpId Group ID. * @return WAL enabled flag. */ - public boolean walEnabled(int grpId) { + public boolean walEnabled(int grpId, boolean local) { return false; } @@ -1045,7 +1045,7 @@ public boolean walEnabled(int grpId) { * @param grpId Group id. * @param enabled flag. */ - public void walEnabled(int grpId, boolean enabled) { + public void walEnabled(int grpId, boolean enabled, boolean local) { // No-op. } } 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 837f3d01b2b2c..1c1b3e24e175d 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 @@ -845,16 +845,22 @@ public PageStore getStore(int grpId, int partId) throws IgniteCheckedException { /** {@inheritDoc} */ @Override public void beforeCacheGroupStart(CacheGroupDescriptor grpDesc) { - if (grpDesc.persistenceEnabled() && !cctx.database().walEnabled(grpDesc.groupId())) { - File dir = cacheWorkDir(grpDesc.config()); + if (grpDesc.persistenceEnabled()) { + boolean localEnabled = cctx.database().walEnabled(grpDesc.groupId(), true); + boolean globalEnabled = cctx.database().walEnabled(grpDesc.groupId(), false); - assert dir.exists(); + if (!localEnabled || !globalEnabled) { + File dir = cacheWorkDir(grpDesc.config()); - boolean res = IgniteUtils.delete(dir); + assert dir.exists(); - assert res; + boolean res = IgniteUtils.delete(dir); - grpDesc.walEnabled(false); + assert res; + + if (!globalEnabled) + grpDesc.walEnabled(false); + } } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java new file mode 100644 index 0000000000000..aa2613fa3a1a2 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java @@ -0,0 +1,566 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.MappedByteBuffer; +import java.nio.file.OpenOption; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.cluster.ClusterNode; +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.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage; +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.util.lang.GridAbsPredicate; +import org.apache.ignite.internal.util.typedef.G; +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.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; + +/** + * + */ +public class LocalWalModeChangeDuringRebalancingSelfTest extends GridCommonAbstractTest { + /** */ + private static boolean disableWalDuringRebalancing = true; + + /** */ + private static final AtomicReference supplyMessageLatch = new AtomicReference<>(); + + /** */ + private static final AtomicReference fileIOLatch = new AtomicReference<>(); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setDataStorageConfiguration( + new DataStorageConfiguration() + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration() + .setPersistenceEnabled(true) + .setMaxSize(200 * 1024 * 1024) + .setInitialSize(200 * 1024 * 1024) + ) + // Test verifies checkpoint count, so it is essencial that no checkpoint is triggered by timeout + .setCheckpointFrequency(999_999_999_999L) + .setFileIOFactory(new TestFileIOFactory(new DataStorageConfiguration().getFileIOFactory())) + ); + + cfg.setCacheConfiguration( + new CacheConfiguration(DEFAULT_CACHE_NAME) + // Test checks internal state before and after rebalance, so it is configured to be triggered manually + .setRebalanceDelay(-1) + ); + + cfg.setCommunicationSpi(new TcpCommunicationSpi() { + @Override public void sendMessage(ClusterNode node, Message msg) throws IgniteSpiException { + if (msg instanceof GridIoMessage && ((GridIoMessage)msg).message() instanceof GridDhtPartitionSupplyMessage) { + int grpId = ((GridDhtPartitionSupplyMessage)((GridIoMessage)msg).message()).groupId(); + + if (grpId == CU.cacheId(DEFAULT_CACHE_NAME)) { + CountDownLatch latch0 = supplyMessageLatch.get(); + + if (latch0 != null) + try { + latch0.await(); + } + catch (InterruptedException ex) { + throw new IgniteException(ex); + } + } + } + + super.sendMessage(node, msg); + } + + @Override public void sendMessage(ClusterNode node, Message msg, + IgniteInClosure ackC) throws IgniteSpiException { + if (msg instanceof GridIoMessage && ((GridIoMessage)msg).message() instanceof GridDhtPartitionSupplyMessage) { + int grpId = ((GridDhtPartitionSupplyMessage)((GridIoMessage)msg).message()).groupId(); + + if (grpId == CU.cacheId(DEFAULT_CACHE_NAME)) { + CountDownLatch latch0 = supplyMessageLatch.get(); + + if (latch0 != null) + try { + latch0.await(); + } + catch (InterruptedException ex) { + throw new IgniteException(ex); + } + } + } + + super.sendMessage(node, msg, ackC); + } + }); + + cfg.setConsistentId(igniteInstanceName); + + System.setProperty(IgniteSystemProperties.IGNITE_DISABLE_WAL_DURING_REBALANCING, + Boolean.toString(disableWalDuringRebalancing)); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + CountDownLatch msgLatch = supplyMessageLatch.get(); + + if (msgLatch != null) { + while (msgLatch.getCount() > 0) + msgLatch.countDown(); + + supplyMessageLatch.set(null); + } + + CountDownLatch fileLatch = fileIOLatch.get(); + + if (fileLatch != null) { + while (fileLatch.getCount() > 0) + fileLatch.countDown(); + + fileIOLatch.set(null); + } + + stopAllGrids(); + + cleanPersistenceDir(); + + disableWalDuringRebalancing = true; + } + + /** + * @throws Exception If failed. + */ + public void testWalDisabledDuringRebalancing() throws Exception { + doTestSimple(); + } + + /** + * @throws Exception If failed. + */ + public void testWalNotDisabledIfParameterSetToFalse() throws Exception { + disableWalDuringRebalancing = false; + + doTestSimple(); + } + + /** + * @throws Exception If failed. + */ + private void doTestSimple() throws Exception { + Ignite ignite = startGrids(3); + + ignite.cluster().active(true); + + IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME); + + for (int k = 0; k < 10_000; k++) + cache.put(k, k); + + IgniteEx newIgnite = startGrid(3); + + final GridCacheDatabaseSharedManager.CheckpointHistory cpHistory = + ((GridCacheDatabaseSharedManager)newIgnite.context().cache().context().database()).checkpointHistory(); + + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return !cpHistory.checkpoints().isEmpty(); + } + }, 10_000); + + U.sleep(10); // To ensure timestamp granularity. + + long newIgniteStartedTimestamp = System.currentTimeMillis(); + + ignite.cluster().setBaselineTopology(4); + + CacheGroupContext grpCtx = newIgnite.cachex(DEFAULT_CACHE_NAME).context().group(); + + assertEquals(!disableWalDuringRebalancing, grpCtx.walEnabled()); + + U.sleep(10); // To ensure timestamp granularity. + + long rebalanceStartedTimestamp = System.currentTimeMillis(); + + for (Ignite g : G.allGrids()) + g.cache(DEFAULT_CACHE_NAME).rebalance(); + + awaitPartitionMapExchange(); + + assertTrue(grpCtx.walEnabled()); + + U.sleep(10); // To ensure timestamp granularity. + + long rebalanceFinishedTimestamp = System.currentTimeMillis(); + + for (Integer k = 0; k < 1000; k++) + assertEquals("k=" + k, k, cache.get(k)); + + int checkpointsBeforeNodeStarted = 0; + int checkpointsBeforeRebalance = 0; + int checkpointsAfterRebalance = 0; + + for (Long timestamp : cpHistory.checkpoints()) { + if (timestamp < newIgniteStartedTimestamp) + checkpointsBeforeNodeStarted++; + else if (timestamp >= newIgniteStartedTimestamp && timestamp < rebalanceStartedTimestamp) + checkpointsBeforeRebalance++; + else if (timestamp >= rebalanceStartedTimestamp && timestamp <= rebalanceFinishedTimestamp) + checkpointsAfterRebalance++; + } + + assertEquals(1, checkpointsBeforeNodeStarted); // checkpoint on start + assertEquals(0, checkpointsBeforeRebalance); + assertEquals(disableWalDuringRebalancing ? 1 : 0, checkpointsAfterRebalance); // checkpoint if WAL was re-activated + } + + /** + * @throws Exception If failed. + */ + public void testLocalAndGlobalWalStateInterdependence() throws Exception { + Ignite ignite = startGrids(3); + + ignite.cluster().active(true); + + IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME); + + for (int k = 0; k < 10_000; k++) + cache.put(k, k); + + IgniteEx newIgnite = startGrid(3); + + ignite.cluster().setBaselineTopology(ignite.cluster().nodes()); + + CacheGroupContext grpCtx = newIgnite.cachex(DEFAULT_CACHE_NAME).context().group(); + + assertFalse(grpCtx.walEnabled()); + + ignite.cluster().disableWal(DEFAULT_CACHE_NAME); + + for (Ignite g : G.allGrids()) + g.cache(DEFAULT_CACHE_NAME).rebalance(); + + awaitPartitionMapExchange(); + + assertFalse(grpCtx.walEnabled()); // WAL is globally disabled + + ignite.cluster().enableWal(DEFAULT_CACHE_NAME); + + assertTrue(grpCtx.walEnabled()); + } + + /** + * @throws Exception If failed. + */ + public void testParallelExchangeDuringRebalance() throws Exception { + doTestParallelExchange(supplyMessageLatch); + } + + /** + * @throws Exception If failed. + */ + public void testParallelExchangeDuringCheckpoint() throws Exception { + doTestParallelExchange(fileIOLatch); + } + + /** + * @throws Exception If failed. + */ + private void doTestParallelExchange(AtomicReference latchRef) throws Exception { + Ignite ignite = startGrids(3); + + ignite.cluster().active(true); + + IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME); + + for (int k = 0; k < 10_000; k++) + cache.put(k, k); + + IgniteEx newIgnite = startGrid(3); + + CacheGroupContext grpCtx = newIgnite.cachex(DEFAULT_CACHE_NAME).context().group(); + + CountDownLatch latch = new CountDownLatch(1); + + latchRef.set(latch); + + ignite.cluster().setBaselineTopology(ignite.cluster().nodes()); + + for (Ignite g : G.allGrids()) + g.cache(DEFAULT_CACHE_NAME).rebalance(); + + assertFalse(grpCtx.walEnabled()); + + // TODO : test with client node as well + startGrid(4); // Trigger exchange + + assertFalse(grpCtx.walEnabled()); + + latch.countDown(); + + assertFalse(grpCtx.walEnabled()); + + for (Ignite g : G.allGrids()) + g.cache(DEFAULT_CACHE_NAME).rebalance(); + + awaitPartitionMapExchange(); + + assertTrue(grpCtx.walEnabled()); + } + + /** + * @throws Exception If failed. + */ + public void testDataClearedAfterRestartWithDisabledWal() throws Exception { + Ignite ignite = startGrid(0); + + ignite.cluster().active(true); + + IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME); + + for (int k = 0; k < 10_000; k++) + cache.put(k, k); + + IgniteEx newIgnite = startGrid(1); + + ignite.cluster().setBaselineTopology(2); + + CacheGroupContext grpCtx = newIgnite.cachex(DEFAULT_CACHE_NAME).context().group(); + + assertFalse(grpCtx.localWalEnabled()); + + stopGrid(1); + stopGrid(0); + + newIgnite = startGrid(1); + + newIgnite.cluster().active(true); + + newIgnite.cluster().setBaselineTopology(newIgnite.cluster().nodes()); + + cache = newIgnite.cache(DEFAULT_CACHE_NAME); + + for (int k = 0; k < 10_000; k++) + assertFalse("k=" + k +", v=" + cache.get(k), cache.containsKey(k)); + } + + /** + * @throws Exception If failed. + */ + public void testWalNotDisabledAfterShrinkingBaselineTopology() throws Exception { + Ignite ignite = startGrids(4); + + ignite.cluster().active(true); + + IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME); + + for (int k = 0; k < 10_000; k++) + cache.put(k, k); + + for (Ignite g : G.allGrids()) { + CacheGroupContext grpCtx = ((IgniteEx)g).cachex(DEFAULT_CACHE_NAME).context().group(); + + assertTrue(grpCtx.walEnabled()); + } + + stopGrid(2); + + ignite.cluster().setBaselineTopology(5); + + for (Ignite g : G.allGrids()) { + CacheGroupContext grpCtx = ((IgniteEx)g).cachex(DEFAULT_CACHE_NAME).context().group(); + + assertTrue(grpCtx.walEnabled()); + + g.cache(DEFAULT_CACHE_NAME).rebalance(); + } + + awaitPartitionMapExchange(); + + for (Ignite g : G.allGrids()) { + CacheGroupContext grpCtx = ((IgniteEx)g).cachex(DEFAULT_CACHE_NAME).context().group(); + + assertTrue(grpCtx.walEnabled()); + } + } + + /** + * + */ + private static class TestFileIOFactory implements FileIOFactory { + /** */ + private final FileIOFactory delegate; + + /** + * @param delegate Delegate. + */ + TestFileIOFactory(FileIOFactory delegate) { + this.delegate = delegate; + } + + /** {@inheritDoc} */ + @Override public FileIO create(File file) throws IOException { + return new TestFileIO(delegate.create(file)); + } + + /** {@inheritDoc} */ + @Override public FileIO create(File file, OpenOption... modes) throws IOException { + return new TestFileIO(delegate.create(file, modes)); + } + } + + /** + * + */ + private static class TestFileIO implements FileIO { + /** */ + private final FileIO delegate; + + /** + * @param delegate Delegate. + */ + TestFileIO(FileIO delegate) { + this.delegate = delegate; + } + + /** {@inheritDoc} */ + @Override public long position() throws IOException { + return delegate.position(); + } + + /** {@inheritDoc} */ + @Override public void position(long newPosition) throws IOException { + delegate.position(newPosition); + } + + /** {@inheritDoc} */ + @Override public int read(ByteBuffer destBuf) throws IOException { + return delegate.read(destBuf); + } + + /** {@inheritDoc} */ + @Override public int read(ByteBuffer destBuf, long position) throws IOException { + return delegate.read(destBuf, position); + } + + /** {@inheritDoc} */ + @Override public int read(byte[] buf, int off, int len) throws IOException { + return delegate.read(buf, off, len); + } + + /** {@inheritDoc} */ + @Override public int write(ByteBuffer srcBuf) throws IOException { + CountDownLatch latch = fileIOLatch.get(); + + if (latch != null && Thread.currentThread().getName().contains("checkpoint")) + try { + latch.await(); + } + catch (InterruptedException ex) { + throw new IgniteException(ex); + } + + return delegate.write(srcBuf); + } + + /** {@inheritDoc} */ + @Override public int write(ByteBuffer srcBuf, long position) throws IOException { + CountDownLatch latch = fileIOLatch.get(); + + if (latch != null && Thread.currentThread().getName().contains("checkpoint")) + try { + latch.await(); + } + catch (InterruptedException ex) { + throw new IgniteException(ex); + } + + return delegate.write(srcBuf, position); + } + + /** {@inheritDoc} */ + @Override public void write(byte[] buf, int off, int len) throws IOException { + CountDownLatch latch = fileIOLatch.get(); + + if (latch != null && Thread.currentThread().getName().contains("checkpoint")) + try { + latch.await(); + } + catch (InterruptedException ex) { + throw new IgniteException(ex); + } + + delegate.write(buf, off, len); + } + + /** {@inheritDoc} */ + @Override public MappedByteBuffer map(int maxWalSegmentSize) throws IOException { + return delegate.map(maxWalSegmentSize); + } + + /** {@inheritDoc} */ + @Override public void force() throws IOException { + delegate.force(); + } + + /** {@inheritDoc} */ + @Override public long size() throws IOException { + return delegate.size(); + } + + /** {@inheritDoc} */ + @Override public void clear() throws IOException { + delegate.clear(); + } + + /** {@inheritDoc} */ + @Override public void close() throws IOException { + delegate.close(); + } + } +} 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 f955b118d7e92..ede537ebb6df1 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,6 +27,7 @@ import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsPageSizesTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsRecoveryAfterFileCorruptionTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePersistentStoreDataStructuresTest; +import org.apache.ignite.internal.processors.cache.persistence.LocalWalModeChangeDuringRebalancingSelfTest; 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; @@ -135,5 +136,7 @@ public static void addRealPageStoreTests(TestSuite suite) { suite.addTestSuite(IgniteCheckpointDirtyPagesForLowLoadTest.class); suite.addTestSuite(IgnitePdsCorruptedStoreTest.class); + + suite.addTestSuite(LocalWalModeChangeDuringRebalancingSelfTest.class); } } From b80fdbedc11b48aa89c6f29146363cec6f552abb Mon Sep 17 00:00:00 2001 From: Ilya Lantukh Date: Wed, 18 Apr 2018 19:04:39 +0300 Subject: [PATCH 0080/1463] IGNITE-8276 Fixed incorrect assertion during initialValue - Fixes #3827. Signed-off-by: Alexey Goncharuk --- .../ignite/internal/processors/cache/GridCacheMapEntry.java | 4 +--- 1 file changed, 1 insertion(+), 3 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 a6ef0d284d6b4..9f3686aad47ec 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 @@ -3571,12 +3571,10 @@ private IgniteTxLocalAdapter currentTx() { * @param oldRow Old row if available. * @throws IgniteCheckedException If update failed. */ - protected boolean storeValue(CacheObject val, + protected boolean storeValue(@Nullable CacheObject val, long expireTime, GridCacheVersion ver, @Nullable CacheDataRow oldRow) throws IgniteCheckedException { - assert val != null : "null values in update for key: " + key; - return storeValue(val, expireTime, ver, oldRow, null); } From 5cd32329fe5f303eaf369519771ee22f2a2cf822 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 18 Apr 2018 19:41:44 +0300 Subject: [PATCH 0081/1463] IGNITE-8116 Fixed historical rebalance from WAL --- .../cache/IgniteCacheOffheapManagerImpl.java | 28 +-- .../preloader/GridDhtPartitionDemander.java | 118 ++++++++----- .../preloader/GridDhtPartitionSupplier.java | 54 ++++-- .../dht/preloader/GridDhtPreloader.java | 4 +- .../IgniteDhtPartitionsToReloadMap.java | 2 +- .../persistence/GridCacheOffheapManager.java | 82 +++++++-- modules/core/src/test/config/log4j-test.xml | 6 - ...sAtomicCacheHistoricalRebalancingTest.java | 40 +++++ ...IgnitePdsCacheRebalancingAbstractTest.java | 32 ++-- ...tePdsTxCacheHistoricalRebalancingTest.java | 39 +++++ .../db/wal/IgniteWalRebalanceTest.java | 164 ++++++++++++++++++ ...entQueryReplicatedNodeRestartSelfTest.java | 1 - .../IgnitePdsWithIndexingCoreTestSuite.java | 7 + 13 files changed, 467 insertions(+), 110 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsAtomicCacheHistoricalRebalancingTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsTxCacheHistoricalRebalancingTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRebalanceTest.java 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 f8cc86f8cecdb..5c78eb5b4b4a4 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 @@ -861,7 +861,8 @@ private long allocateForTree() throws IgniteCheckedException { } @Override protected void onClose() throws IgniteCheckedException { - assert loc != null && loc.state() == OWNING && loc.reservations() > 0; + assert loc != null && loc.state() == OWNING && loc.reservations() > 0 + : "Partition should be in OWNING state and has at least 1 reservation: " + loc; loc.release(); } @@ -874,36 +875,37 @@ private long allocateForTree() throws IgniteCheckedException { throws IgniteCheckedException { final TreeMap> iterators = new TreeMap<>(); - Set missing = null; + + Set missing = new HashSet<>(); for (Integer p : parts.fullSet()) { GridCloseableIterator partIter = reservedIterator(p, topVer); if (partIter == null) { - if (missing == null) - missing = new HashSet<>(); - missing.add(p); + + continue; } - else - iterators.put(p, partIter); + + iterators.put(p, partIter); } - IgniteRebalanceIterator iter = new IgniteRebalanceIteratorImpl(iterators, historicalIterator(parts.historicalMap())); + IgniteHistoricalIterator historicalIterator = historicalIterator(parts.historicalMap(), missing); - if (missing != null) { - for (Integer p : missing) - iter.setPartitionMissing(p); - } + IgniteRebalanceIterator iter = new IgniteRebalanceIteratorImpl(iterators, historicalIterator); + + for (Integer p : missing) + iter.setPartitionMissing(p); return iter; } /** * @param partCntrs Partition counters map. + * @param missing Set of partitions need to populate if partition is missing or failed to reserve. * @return Historical iterator. */ - @Nullable protected IgniteHistoricalIterator historicalIterator(CachePartitionPartialCountersMap partCntrs) + @Nullable protected IgniteHistoricalIterator historicalIterator(CachePartitionPartialCountersMap partCntrs, Set missing) throws IgniteCheckedException { return null; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java index dc4bfe9bc1720..c94f511d91887 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 @@ -459,7 +459,9 @@ private void requestPartitions(final RebalanceFuture fut, GridDhtPreloaderAssign + ", topology=" + fut.topologyVersion() + ", rebalanceId=" + fut.rebalanceId + "]"); } - int stripes = ctx.gridConfig().getRebalanceThreadPoolSize(); + int totalStripes = ctx.gridConfig().getRebalanceThreadPoolSize(); + + int stripes = totalStripes; final List stripePartitions = new ArrayList<>(stripes); for (int i = 0; i < stripes; i++) @@ -467,7 +469,7 @@ private void requestPartitions(final RebalanceFuture fut, GridDhtPreloaderAssign // Reserve one stripe for historical partitions. if (parts.hasHistorical()) { - stripePartitions.add(stripes - 1, new IgniteDhtDemandedPartitionsMap(parts.historicalMap(), null)); + stripePartitions.set(stripes - 1, new IgniteDhtDemandedPartitionsMap(parts.historicalMap(), null)); if (stripes > 1) stripes--; @@ -478,7 +480,7 @@ private void requestPartitions(final RebalanceFuture fut, GridDhtPreloaderAssign for (int i = 0; it.hasNext(); i++) stripePartitions.get(i % stripes).addFull(it.next()); - for (int stripe = 0; stripe < stripes; stripe++) { + for (int stripe = 0; stripe < totalStripes; stripe++) { if (!stripePartitions.get(stripe).isEmpty()) { // Create copy of demand message with new striped partitions map. final GridDhtPartitionDemandMessage demandMsg = d.withNewPartitionsMap(stripePartitions.get(stripe)); @@ -489,23 +491,27 @@ private void requestPartitions(final RebalanceFuture fut, GridDhtPreloaderAssign final int topicId = stripe; - Runnable initDemandRequestTask = () -> { + IgniteInternalFuture clearAllFuture = clearFullPartitions(fut, demandMsg.partitions().fullSet()); + + // Start rebalancing after clearing full partitions is finished. + clearAllFuture.listen(f -> ctx.kernalContext().closure().runLocalSafe(() -> { + if (fut.isDone()) + return; + try { - if (!fut.isDone()) { - ctx.io().sendOrderedMessage(node, rebalanceTopics.get(topicId), - demandMsg.convertIfNeeded(node.version()), grp.ioPolicy(), demandMsg.timeout()); - - // Cleanup required in case partitions demanded in parallel with cancellation. - synchronized (fut) { - if (fut.isDone()) - fut.cleanupRemoteContexts(node.id()); - } + ctx.io().sendOrderedMessage(node, rebalanceTopics.get(topicId), + demandMsg.convertIfNeeded(node.version()), grp.ioPolicy(), demandMsg.timeout()); - if (log.isDebugEnabled()) - log.debug("Requested rebalancing [from node=" + node.id() + ", listener index=" + - topicId + ", partitions count=" + stripePartitions.get(topicId).size() + - " (" + stripePartitions.get(topicId).partitionsList() + ")]"); + // Cleanup required in case partitions demanded in parallel with cancellation. + synchronized (fut) { + if (fut.isDone()) + fut.cleanupRemoteContexts(node.id()); } + + if (log.isDebugEnabled()) + log.debug("Requested rebalancing [from node=" + node.id() + ", listener index=" + + topicId + " " + demandMsg.rebalanceId() + ", partitions count=" + stripePartitions.get(topicId).size() + + " (" + stripePartitions.get(topicId).partitionsList() + ")]"); } catch (IgniteCheckedException e1) { ClusterTopologyCheckedException cause = e1.getCause(ClusterTopologyCheckedException.class); @@ -522,31 +528,26 @@ private void requestPartitions(final RebalanceFuture fut, GridDhtPreloaderAssign fut.cancel(); } - }; - - awaitClearingAndStartRebalance(fut, demandMsg, initDemandRequestTask); + }, true)); } } } } /** - * Awaits partitions clearing for full partitions and sends initial demand request - * after all partitions are cleared and safe to consume data. + * Creates future which will be completed when all {@code fullPartitions} are cleared. * * @param fut Rebalance future. - * @param demandMessage Initial demand message which contains set of full partitions to await. - * @param initDemandRequestTask Task which sends initial demand request. + * @param fullPartitions Set of full partitions need to be cleared. + * @return Future which will be completed when given partitions are cleared. */ - private void awaitClearingAndStartRebalance(RebalanceFuture fut, - GridDhtPartitionDemandMessage demandMessage, - Runnable initDemandRequestTask) { - Set fullPartitions = demandMessage.partitions().fullSet(); + private IgniteInternalFuture clearFullPartitions(RebalanceFuture fut, Set fullPartitions) { + final GridFutureAdapter clearAllFuture = new GridFutureAdapter(); if (fullPartitions.isEmpty()) { - ctx.kernalContext().closure().runLocalSafe(initDemandRequestTask, true); + clearAllFuture.onDone(); - return; + return clearAllFuture; } for (GridCacheContext cctx : grp.caches()) { @@ -560,16 +561,19 @@ private void awaitClearingAndStartRebalance(RebalanceFuture fut, final AtomicInteger clearingPartitions = new AtomicInteger(fullPartitions.size()); for (int partId : fullPartitions) { - if (fut.isDone()) - return; + if (fut.isDone()) { + clearAllFuture.onDone(); + + return clearAllFuture; + } GridDhtLocalPartition part = grp.topology().localPartition(partId); if (part != null && part.state() == MOVING) { part.onClearFinished(f -> { - // Cancel rebalance if partition clearing was failed. - if (f.error() != null) { - if (!fut.isDone()) { + if (!fut.isDone()) { + // Cancel rebalance if partition clearing was failed. + if (f.error() != null) { for (GridCacheContext cctx : grp.caches()) { if (cctx.statisticsEnabled()) { final CacheMetricsImpl metrics = cctx.cache().metrics0(); @@ -581,30 +585,54 @@ private void awaitClearingAndStartRebalance(RebalanceFuture fut, log.error("Unable to await partition clearing " + part, f.error()); fut.cancel(); + + clearAllFuture.onDone(f.error()); } - } - else { - if (!fut.isDone()) { - int existed = clearingPartitions.decrementAndGet(); + else { + int remaining = clearingPartitions.decrementAndGet(); for (GridCacheContext cctx : grp.caches()) { if (cctx.statisticsEnabled()) { final CacheMetricsImpl metrics = cctx.cache().metrics0(); - metrics.rebalanceClearingPartitions(existed); + metrics.rebalanceClearingPartitions(remaining); } } - // If all partitions are cleared send initial demand message. - if (existed == 0) - ctx.kernalContext().closure().runLocalSafe(initDemandRequestTask, true); + if (log.isDebugEnabled()) + log.debug("Remaining clearing partitions [grp=" + grp.cacheOrGroupName() + + ", remaining=" + remaining + "]"); + + if (remaining == 0) + clearAllFuture.onDone(); } } + else { + clearAllFuture.onDone(); + } }); } - else - clearingPartitions.decrementAndGet(); + else { + int remaining = clearingPartitions.decrementAndGet(); + + for (GridCacheContext cctx : grp.caches()) { + if (cctx.statisticsEnabled()) { + final CacheMetricsImpl metrics = cctx.cache().metrics0(); + + metrics.rebalanceClearingPartitions(remaining); + } + } + + if (log.isDebugEnabled()) + log.debug("Remaining clearing partitions [grp=" + grp.cacheOrGroupName() + + ", remaining=" + remaining + "]"); + + if (remaining == 0) + clearAllFuture.onDone(); + } } + + return clearAllFuture; } /** 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 6d2f526d73780..a3ee305406207 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 @@ -173,7 +173,8 @@ public void handleDemandMessage(int topicId, UUID nodeId, GridDhtPartitionDemand if (curTop.compareTo(demTop) > 0) { if (log.isDebugEnabled()) - log.debug("Demand request outdated [currentTopVer=" + curTop + log.debug("Demand request outdated [grp=" + grp.cacheOrGroupName() + + ", currentTopVer=" + curTop + ", demandTopVer=" + demTop + ", from=" + nodeId + ", topicId=" + topicId + "]"); @@ -189,10 +190,19 @@ public void handleDemandMessage(int topicId, UUID nodeId, GridDhtPartitionDemand if (sctx != null && sctx.rebalanceId == -d.rebalanceId()) { clearContext(scMap.remove(contextId), log); + + if (log.isDebugEnabled()) + log.debug("Supply context cleaned [grp=" + grp.cacheOrGroupName() + + ", from=" + nodeId + + ", demandMsg=" + d + + ", supplyContext=" + sctx + "]"); } else { if (log.isDebugEnabled()) - log.debug("Stale context cleanup message " + d + ", supplyContext=" + sctx); + log.debug("Stale supply context cleanup message [grp=" + grp.cacheOrGroupName() + + ", from=" + nodeId + + ", demandMsg=" + d + + ", supplyContext=" + sctx + "]"); } return; @@ -200,13 +210,16 @@ public void handleDemandMessage(int topicId, UUID nodeId, GridDhtPartitionDemand } if (log.isDebugEnabled()) - log.debug("Demand request accepted [current=" + curTop + ", demanded=" + demTop + - ", from=" + nodeId + ", topicId=" + topicId + "]"); + log.debug("Demand request accepted [grp=" + grp.cacheOrGroupName() + + ", from=" + nodeId + + ", currentVer=" + curTop + + ", demandedVer=" + demTop + + ", topicId=" + topicId + "]"); ClusterNode node = grp.shared().discovery().node(nodeId); if (node == null) - return; // Context will be cleaned at topology change. + return; try { SupplyContext sctx; @@ -217,13 +230,27 @@ public void handleDemandMessage(int topicId, UUID nodeId, GridDhtPartitionDemand if (sctx != null && d.rebalanceId() < sctx.rebalanceId) { // Stale message, return context back and return. scMap.put(contextId, sctx); + + if (log.isDebugEnabled()) + log.debug("Stale demand message [grp=" + grp.cacheOrGroupName() + + ", actualContext=" + sctx + + ", from=" + nodeId + + ", demandMsg=" + d + "]"); + return; } } // 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 (sctx == null && (d.partitions() == null || d.partitions().isEmpty())) { + if (log.isDebugEnabled()) + log.debug("Empty demand message [grp=" + grp.cacheOrGroupName() + + ", from=" + nodeId + + ", topicId=" + topicId + + ", demandMsg=" + d + "]"); + return; + } assert !(sctx != null && !d.partitions().isEmpty()); @@ -271,7 +298,8 @@ public void handleDemandMessage(int topicId, UUID nodeId, GridDhtPartitionDemand GridDhtLocalPartition loc = top.localPartition(part, d.topologyVersion(), false); - assert loc != null && loc.state() == GridDhtPartitionState.OWNING; + assert loc != null && loc.state() == GridDhtPartitionState.OWNING + : "Partition should be in OWNING state: " + loc; s.addEstimatedKeysCount(grp.offheap().totalPartitionEntriesCount(part)); } @@ -323,7 +351,8 @@ public void handleDemandMessage(int topicId, UUID nodeId, GridDhtPartitionDemand GridDhtLocalPartition loc = top.localPartition(part, d.topologyVersion(), false); - assert (loc != null && loc.state() == OWNING && loc.reservations() > 0) || iter.isPartitionMissing(part) : loc; + assert (loc != null && loc.state() == OWNING && loc.reservations() > 0) || iter.isPartitionMissing(part) + : "Partition should be in OWNING state and has at least 1 reservation " + loc; if (iter.isPartitionMissing(part) && remainingParts.contains(part)) { s.missed(part); @@ -361,9 +390,6 @@ public void handleDemandMessage(int topicId, UUID nodeId, GridDhtPartitionDemand remainingParts.remove(part); } - - // Need to manually prepare cache message. - // TODO GG-11141. } Iterator remainingIter = remainingParts.iterator(); @@ -374,7 +400,8 @@ public void handleDemandMessage(int topicId, UUID nodeId, GridDhtPartitionDemand if (iter.isPartitionDone(p)) { GridDhtLocalPartition loc = top.localPartition(p, d.topologyVersion(), false); - assert loc != null; + assert loc != null + : "Supply partition is gone: grp=" + grp.cacheOrGroupName() + ", p=" + p; s.last(p, loc.updateCounter()); @@ -387,7 +414,8 @@ else if (iter.isPartitionMissing(p)) { } } - assert remainingParts.isEmpty(); + assert remainingParts.isEmpty() + : "Partitions after rebalance should be either done or missing: " + remainingParts; if (sctx != null) clearContext(sctx, log); 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 ddcb81e238965..700f0cf98b7a1 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 @@ -187,7 +187,7 @@ private IgniteCheckedException stopError() { AffinityAssignment aff = grp.affinity().cachedAffinity(topVer); - CachePartitionFullCountersMap cntrMap = top.fullUpdateCounters(); + CachePartitionFullCountersMap countersMap = grp.topology().fullUpdateCounters(); for (int p = 0; p < partCnt; p++) { if (ctx.exchange().hasPendingExchange()) { @@ -251,7 +251,7 @@ private IgniteCheckedException stopError() { ); } - msg.partitions().addHistorical(p, cntrMap.initialUpdateCounter(p), cntrMap.updateCounter(p), partCnt); + msg.partitions().addHistorical(p, part.initialUpdateCounter(), countersMap.updateCounter(p), partCnt); } else { Collection picked = pickOwners(p, topVer); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgniteDhtPartitionsToReloadMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgniteDhtPartitionsToReloadMap.java index 7066e0d72ac3d..8515004c3fdf4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgniteDhtPartitionsToReloadMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgniteDhtPartitionsToReloadMap.java @@ -90,7 +90,7 @@ public synchronized void put(UUID nodeId, int cacheId, int partId) { /** * @return {@code True} if empty. */ - public boolean isEmpty() { + public synchronized boolean isEmpty() { return map == null || map.isEmpty(); } 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 68ec83db1b4b6..5feaa252dd3fa 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 @@ -755,8 +755,8 @@ private Metas getOrAllocateCacheMetas() throws IgniteCheckedException { } /** {@inheritDoc} */ - @Override @Nullable protected IgniteHistoricalIterator historicalIterator( - CachePartitionPartialCountersMap partCntrs) throws IgniteCheckedException { + @Override @Nullable protected WALHistoricalIterator historicalIterator( + CachePartitionPartialCountersMap partCntrs, Set missing) throws IgniteCheckedException { if (partCntrs == null || partCntrs.isEmpty()) return null; @@ -773,13 +773,18 @@ private Metas getOrAllocateCacheMetas() throws IgniteCheckedException { if (startPtr == null) throw new IgniteCheckedException("Could not find start pointer for partition [part=" + p + ", partCntrSince=" + initCntr + "]"); - if (minPtr == null || startPtr.compareTo(minPtr) == -1) + if (minPtr == null || startPtr.compareTo(minPtr) < 0) minPtr = startPtr; } WALIterator it = grp.shared().wal().replay(minPtr); - return new WALIteratorAdapter(grp, partCntrs, it); + WALHistoricalIterator iterator = new WALHistoricalIterator(grp, partCntrs, it); + + // Add historical partitions which are unabled to reserve to missing set. + missing.addAll(iterator.missingParts); + + return iterator; } /** @@ -807,7 +812,7 @@ long freeSpace() { /** * */ - private static class WALIteratorAdapter implements IgniteHistoricalIterator { + private static class WALHistoricalIterator implements IgniteHistoricalIterator { /** */ private static final long serialVersionUID = 0L; @@ -817,6 +822,9 @@ private static class WALIteratorAdapter implements IgniteHistoricalIterator { /** Partition counters map. */ private final CachePartitionPartialCountersMap partMap; + /** Partitions marked as missing (unable to reserve or partition is not in OWNING state). */ + private final Set missingParts = new HashSet<>(); + /** Partitions marked as done. */ private final Set doneParts = new HashSet<>(); @@ -830,19 +838,24 @@ private static class WALIteratorAdapter implements IgniteHistoricalIterator { private Iterator entryIt; /** */ - private CacheDataRow next; + private DataEntry next; + + /** Flag indicates that partition belongs to current {@link #next} is finished and no longer needs to rebalance. */ + private boolean reachedPartitionEnd; /** * @param grp Cache context. * @param walIt WAL iterator. */ - private WALIteratorAdapter(CacheGroupContext grp, CachePartitionPartialCountersMap partMap, WALIterator walIt) { + private WALHistoricalIterator(CacheGroupContext grp, CachePartitionPartialCountersMap partMap, WALIterator walIt) { this.grp = grp; this.partMap = partMap; this.walIt = walIt; cacheIds = grp.cacheIds(); + reservePartitions(); + advance(); } @@ -859,6 +872,7 @@ private WALIteratorAdapter(CacheGroupContext grp, CachePartitionPartialCountersM /** {@inheritDoc} */ @Override public void close() throws IgniteCheckedException { walIt.close(); + releasePartitions(); } /** {@inheritDoc} */ @@ -896,7 +910,13 @@ private WALIteratorAdapter(CacheGroupContext grp, CachePartitionPartialCountersM if (next == null) throw new NoSuchElementException(); - CacheDataRow val = next; + CacheDataRow val = new DataEntryRow(next); + + if (reachedPartitionEnd) { + doneParts.add(next.partitionId()); + + reachedPartitionEnd = false; + } advance(); @@ -908,6 +928,46 @@ private WALIteratorAdapter(CacheGroupContext grp, CachePartitionPartialCountersM throw new UnsupportedOperationException(); } + /** + * Reserve historical partitions. + * If partition is unable to reserve, id of that partition is placed to {@link #missingParts} set. + */ + private void reservePartitions() { + for (int i = 0; i < partMap.size(); i++) { + int p = partMap.partitionAt(i); + GridDhtLocalPartition part = grp.topology().localPartition(p); + + if (part == null || !part.reserve()) { + missingParts.add(p); + continue; + } + + if (part.state() != OWNING) { + part.release(); + missingParts.add(p); + } + } + } + + /** + * Release historical partitions. + */ + private void releasePartitions() { + for (int i = 0; i < partMap.size(); i++) { + int p = partMap.partitionAt(i); + + if (missingParts.contains(p)) + continue; + + GridDhtLocalPartition part = grp.topology().localPartition(p); + + assert part != null && part.state() == OWNING && part.reservations() > 0 + : "Partition should in OWNING state and has at least 1 reservation"; + + part.release(); + } + } + /** * */ @@ -922,7 +982,7 @@ private void advance() { if (cacheIds.contains(entry.cacheId())) { int idx = partMap.partitionIndex(entry.partitionId()); - if (idx < 0) + if (idx < 0 || missingParts.contains(idx)) continue; long from = partMap.initialUpdateCounterAt(idx); @@ -930,9 +990,9 @@ private void advance() { if (entry.partitionCounter() >= from && entry.partitionCounter() <= to) { if (entry.partitionCounter() == to) - doneParts.add(entry.partitionId()); + reachedPartitionEnd = true; - next = new DataEntryRow(entry); + next = entry; return; } diff --git a/modules/core/src/test/config/log4j-test.xml b/modules/core/src/test/config/log4j-test.xml index 9138c02e00248..b0b08e7d1a394 100755 --- a/modules/core/src/test/config/log4j-test.xml +++ b/modules/core/src/test/config/log4j-test.xml @@ -78,12 +78,6 @@ - - - **/packaging/rpm/SOURCES/name.service - **/packaging/rpm/SOURCES/service.sh - **/packaging/rpm/SPECS/apache-ignite.spec + + packaging/** From f2c5f11071f40fc60dc4ebc7e613fca0a2248c05 Mon Sep 17 00:00:00 2001 From: Turik Campbell Date: Mon, 9 Apr 2018 00:52:10 +0300 Subject: [PATCH 0123/1463] IGNITE-7877: Improve code style in GA part this closes #3695 (cherry picked from commit b97b1ee) --- .../examples/ml/genetic/change/Coin.java | 7 ++-- .../OptimizeMakeChangeFitnessFunction.java | 3 +- .../change/OptimizeMakeChangeGAExample.java | 8 +++- .../OptimizeMakeChangeTerminateCriteria.java | 3 +- .../helloworld/HelloWorldFitnessFunction.java | 3 +- .../helloworld/HelloWorldGAExample.java | 9 ++++- .../HelloWorldTerminateCriteria.java | 3 +- .../examples/ml/genetic/movie/Movie.java | 4 +- .../genetic/movie/MovieFitnessFunction.java | 3 +- .../ml/genetic/movie/MovieGAExample.java | 4 +- .../genetic/movie/MovieTerminateCriteria.java | 4 +- .../apache/ignite/ml/genetic/Chromosome.java | 8 ++-- .../ignite/ml/genetic/CrossOverJob.java | 5 +-- .../ignite/ml/genetic/CrossOverTask.java | 3 +- .../apache/ignite/ml/genetic/FitnessJob.java | 6 +-- .../apache/ignite/ml/genetic/FitnessTask.java | 6 +-- .../org/apache/ignite/ml/genetic/GAGrid.java | 38 ++++++++----------- .../org/apache/ignite/ml/genetic/Gene.java | 6 +-- .../ignite/ml/genetic/IFitnessFunction.java | 2 - .../apache/ignite/ml/genetic/MutateJob.java | 10 +---- .../apache/ignite/ml/genetic/MutateTask.java | 19 ++++------ .../ml/genetic/TruncateSelectionJob.java | 11 +----- .../ml/genetic/TruncateSelectionTask.java | 17 ++------- .../ml/genetic/cache/GeneCacheConfig.java | 2 - .../genetic/cache/PopulationCacheConfig.java | 2 - .../ml/genetic/functions/GAGridFunction.java | 9 +---- .../genetic/parameter/ChromosomeCriteria.java | 7 ++-- .../ml/genetic/parameter/GAConfiguration.java | 6 +-- .../ml/genetic/parameter/GAGridConstants.java | 4 +- .../ignite/ml/genetic/utils/GAGridUtils.java | 10 ++--- .../genetic/GAGridCalculateFitnessTest.java | 6 ++- .../GAGridInitializePopulationTest.java | 7 +++- 32 files changed, 101 insertions(+), 134 deletions(-) diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/change/Coin.java b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/change/Coin.java index 90f6e8fedb52d..4944a6b523dd2 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/change/Coin.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/change/Coin.java @@ -23,7 +23,7 @@ * POJO to model a coin */ public class Coin implements Serializable { - + /** Define Coin Type */ public static enum CoinType { PENNY, QUARTER, NICKEL, DIME } @@ -33,6 +33,7 @@ public static enum CoinType { /** number of coins */ private int numberOfCoins = 0; + /** CoinType */ private CoinType coinType = null; /** @@ -80,8 +81,8 @@ public void setCoinType(CoinType coinType) { this.coinType = coinType; } - @Override - public String toString() { + /** {@inheritDoc} */ + @Override public String toString() { return "Coin [numberOfCoins=" + numberOfCoins + ", coinType=" + coinType + "]"; } diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/change/OptimizeMakeChangeFitnessFunction.java b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/change/OptimizeMakeChangeFitnessFunction.java index faa113b657f1d..a7c248ad53745 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/change/OptimizeMakeChangeFitnessFunction.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/change/OptimizeMakeChangeFitnessFunction.java @@ -31,9 +31,8 @@ * * an individual solution relative to other solutions.
    */ - public class OptimizeMakeChangeFitnessFunction implements IFitnessFunction { - + /** target amount */ int targetAmount = 0; /** 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 000424f003e32..6b832de899c25 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 @@ -48,14 +48,20 @@ * 'ignite.{sh|bat} examples/config/example-ignite.xml'}.

    Alternatively you can run ExampleNodeStartup in * another JVM which will start node with {@code examples/config/example-ignite.xml} configuration.

    */ - public class OptimizeMakeChangeGAExample { + /** Ignite instance */ private static Ignite ignite = null; + + /** GAGrid */ private static GAGrid gaGrid = null; + + /** GAConfiguration */ private static GAConfiguration gaConfig = null; + /** amount of change */ private static String sAmountChange = null; + /** Ignite logger */ private static IgniteLogger logger = null; /** 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 79601fe09052c..a4ea2636b6be2 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 @@ -31,8 +31,9 @@ * Terminate Condition implementation for OptimizeMakeChangeGATest
    */ public class OptimizeMakeChangeTerminateCriteria implements ITerminateCriteria { - + /** Ignite logger */ private IgniteLogger igniteLogger = null; + /** Ignite instance */ private Ignite ignite = null; /** diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/helloworld/HelloWorldFitnessFunction.java b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/helloworld/HelloWorldFitnessFunction.java index f5f1ee3081ba5..85c00e902963f 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/helloworld/HelloWorldFitnessFunction.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/helloworld/HelloWorldFitnessFunction.java @@ -39,9 +39,8 @@ * * we achieve a fitness score of '11', as 'HELLO WORLD' contains '11' characters. */ - public class HelloWorldFitnessFunction implements IFitnessFunction { - + /** Optimal target solution */ private String targetString = "HELLO WORLD"; /** 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 70b00d80db05f..78374d2458d11 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 @@ -44,12 +44,19 @@ * 'ignite.{sh|bat} examples/config/example-ignite.xml'}.

    Alternatively you can run ExampleNodeStartup in * another JVM which will start node with {@code examples/config/example-ignite.xml} configuration.

    */ - public class HelloWorldGAExample { + /** Ignite instance */ private static Ignite ignite = null; + /** GAGrid */ private static GAGrid gaGrid = null; + /** GAConfiguration */ private static GAConfiguration gaConfig = null; + /** + * Executes example. + * + * @param args Command line arguments, none required. + */ public static void main(String args[]) { System.setProperty("IGNITE_QUIET", "false"); 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 a0d6f9394634b..24bb5e0365e08 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 @@ -33,8 +33,9 @@ * Class terminates Genetic algorithm when fitnessScore > 10 */ public class HelloWorldTerminateCriteria implements ITerminateCriteria { - + /** Ignite logger */ private IgniteLogger igniteLogger = null; + /** Ignite instance */ private Ignite ignite = null; /** diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/movie/Movie.java b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/movie/Movie.java index fd4afda4d84d7..38d27ff294896 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/movie/Movie.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/movie/Movie.java @@ -22,7 +22,6 @@ /** * POJO to model a movie. */ - public class Movie { /** name of movie */ private String name; @@ -125,7 +124,8 @@ public void setRating(String rating) { this.rating = rating; } - public String toString() { + /** {@inheritDoc} */ + @Override public String toString() { return "Movie [name=" + name + ", genre=" + genre + ", rating=" + rating + ", imdbRating=" + imdbRating + ", year=" + year + "]"; } diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/movie/MovieFitnessFunction.java b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/movie/MovieFitnessFunction.java index 9690d692986aa..20e04ce38f41b 100644 --- a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/movie/MovieFitnessFunction.java +++ b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/movie/MovieFitnessFunction.java @@ -45,9 +45,8 @@ * * fitness score. */ - public class MovieFitnessFunction implements IFitnessFunction { - + /** genes */ private List genres = null; /** 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 f14cb83103d23..e601081e95987 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 @@ -45,10 +45,12 @@ * 'ignite.{sh|bat} examples/config/example-ignite.xml'}.

    Alternatively you can run ExampleNodeStartup in * another JVM which will start node with {@code examples/config/example-ignite.xml} configuration.

    */ - public class MovieGAExample { + /** Ignite instance */ private static Ignite ignite = null; + /** GAGrid */ private static GAGrid gaGrid = null; + /** GAConfiguration */ private static GAConfiguration gaConfig = null; /** 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 904c66dc67569..747d51441d21e 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 @@ -32,10 +32,10 @@ * * Class terminates Genetic algorithm when fitnessScore > 32
    */ - public class MovieTerminateCriteria implements ITerminateCriteria { - + /** Ignite logger */ private IgniteLogger igniteLogger = null; + /** Ignite instance */ private Ignite ignite = null; /** diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/Chromosome.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/Chromosome.java index 03e949f7c567a..ed78e855dc473 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/Chromosome.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/Chromosome.java @@ -32,9 +32,10 @@ *

    */ public class Chromosome { - + /** primary key of Chromosome */ private static final AtomicLong ID_GEN = new AtomicLong(); + /** fitness score */ @QuerySqlField(index = true) private double fitnessScore = -1; @@ -46,7 +47,6 @@ public class Chromosome { private long[] genes; /** - * * @param genes Primary keys of Genes */ public Chromosome(long[] genes) { @@ -99,8 +99,8 @@ public Long id() { return id; } - @Override - public String toString() { + /** {@inheritDoc} */ + @Override public String toString() { return "Chromosome [fitnessScore=" + fitnessScore + ", id=" + id + ", genes=" + Arrays.toString(genes) + "]"; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/CrossOverJob.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/CrossOverJob.java index 74f81b8d4291b..35f4a78d22ea6 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/CrossOverJob.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/CrossOverJob.java @@ -18,7 +18,6 @@ package org.apache.ignite.ml.genetic; import java.util.Arrays; -import java.util.List; import java.util.Random; import org.apache.ignite.Ignite; @@ -49,10 +48,11 @@ *

    */ public class CrossOverJob extends ComputeJobAdapter { - + /** Ignite resource */ @IgniteInstanceResource private Ignite ignite = null; + /** Ignite logger */ @LoggerResource private IgniteLogger log = null; @@ -84,7 +84,6 @@ public CrossOverJob(Long key1, Long key2, double crossOverRate) { * @param genekeys Original gene Keys for a chromosome * @return New Gene keys */ - private long[] crossOver(long[] newKeySwapArrayForChrome, int updateIdx, long[] genekeys) { long[] newGeneKeys = genekeys.clone(); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/CrossOverTask.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/CrossOverTask.java index 377ec3d3bd95e..5d342dad5a477 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/CrossOverTask.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/CrossOverTask.java @@ -46,7 +46,7 @@ *

    */ public class CrossOverTask extends ComputeTaskAdapter, Boolean> { - + /** Ignite instance */ @IgniteInstanceResource private Ignite ignite = null; @@ -117,7 +117,6 @@ public ComputeJobResultPolicy result(ComputeJobResult res, List setupCrossOver(ClusterNode clusterNode, List keys, Map map) { // Calculate number of Jobs = keys / 2 diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/FitnessJob.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/FitnessJob.java index 064846e959d01..182b7dc24deda 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/FitnessJob.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/FitnessJob.java @@ -24,7 +24,6 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; -import org.apache.ignite.compute.ComputeExecutionRejectedException; import org.apache.ignite.compute.ComputeJobAdapter; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.LoggerResource; @@ -36,15 +35,16 @@ * Responsible for performing fitness evaluation on an individual chromosome */ public class FitnessJob extends ComputeJobAdapter { - /** * Chromosome primary Key */ private Long key; + /** Ignite instance */ @IgniteInstanceResource private Ignite ignite = null; + /** Ignite logger */ @LoggerResource private IgniteLogger log = null; @@ -77,7 +77,7 @@ public Double execute() throws IgniteException { long[] geneKeys = chromosome.getGenes(); - List genes = new ArrayList(); + List genes = new ArrayList(); for (int i = 0; i < geneKeys.length; i++) { long aKey = geneKeys[i]; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/FitnessTask.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/FitnessTask.java index 58b69fae8baf9..4403be4aa5d19 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/FitnessTask.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/FitnessTask.java @@ -17,8 +17,6 @@ package org.apache.ignite.ml.genetic; -import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -40,11 +38,11 @@ * Responsible for fitness operation */ public class FitnessTask extends ComputeTaskAdapter, Boolean> { - + /** Ignite instance */ @IgniteInstanceResource private Ignite ignite = null; - /** GAConfiguration **/ + /** GAConfiguration */ private GAConfiguration config = null; /** diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/GAGrid.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/GAGrid.java index 56895f72b2aa8..2f670544733a6 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/GAGrid.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/GAGrid.java @@ -34,7 +34,6 @@ import org.apache.ignite.ml.genetic.cache.PopulationCacheConfig; import org.apache.ignite.ml.genetic.parameter.GAConfiguration; import org.apache.ignite.ml.genetic.parameter.GAGridConstants; -import org.apache.ignite.ml.genetic.utils.GAGridUtils; /** * Central class responsible for orchestrating distributive Genetic Algorithm. @@ -42,7 +41,7 @@ * This class accepts a GAConfigriation and Ignite instance. */ public class GAGrid { - + /** Ignite logger */ IgniteLogger igniteLogger = null; /** GAConfiguraton */ private GAConfiguration config = null; @@ -53,7 +52,7 @@ public class GAGrid { /** Gene cache */ private IgniteCache geneCache = null; /** population keys */ - private List populationKeys = new ArrayList(); + private List populationKeys = new ArrayList(); /** * @param config GAConfiguration @@ -79,7 +78,6 @@ public GAGrid(GAConfiguration config, Ignite ignite) { * * @return Average fitness score */ - private Double calculateAverageFitness() { double avgFitnessScore = 0; @@ -104,7 +102,7 @@ private Double calculateAverageFitness() { * @param chromosomeKeys List of chromosome primary keys */ private void calculateFitness(List chromosomeKeys) { - Boolean boolValue = this.ignite.compute().execute(new FitnessTask(this.config), chromosomeKeys); + this.ignite.compute().execute(new FitnessTask(this.config), chromosomeKeys); } /** @@ -122,9 +120,9 @@ private Boolean copyFitterChromosomesToPopulation(List fittestKeys, List fittestKeys, List keys = new ArrayList(); + List keys = new ArrayList(); int k = 0; while (k < numberOfGenes) { long key = selectGene(k); @@ -157,7 +155,7 @@ private Chromosome createChromosome(int numberOfGenes) { * @param leastFitKeys List of primary keys for Chromsomes that are considered 'least fit' */ private void crossover(List leastFitKeys) { - Boolean boolValue = this.ignite.compute().execute(new CrossOverTask(this.config), leastFitKeys); + this.ignite.compute().execute(new CrossOverTask(this.config), leastFitKeys); } /** @@ -186,8 +184,6 @@ public Chromosome evolve() { fittestChomosome = populationCache.get(keys.get(0)); - List genes = GAGridUtils.getGenesForChromosome(ignite, fittestChomosome); - // while NOT terminateCondition met while (!(config.getTerminateCriteria().isTerminationConditionMet(fittestChomosome, averageFitnessScore, generationCount))) { @@ -227,13 +223,12 @@ public Chromosome evolve() { * @return List of primary keys for chromosomes. */ private List getChromosomesByFittest() { - List orderChromKeysByFittest = new ArrayList(); + List orderChromKeysByFittest = new ArrayList(); String orderDirection = "desc"; - if(config.isHigherFitnessValueFitter()==false) - { - orderDirection = "asc"; - } + if (!config.isHigherFitnessValueFitter()) + orderDirection = "asc"; + String fittestSQL = "select _key from Chromosome order by fitnessScore " + orderDirection; // Execute query to retrieve keys for ALL Chromosomes by fittnessScore @@ -312,7 +307,7 @@ void intializePopulation() { * @param leastFitKeys List of primary keys for Chromosomes that are considered 'least fit'. */ private void mutation(List leastFitKeys) { - Boolean boolValue = this.ignite.compute().execute(new MutateTask(this.config), leastFitKeys); + this.ignite.compute().execute(new MutateTask(this.config), leastFitKeys); } /** @@ -348,7 +343,6 @@ private List selectByElitism(List keys) { * @param keys * @return List of keys */ - private List selectByTruncation(List keys) { double truncatePercentage = this.config.getTruncateRate(); @@ -364,12 +358,10 @@ private List selectByTruncation(List keys) { * @return Primary key of respective Gene chosen */ private long selectGene(int k) { - if (config.getChromosomeCriteria() == null) { + if (config.getChromosomeCriteria() == null) return (selectAnyGene()); - } - else { + else return (selectGeneByChromsomeCriteria(k)); - } } /** @@ -431,7 +423,7 @@ private List selection(List chromosomeKeys) { List fittestKeys = getFittestKeysForTruncation(chromosomeKeys); - Boolean boolValue = copyFitterChromosomesToPopulation(fittestKeys, selectedKeys); + copyFitterChromosomesToPopulation(fittestKeys, selectedKeys); // copy more fit keys to rest of population break; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/Gene.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/Gene.java index e7935477faa9a..d585330b09099 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/Gene.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/Gene.java @@ -35,7 +35,7 @@ *

    */ public class Gene { - + /** primary key of Gene */ private static final AtomicLong ID_GEN = new AtomicLong(); /** Id (indexed). */ @@ -78,8 +78,8 @@ public Long id() { return id; } - @Override - public String toString() { + /** {@inheritDoc} */ + @Override public String toString() { return "Gene [id=" + id + ", value=" + value + "]"; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/IFitnessFunction.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/IFitnessFunction.java index ef031ec76d27d..fa84946f4eaf8 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/IFitnessFunction.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/IFitnessFunction.java @@ -28,9 +28,7 @@ * *

    */ - public interface IFitnessFunction { - /** * @param genes Genes within an individual Chromosome * @return Fitness score diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/MutateJob.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/MutateJob.java index ed3f0d93d2e5b..b03e7cac45eb4 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/MutateJob.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/MutateJob.java @@ -17,11 +17,7 @@ package org.apache.ignite.ml.genetic; -import java.util.ArrayList; -import java.util.Collection; import java.util.List; -import java.util.Map; - import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteException; @@ -35,13 +31,13 @@ * Responsible for applying mutation on respective Chromosome based on mutation Rate */ public class MutateJob extends ComputeJobAdapter { - /** primary key of Chromosome to mutate **/ private Long key; /** primary keys of genes to be used in mutation **/ private List mutatedGeneKeys; + /** Ignite instance */ @IgniteInstanceResource private Ignite ignite = null; @@ -65,7 +61,6 @@ public MutateJob(Long key, List mutatedGeneKeys, double mutationRate) { * @return Boolean value */ public Boolean execute() throws IgniteException { - // TODO Auto-generated method stub IgniteCache populationCache = ignite.cache(GAGridConstants.POPULATION_CACHE); @@ -75,9 +70,8 @@ public Boolean execute() throws IgniteException { for (int k = 0; k < this.mutatedGeneKeys.size(); k++) { // Mutate gene based on MutatonRate - if (this.mutationRate > Math.random()) { + if (this.mutationRate > Math.random()) geneKeys[k] = this.mutatedGeneKeys.get(k); - } } chromosome.setGenes(geneKeys); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/MutateTask.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/MutateTask.java index c72a73aa0fce1..3a7b0c5877fd8 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/MutateTask.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/MutateTask.java @@ -48,10 +48,11 @@ * chromosomes reside.
    */ public class MutateTask extends ComputeTaskAdapter, Boolean> { - + /** Ignite instance */ @IgniteInstanceResource private Ignite ignite = null; + /** GAConfiguration */ private GAConfiguration config = null; /** @@ -67,14 +68,12 @@ public MutateTask(GAConfiguration config) { * @return Gene primary keys */ private List getMutatedGenes() { - List mutatedGenes = new ArrayList(); + List mutatedGenes = new ArrayList(); config.getChromosomeLength(); - for (int i = 0; i < config.getChromosomeLength(); i++) { - // Gene gene=config.getGenePool().get(selectRandomIndex(config.getGenePool().size())); + for (int i = 0; i < config.getChromosomeLength(); i++) mutatedGenes.add(selectGene(i)); - } - + return mutatedGenes; } @@ -139,12 +138,10 @@ private long selectAnyGene() { * @return Primary key of Gene */ private long selectGene(int k) { - if (config.getChromosomeCriteria() == null) { + if (config.getChromosomeCriteria() == null) return (selectAnyGene()); - } - else { + else return (selectGeneByChromsomeCriteria(k)); - } } /** @@ -154,7 +151,7 @@ private long selectGene(int k) { * @return Primary key of Gene */ private long selectGeneByChromsomeCriteria(int k) { - List genes = new ArrayList(); + List genes = new ArrayList(); StringBuffer sbSqlClause = new StringBuffer("_val like '"); sbSqlClause.append("%"); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/TruncateSelectionJob.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/TruncateSelectionJob.java index 4404e5c8789bc..1317b0e39a0d2 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/TruncateSelectionJob.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/TruncateSelectionJob.java @@ -17,11 +17,7 @@ package org.apache.ignite.ml.genetic; -import java.util.ArrayList; -import java.util.Collection; import java.util.List; -import java.util.Map; - import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteException; @@ -35,13 +31,13 @@ * Responsible for performing truncate selection */ public class TruncateSelectionJob extends ComputeJobAdapter { - /** primary key of Chromosome to mutate */ private Long key; /** primary keys of genes to be used in mutation */ private List mutatedGeneKeys; + /** Ignite instance */ @IgniteInstanceResource private Ignite ignite = null; @@ -60,7 +56,6 @@ public TruncateSelectionJob(Long key, List mutatedGeneKeys) { * @return Boolean value */ public Boolean execute() throws IgniteException { - // TODO Auto-generated method stub IgniteCache populationCache = ignite.cache(GAGridConstants.POPULATION_CACHE); @@ -69,9 +64,7 @@ public Boolean execute() throws IgniteException { long[] geneKeys = chromosome.getGenes(); for (int k = 0; k < this.mutatedGeneKeys.size(); k++) { - { - geneKeys[k] = this.mutatedGeneKeys.get(k); - } + geneKeys[k] = this.mutatedGeneKeys.get(k); } chromosome.setGenes(geneKeys); diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/TruncateSelectionTask.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/TruncateSelectionTask.java index 7a5817fff3b19..ee7d260565220 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/TruncateSelectionTask.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/TruncateSelectionTask.java @@ -18,7 +18,6 @@ package org.apache.ignite.ml.genetic; import java.util.ArrayList; -import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -37,22 +36,16 @@ import org.apache.ignite.compute.ComputeJobResultPolicy; import org.apache.ignite.compute.ComputeTaskAdapter; import org.apache.ignite.resources.IgniteInstanceResource; - -import org.apache.ignite.ml.genetic.parameter.GAConfiguration; import org.apache.ignite.ml.genetic.parameter.GAGridConstants; /** * Responsible for performing truncate selection. */ - public class TruncateSelectionTask extends ComputeTaskAdapter, Boolean> { - + /** Ignite resource */ @IgniteInstanceResource private Ignite ignite = null; - /** GAConfiguraiton */ - private GAConfiguration config = null; - /** fittest keys */ private List fittestKeys = null; @@ -64,8 +57,7 @@ public class TruncateSelectionTask extends ComputeTaskAdapter, Boolea * @param fittestKeys List of long * @param numberOfCopies Number of Copies */ - public TruncateSelectionTask(GAConfiguration config, List fittestKeys, int numberOfCopies) { - this.config = config; + public TruncateSelectionTask(List fittestKeys, int numberOfCopies) { this.fittestKeys = fittestKeys; this.numberOfCopies = numberOfCopies; } @@ -103,13 +95,13 @@ private Chromosome getChromosome(Long key) { * @return List of lists containing keys */ private List> getEnhancedPopulation() { - List> list = new ArrayList(); + List> list = new ArrayList>(); for (Long key : fittestKeys) { Chromosome copy = getChromosome(key); for (int i = 0; i < numberOfCopies; i++) { long[] thegenes = copy.getGenes(); - List geneList = new ArrayList(); + List geneList = new ArrayList(); for (int k = 0; k < copy.getGenes().length; k++) { geneList.add(thegenes[k]); } @@ -150,7 +142,6 @@ public Map map(List nodes, List chromosomeKeys) throws Ignite * @return Boolean value */ public Boolean reduce(List list) throws IgniteException { - // TODO Auto-generated method stub return Boolean.TRUE; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/cache/GeneCacheConfig.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/cache/GeneCacheConfig.java index 1976941099cca..c5302ee3dd5f2 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/cache/GeneCacheConfig.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/cache/GeneCacheConfig.java @@ -30,9 +30,7 @@ * * cache maintains full population of genes. */ - public class GeneCacheConfig { - /** * @return Cache Configuration */ diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/cache/PopulationCacheConfig.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/cache/PopulationCacheConfig.java index bc7fcd60b0aa2..cae7c1a17b341 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/cache/PopulationCacheConfig.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/cache/PopulationCacheConfig.java @@ -30,9 +30,7 @@ * * cache population of chromosomes (ie: potential solutions) */ - public class PopulationCacheConfig { - /** * @return Cache Configuration */ diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/functions/GAGridFunction.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/functions/GAGridFunction.java index 3848b683d8560..b3e4fed2ff21c 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/functions/GAGridFunction.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/functions/GAGridFunction.java @@ -35,15 +35,10 @@ * Responsible for providing custom SQL functions to retrieve optimization results */ public class GAGridFunction { - - /** GAConfiguration **/ - private GAConfiguration config = null; - - /** + /** * @param config GA Configuration */ - public GAGridFunction(GAConfiguration config) { - this.config = config; + public GAGridFunction() { } /** diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/parameter/ChromosomeCriteria.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/parameter/ChromosomeCriteria.java index ffffd43a76a76..bc4b839d5709b 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/parameter/ChromosomeCriteria.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/parameter/ChromosomeCriteria.java @@ -25,16 +25,15 @@ * Responsible for describing the characteristics of an individual Chromosome. */ public class ChromosomeCriteria { - /** List of criteria for a Chromosome */ - private List criteria = new ArrayList(); + private List criteria = new ArrayList(); /** * Retrieve criteria * * @return List of strings */ - public List getCriteria() { + public List getCriteria() { return criteria; } @@ -43,7 +42,7 @@ public List getCriteria() { * * @param criteria List of criteria to be applied for a Chromosome ;Use format "name=value", ie: "coinType=QUARTER" */ - public void setCriteria(List criteria) { + public void setCriteria(List criteria) { this.criteria = criteria; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/parameter/GAConfiguration.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/parameter/GAConfiguration.java index c66044ed93545..70c7ad9999dae 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/parameter/GAConfiguration.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/parameter/GAConfiguration.java @@ -39,7 +39,7 @@ *

    */ public class GAConfiguration { - + /** Selection method */ private GAGridConstants.SELECTION_METHOD selectionMethod = null; /** Criteria used to describe a chromosome */ @@ -76,7 +76,7 @@ public class GAConfiguration { private int populationSize = 500; /** Gene pool is the sum of ALL genes utilized to create chromsomes */ - private List genePool = new ArrayList(); + private List genePool = new ArrayList(); /** Number of genes within a chromosome */ private int chromosomeLength = 0; @@ -297,7 +297,7 @@ public void setSelectionMethod(GAGridConstants.SELECTION_METHOD selectionMethod) /** * Retreive the termination criteria * - * @return Termination Criteria + * @return Termination Criteria */ public ITerminateCriteria getTerminateCriteria() { return terminateCriteria; diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/parameter/GAGridConstants.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/parameter/GAGridConstants.java index 41be973da6c2c..9a5bd66c3cc36 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/parameter/GAGridConstants.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/parameter/GAGridConstants.java @@ -21,7 +21,6 @@ * GAGridConstants */ public interface GAGridConstants { - /** populationCache constant */ public static final String POPULATION_CACHE = "populationCache"; @@ -31,7 +30,6 @@ public interface GAGridConstants { /** Selection Method type **/ public static enum SELECTION_METHOD { SELECTON_METHOD_ELETISM, SELECTION_METHOD_TRUNCATION - } + }; - ; } diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/utils/GAGridUtils.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/utils/GAGridUtils.java index d951ea3359f37..326cdda8f3769 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/utils/GAGridUtils.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/utils/GAGridUtils.java @@ -26,20 +26,17 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.query.QueryCursor; -import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.SqlQuery; import org.apache.ignite.ml.genetic.Chromosome; import org.apache.ignite.ml.genetic.Gene; import org.apache.ignite.ml.genetic.cache.PopulationCacheConfig; -import org.apache.ignite.ml.genetic.parameter.GAConfiguration; import org.apache.ignite.ml.genetic.parameter.GAGridConstants; /** * GA Grid Helper routines */ public class GAGridUtils { - /** * Retrieve chromosomes * @@ -48,7 +45,7 @@ public class GAGridUtils { * @return List of Chromosomes */ public static List getChromosomes(Ignite ignite, String query) { - List chromosomes = new ArrayList(); + List chromosomes = new ArrayList(); IgniteCache populationCache = ignite.getOrCreateCache(PopulationCacheConfig.populationCache()); @@ -68,7 +65,7 @@ public static List getChromosomes(Ignite ignite, String query) { * @return List of Genes */ public static List getGenesForChromosome(Ignite ignite, Chromosome chromosome) { - List genes = new ArrayList(); + List genes = new ArrayList(); IgniteCache cache = ignite.cache(GAGridConstants.GENE_CACHE); StringBuffer sbSqlClause = new StringBuffer(); sbSqlClause.append("_key IN "); @@ -96,9 +93,8 @@ public static List getGenesForChromosome(Ignite ignite, Chromosome chromos * @param chromosome Chromosome * @return List of Genes */ - public static List getGenesInOrderForChromosome(Ignite ignite, Chromosome chromosome) { - List genes = new ArrayList(); + List genes = new ArrayList(); IgniteCache cache = ignite.cache(GAGridConstants.GENE_CACHE); long[] primaryKeys = chromosome.getGenes(); diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/genetic/GAGridCalculateFitnessTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/genetic/GAGridCalculateFitnessTest.java index 16e4dda47fc5e..5cc6ffee05d94 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/genetic/GAGridCalculateFitnessTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/genetic/GAGridCalculateFitnessTest.java @@ -37,9 +37,13 @@ * Calculate Fitness Test */ public class GAGridCalculateFitnessTest { - + /** Ignite instance */ private Ignite ignite = null; + + /** GAGrid **/ private GAGrid gaGrid = null; + + /** GAConfiguraton */ private GAConfiguration gaConfig = null; /** diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/genetic/GAGridInitializePopulationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/genetic/GAGridInitializePopulationTest.java index f4f9ca4b857f1..f3023ed509a44 100644 --- a/modules/ml/src/test/java/org/apache/ignite/ml/genetic/GAGridInitializePopulationTest.java +++ b/modules/ml/src/test/java/org/apache/ignite/ml/genetic/GAGridInitializePopulationTest.java @@ -36,11 +36,14 @@ /** * Initialize Gene and Chromosome Test */ - public class GAGridInitializePopulationTest { - + /** Ignite instance */ private Ignite ignite = null; + + /** GAGrid **/ private GAGrid gaGrid = null; + + /** GAConfiguraton */ private GAConfiguration gaConfig = null; @Before From 8186a89a12bed5c9f386c16f2c8123d2f4573742 Mon Sep 17 00:00:00 2001 From: YuriBabak Date: Mon, 9 Apr 2018 15:22:03 +0300 Subject: [PATCH 0124/1463] IGNITE-8181: Broken javadoc in GA Grid this closes #3776 (cherry picked from commit 5b8d43d) --- .../ml/genetic/TruncateSelectionTask.java | 51 +++++++++---------- .../ml/genetic/functions/GAGridFunction.java | 26 +++++----- 2 files changed, 35 insertions(+), 42 deletions(-) diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/TruncateSelectionTask.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/TruncateSelectionTask.java index ee7d260565220..d01774157a206 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/TruncateSelectionTask.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/TruncateSelectionTask.java @@ -42,31 +42,30 @@ * Responsible for performing truncate selection. */ public class TruncateSelectionTask extends ComputeTaskAdapter, Boolean> { - /** Ignite resource */ + /** Ignite resource. */ @IgniteInstanceResource private Ignite ignite = null; - /** fittest keys */ + /** Fittest keys. */ private List fittestKeys = null; - /** number of Copies */ - private int numberOfCopies = 0; + /** Number of Copies. */ + private int numOfCopies = 0; /** - * @param config GAConfiguration - * @param fittestKeys List of long - * @param numberOfCopies Number of Copies + * @param fittestKeys List of long. + * @param numOfCopies Number of Copies. */ - public TruncateSelectionTask(List fittestKeys, int numberOfCopies) { + public TruncateSelectionTask(List fittestKeys, int numOfCopies) { this.fittestKeys = fittestKeys; - this.numberOfCopies = numberOfCopies; + this.numOfCopies = numOfCopies; } /** - * Retrieve a chromosome + * Retrieve a chromosome. * - * @param key Primary key of chromosome - * @return Chromosome + * @param key Primary key of chromosome. + * @return Chromosome. */ private Chromosome getChromosome(Long key) { IgniteCache cache = ignite.cache(GAGridConstants.POPULATION_CACHE); @@ -81,25 +80,23 @@ private Chromosome getChromosome(Long key) { try (QueryCursor> cursor = cache.query(sql)) { for (Entry e : cursor) - chromosome = (e.getValue()) - - ; + chromosome = (e.getValue()); } return chromosome; } /** - * Return a List of lists containing keys + * Return a List of lists containing keys. * - * @return List of lists containing keys + * @return List of lists containing keys. */ private List> getEnhancedPopulation() { List> list = new ArrayList>(); for (Long key : fittestKeys) { Chromosome copy = getChromosome(key); - for (int i = 0; i < numberOfCopies; i++) { + for (int i = 0; i < numOfCopies; i++) { long[] thegenes = copy.getGenes(); List geneList = new ArrayList(); for (int k = 0; k < copy.getGenes().length; k++) { @@ -113,9 +110,9 @@ private List> getEnhancedPopulation() { } /** - * @param nodes List of ClusterNode - * @param chromosomeKeys Primary keys for respective chromosomes - * @return Map of nodes to jobs + * @param nodes List of ClusterNode. + * @param chromosomeKeys Primary keys for respective chromosomes. + * @return Map of nodes to jobs. */ public Map map(List nodes, List chromosomeKeys) throws IgniteException { Map map = new HashMap<>(); @@ -132,23 +129,22 @@ public Map map(List nodes, List chromosomeKeys) throws Ignite k = k + 1; } return map; - } /** * We return TRUE if success, else Exception is thrown. * - * @param list List of ComputeJobResult - * @return Boolean value + * @param list List of ComputeJobResult. + * @return Boolean value. */ public Boolean reduce(List list) throws IgniteException { return Boolean.TRUE; } /** - * @param res ComputeJobResult - * @param rcvd List of ComputeJobResult - * @return ComputeJobResultPolicy + * @param res ComputeJobResult. + * @param rcvd List of ComputeJobResult. + * @return ComputeJobResultPolicy. */ public ComputeJobResultPolicy result(ComputeJobResult res, List rcvd) { IgniteException err = res.getException(); @@ -158,7 +154,6 @@ public ComputeJobResultPolicy result(ComputeJobResult res, List Date: Mon, 16 Apr 2018 20:43:49 +0300 Subject: [PATCH 0125/1463] IGNITE-8242: Remove method GAGridUtils.getGenesForChromosome() as problematic when Chromosome contains duplicate genes. GAGridUtils.getGenesInOrderForChromosome() will be used instead. this closes #3813 (cherry picked from commit baaef27) --- .../OptimizeMakeChangeTerminateCriteria.java | 2 +- .../genetic/movie/MovieTerminateCriteria.java | 2 +- .../ignite/ml/genetic/utils/GAGridUtils.java | 27 ------------------- 3 files changed, 2 insertions(+), 29 deletions(-) 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 a4ea2636b6be2..ab3f462aae768 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 @@ -58,7 +58,7 @@ public boolean isTerminationConditionMet(Chromosome fittestChromosome, double av igniteLogger.info("Generation: " + currentGeneration); igniteLogger.info("Fittest is Chromosome Key: " + fittestChromosome); igniteLogger.info("Chromsome: " + fittestChromosome); - printCoins(GAGridUtils.getGenesForChromosome(ignite, fittestChromosome)); + printCoins(GAGridUtils.getGenesInOrderForChromosome(ignite, fittestChromosome)); igniteLogger.info("Avg Chromsome Fitness: " + averageFitnessScore); igniteLogger.info("##########################################################################################"); 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 747d51441d21e..34a7331758203 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 @@ -61,7 +61,7 @@ public boolean isTerminationConditionMet(Chromosome fittestChromosome, double av igniteLogger.info("Generation: " + currentGeneration); igniteLogger.info("Fittest is Chromosome Key: " + fittestChromosome); igniteLogger.info("Chromsome: " + fittestChromosome); - printMovies(GAGridUtils.getGenesForChromosome(ignite, fittestChromosome)); + printMovies(GAGridUtils.getGenesInOrderForChromosome(ignite, fittestChromosome)); igniteLogger.info("##########################################################################################"); if (!(fittestChromosome.getFitnessScore() > 32)) { diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/utils/GAGridUtils.java b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/utils/GAGridUtils.java index 326cdda8f3769..e27ddac23854e 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/genetic/utils/GAGridUtils.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/genetic/utils/GAGridUtils.java @@ -59,33 +59,6 @@ public static List getChromosomes(Ignite ignite, String query) { return chromosomes; } - /** - * @param ignite Ignite - * @param chromosome Chromosome - * @return List of Genes - */ - public static List getGenesForChromosome(Ignite ignite, Chromosome chromosome) { - List genes = new ArrayList(); - IgniteCache cache = ignite.cache(GAGridConstants.GENE_CACHE); - StringBuffer sbSqlClause = new StringBuffer(); - sbSqlClause.append("_key IN "); - String sqlInClause = Arrays.toString(chromosome.getGenes()); - - sqlInClause = sqlInClause.replace("[", "("); - sqlInClause = sqlInClause.replace("]", ")"); - - sbSqlClause.append(sqlInClause); - - SqlQuery sql = new SqlQuery(Gene.class, sbSqlClause.toString()); - - try (QueryCursor> cursor = cache.query(sql)) { - for (Entry e : cursor) - genes.add(e.getValue()); - } - - return genes; - } - /** * Retrieve genes in order * From bad3e400a84648d2c3167d78e7f94cce3c91d412 Mon Sep 17 00:00:00 2001 From: Ivanov Petr Date: Wed, 25 Apr 2018 20:31:30 +0300 Subject: [PATCH 0126/1463] IGNITE-7821 Unify and improve Apache Ignite and Web Console Dockerfiles Signed-off-by: Andrey Gura --- modules/docker/.dockerignore | 1 + modules/docker/1.0.0/Dockerfile | 40 ------------- modules/docker/1.0.0/run.sh | 50 ---------------- modules/docker/1.1.0/Dockerfile | 40 ------------- modules/docker/1.1.0/run.sh | 50 ---------------- modules/docker/1.2.0/Dockerfile | 40 ------------- modules/docker/1.2.0/run.sh | 50 ---------------- modules/docker/1.3.0/Dockerfile | 40 ------------- modules/docker/1.3.0/run.sh | 50 ---------------- modules/docker/1.4.0/Dockerfile | 40 ------------- modules/docker/1.4.0/run.sh | 50 ---------------- modules/docker/1.5.0.final/Dockerfile | 40 ------------- modules/docker/1.5.0.final/run.sh | 50 ---------------- modules/docker/1.6.0/Dockerfile | 44 -------------- modules/docker/1.6.0/run.sh | 51 ---------------- modules/docker/1.7.0/Dockerfile | 44 -------------- modules/docker/1.7.0/run.sh | 51 ---------------- modules/docker/1.8.0/Dockerfile | 46 --------------- modules/docker/1.8.0/run.sh | 51 ---------------- modules/docker/1.9.0/Dockerfile | 46 --------------- modules/docker/1.9.0/run.sh | 51 ---------------- modules/docker/2.0.0/Dockerfile | 46 --------------- modules/docker/2.0.0/run.sh | 51 ---------------- modules/docker/2.1.0/Dockerfile | 46 --------------- modules/docker/2.1.0/run.sh | 51 ---------------- modules/docker/2.2.0/Dockerfile | 46 --------------- modules/docker/2.2.0/run.sh | 51 ---------------- modules/docker/2.3.0/Dockerfile | 46 --------------- modules/docker/2.3.0/run.sh | 51 ---------------- modules/docker/Dockerfile | 34 ++++------- modules/docker/README.txt | 25 ++++++-- modules/docker/run.sh | 0 .../docker/standalone/.dockerignore | 5 -- .../web-console/docker/standalone/Dockerfile | 54 ++++++----------- .../web-console/docker/standalone/README.txt | 24 ++++++++ .../web-console/docker/standalone/build.sh | 59 ------------------- 36 files changed, 76 insertions(+), 1438 deletions(-) create mode 100644 modules/docker/.dockerignore delete mode 100644 modules/docker/1.0.0/Dockerfile delete mode 100644 modules/docker/1.0.0/run.sh delete mode 100644 modules/docker/1.1.0/Dockerfile delete mode 100644 modules/docker/1.1.0/run.sh delete mode 100644 modules/docker/1.2.0/Dockerfile delete mode 100644 modules/docker/1.2.0/run.sh delete mode 100644 modules/docker/1.3.0/Dockerfile delete mode 100644 modules/docker/1.3.0/run.sh delete mode 100644 modules/docker/1.4.0/Dockerfile delete mode 100644 modules/docker/1.4.0/run.sh delete mode 100644 modules/docker/1.5.0.final/Dockerfile delete mode 100644 modules/docker/1.5.0.final/run.sh delete mode 100644 modules/docker/1.6.0/Dockerfile delete mode 100644 modules/docker/1.6.0/run.sh delete mode 100644 modules/docker/1.7.0/Dockerfile delete mode 100644 modules/docker/1.7.0/run.sh delete mode 100644 modules/docker/1.8.0/Dockerfile delete mode 100644 modules/docker/1.8.0/run.sh delete mode 100644 modules/docker/1.9.0/Dockerfile delete mode 100644 modules/docker/1.9.0/run.sh delete mode 100644 modules/docker/2.0.0/Dockerfile delete mode 100644 modules/docker/2.0.0/run.sh delete mode 100644 modules/docker/2.1.0/Dockerfile delete mode 100644 modules/docker/2.1.0/run.sh delete mode 100644 modules/docker/2.2.0/Dockerfile delete mode 100644 modules/docker/2.2.0/run.sh delete mode 100644 modules/docker/2.3.0/Dockerfile delete mode 100644 modules/docker/2.3.0/run.sh mode change 100644 => 100755 modules/docker/run.sh create mode 100644 modules/web-console/docker/standalone/README.txt delete mode 100755 modules/web-console/docker/standalone/build.sh diff --git a/modules/docker/.dockerignore b/modules/docker/.dockerignore new file mode 100644 index 0000000000000..c4c4ffc6aa41a --- /dev/null +++ b/modules/docker/.dockerignore @@ -0,0 +1 @@ +*.zip diff --git a/modules/docker/1.0.0/Dockerfile b/modules/docker/1.0.0/Dockerfile deleted file mode 100644 index 79c35c866d217..0000000000000 --- a/modules/docker/1.0.0/Dockerfile +++ /dev/null @@ -1,40 +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. -# - -# Start from a Java image. -FROM java:7 - -# Ignite version -ENV IGNITE_VERSION 1.0.0-incubating - -WORKDIR /opt/ignite - -ADD http://www.us.apache.org/dist/ignite/1.0.0/ignite-fabric-1.0.0-incubating.zip /opt/ignite/ignite.zip - -# Ignite home -ENV IGNITE_HOME /opt/ignite/ignite-fabric-1.0.0-incubating - -RUN unzip ignite.zip - -RUN rm ignite.zip - -# Copy sh files and set permission -ADD ./run.sh $IGNITE_HOME/ - -RUN chmod +x $IGNITE_HOME/run.sh - -CMD $IGNITE_HOME/run.sh \ No newline at end of file diff --git a/modules/docker/1.0.0/run.sh b/modules/docker/1.0.0/run.sh deleted file mode 100644 index dbf287118aa41..0000000000000 --- a/modules/docker/1.0.0/run.sh +++ /dev/null @@ -1,50 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -if [ ! -z "$OPTION_LIBS" ]; then - IFS=, LIBS_LIST=("$OPTION_LIBS") - - for lib in ${LIBS_LIST[@]}; do - cp -r $IGNITE_HOME/libs/optional/"$lib"/* \ - $IGNITE_HOME/libs/ - done -fi - -if [ ! -z "$EXTERNAL_LIBS" ]; then - IFS=, LIBS_LIST=("$EXTERNAL_LIBS") - - for lib in ${LIBS_LIST[@]}; do - echo $lib >> temp - done - - wget -i temp -P $IGNITE_HOME/libs - - rm temp -fi - -QUIET="" - -if [ "$IGNITE_QUIET" = "false" ]; then - QUIET="-v" -fi - -if [ -z $CONFIG_URI ]; then - $IGNITE_HOME/bin/ignite.sh $QUIET -else - $IGNITE_HOME/bin/ignite.sh $QUIET $CONFIG_URI -fi diff --git a/modules/docker/1.1.0/Dockerfile b/modules/docker/1.1.0/Dockerfile deleted file mode 100644 index f132309f887a6..0000000000000 --- a/modules/docker/1.1.0/Dockerfile +++ /dev/null @@ -1,40 +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. -# - -# Start from a Java image. -FROM java:7 - -# Ignite version -ENV IGNITE_VERSION 1.1.0-incubating - -WORKDIR /opt/ignite - -ADD http://www.us.apache.org/dist/ignite/1.1.0/apache-ignite-fabric-1.1.0-incubating-bin.zip /opt/ignite/ignite.zip - -# Ignite home -ENV IGNITE_HOME /opt/ignite/apache-ignite-fabric-1.1.0-incubating-bin - -RUN unzip ignite.zip - -RUN rm ignite.zip - -# Copy sh files and set permission -ADD ./run.sh $IGNITE_HOME/ - -RUN chmod +x $IGNITE_HOME/run.sh - -CMD $IGNITE_HOME/run.sh \ No newline at end of file diff --git a/modules/docker/1.1.0/run.sh b/modules/docker/1.1.0/run.sh deleted file mode 100644 index dbf287118aa41..0000000000000 --- a/modules/docker/1.1.0/run.sh +++ /dev/null @@ -1,50 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -if [ ! -z "$OPTION_LIBS" ]; then - IFS=, LIBS_LIST=("$OPTION_LIBS") - - for lib in ${LIBS_LIST[@]}; do - cp -r $IGNITE_HOME/libs/optional/"$lib"/* \ - $IGNITE_HOME/libs/ - done -fi - -if [ ! -z "$EXTERNAL_LIBS" ]; then - IFS=, LIBS_LIST=("$EXTERNAL_LIBS") - - for lib in ${LIBS_LIST[@]}; do - echo $lib >> temp - done - - wget -i temp -P $IGNITE_HOME/libs - - rm temp -fi - -QUIET="" - -if [ "$IGNITE_QUIET" = "false" ]; then - QUIET="-v" -fi - -if [ -z $CONFIG_URI ]; then - $IGNITE_HOME/bin/ignite.sh $QUIET -else - $IGNITE_HOME/bin/ignite.sh $QUIET $CONFIG_URI -fi diff --git a/modules/docker/1.2.0/Dockerfile b/modules/docker/1.2.0/Dockerfile deleted file mode 100644 index b4b1a4abc47f0..0000000000000 --- a/modules/docker/1.2.0/Dockerfile +++ /dev/null @@ -1,40 +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. -# - -# Start from a Java image. -FROM java:7 - -# Ignite version -ENV IGNITE_VERSION 1.2.0-incubating - -WORKDIR /opt/ignite - -ADD http://www.us.apache.org/dist/ignite/1.2.0/apache-ignite-fabric-1.2.0-incubating-bin.zip /opt/ignite/ignite.zip - -# Ignite home -ENV IGNITE_HOME /opt/ignite/apache-ignite-fabric-1.2.0-incubating-bin - -RUN unzip ignite.zip - -RUN rm ignite.zip - -# Copy sh files and set permission -ADD ./run.sh $IGNITE_HOME/ - -RUN chmod +x $IGNITE_HOME/run.sh - -CMD $IGNITE_HOME/run.sh \ No newline at end of file diff --git a/modules/docker/1.2.0/run.sh b/modules/docker/1.2.0/run.sh deleted file mode 100644 index dbf287118aa41..0000000000000 --- a/modules/docker/1.2.0/run.sh +++ /dev/null @@ -1,50 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -if [ ! -z "$OPTION_LIBS" ]; then - IFS=, LIBS_LIST=("$OPTION_LIBS") - - for lib in ${LIBS_LIST[@]}; do - cp -r $IGNITE_HOME/libs/optional/"$lib"/* \ - $IGNITE_HOME/libs/ - done -fi - -if [ ! -z "$EXTERNAL_LIBS" ]; then - IFS=, LIBS_LIST=("$EXTERNAL_LIBS") - - for lib in ${LIBS_LIST[@]}; do - echo $lib >> temp - done - - wget -i temp -P $IGNITE_HOME/libs - - rm temp -fi - -QUIET="" - -if [ "$IGNITE_QUIET" = "false" ]; then - QUIET="-v" -fi - -if [ -z $CONFIG_URI ]; then - $IGNITE_HOME/bin/ignite.sh $QUIET -else - $IGNITE_HOME/bin/ignite.sh $QUIET $CONFIG_URI -fi diff --git a/modules/docker/1.3.0/Dockerfile b/modules/docker/1.3.0/Dockerfile deleted file mode 100644 index eade9d99c4978..0000000000000 --- a/modules/docker/1.3.0/Dockerfile +++ /dev/null @@ -1,40 +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. -# - -# Start from a Java image. -FROM java:7 - -# Ignite version -ENV IGNITE_VERSION 1.3.0-incubating - -WORKDIR /opt/ignite - -ADD http://www.us.apache.org/dist/ignite/1.3.0/apache-ignite-fabric-1.3.0-incubating-bin.zip /opt/ignite/ignite.zip - -# Ignite home -ENV IGNITE_HOME /opt/ignite/apache-ignite-fabric-1.3.0-incubating-bin - -RUN unzip ignite.zip - -RUN rm ignite.zip - -# Copy sh files and set permission -ADD ./run.sh $IGNITE_HOME/ - -RUN chmod +x $IGNITE_HOME/run.sh - -CMD $IGNITE_HOME/run.sh \ No newline at end of file diff --git a/modules/docker/1.3.0/run.sh b/modules/docker/1.3.0/run.sh deleted file mode 100644 index dbf287118aa41..0000000000000 --- a/modules/docker/1.3.0/run.sh +++ /dev/null @@ -1,50 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -if [ ! -z "$OPTION_LIBS" ]; then - IFS=, LIBS_LIST=("$OPTION_LIBS") - - for lib in ${LIBS_LIST[@]}; do - cp -r $IGNITE_HOME/libs/optional/"$lib"/* \ - $IGNITE_HOME/libs/ - done -fi - -if [ ! -z "$EXTERNAL_LIBS" ]; then - IFS=, LIBS_LIST=("$EXTERNAL_LIBS") - - for lib in ${LIBS_LIST[@]}; do - echo $lib >> temp - done - - wget -i temp -P $IGNITE_HOME/libs - - rm temp -fi - -QUIET="" - -if [ "$IGNITE_QUIET" = "false" ]; then - QUIET="-v" -fi - -if [ -z $CONFIG_URI ]; then - $IGNITE_HOME/bin/ignite.sh $QUIET -else - $IGNITE_HOME/bin/ignite.sh $QUIET $CONFIG_URI -fi diff --git a/modules/docker/1.4.0/Dockerfile b/modules/docker/1.4.0/Dockerfile deleted file mode 100644 index 41f6a68aba30b..0000000000000 --- a/modules/docker/1.4.0/Dockerfile +++ /dev/null @@ -1,40 +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. -# - -# Start from a Java image. -FROM java:7 - -# Ignite version -ENV IGNITE_VERSION 1.4.0 - -WORKDIR /opt/ignite - -ADD http://www.us.apache.org/dist/ignite/1.4.0/apache-ignite-fabric-1.4.0-bin.zip /opt/ignite/ignite.zip - -# Ignite home -ENV IGNITE_HOME /opt/ignite/apache-ignite-fabric-1.4.0-bin - -RUN unzip ignite.zip - -RUN rm ignite.zip - -# Copy sh files and set permission -ADD ./run.sh $IGNITE_HOME/ - -RUN chmod +x $IGNITE_HOME/run.sh - -CMD $IGNITE_HOME/run.sh \ No newline at end of file diff --git a/modules/docker/1.4.0/run.sh b/modules/docker/1.4.0/run.sh deleted file mode 100644 index dbf287118aa41..0000000000000 --- a/modules/docker/1.4.0/run.sh +++ /dev/null @@ -1,50 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -if [ ! -z "$OPTION_LIBS" ]; then - IFS=, LIBS_LIST=("$OPTION_LIBS") - - for lib in ${LIBS_LIST[@]}; do - cp -r $IGNITE_HOME/libs/optional/"$lib"/* \ - $IGNITE_HOME/libs/ - done -fi - -if [ ! -z "$EXTERNAL_LIBS" ]; then - IFS=, LIBS_LIST=("$EXTERNAL_LIBS") - - for lib in ${LIBS_LIST[@]}; do - echo $lib >> temp - done - - wget -i temp -P $IGNITE_HOME/libs - - rm temp -fi - -QUIET="" - -if [ "$IGNITE_QUIET" = "false" ]; then - QUIET="-v" -fi - -if [ -z $CONFIG_URI ]; then - $IGNITE_HOME/bin/ignite.sh $QUIET -else - $IGNITE_HOME/bin/ignite.sh $QUIET $CONFIG_URI -fi diff --git a/modules/docker/1.5.0.final/Dockerfile b/modules/docker/1.5.0.final/Dockerfile deleted file mode 100644 index bdc7bafd27651..0000000000000 --- a/modules/docker/1.5.0.final/Dockerfile +++ /dev/null @@ -1,40 +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. -# - -# Start from a Java image. -FROM java:7 - -# Ignite version -ENV IGNITE_VERSION 1.5.0.final - -WORKDIR /opt/ignite - -ADD http://www.us.apache.org/dist/ignite/1.5.0.final/apache-ignite-fabric-1.5.0.final-bin.zip /opt/ignite/ignite.zip - -# Ignite home -ENV IGNITE_HOME /opt/ignite/apache-ignite-fabric-1.5.0.final-bin - -RUN unzip ignite.zip - -RUN rm ignite.zip - -# Copy sh files and set permission -ADD ./run.sh $IGNITE_HOME/ - -RUN chmod +x $IGNITE_HOME/run.sh - -CMD $IGNITE_HOME/run.sh \ No newline at end of file diff --git a/modules/docker/1.5.0.final/run.sh b/modules/docker/1.5.0.final/run.sh deleted file mode 100644 index dbf287118aa41..0000000000000 --- a/modules/docker/1.5.0.final/run.sh +++ /dev/null @@ -1,50 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -if [ ! -z "$OPTION_LIBS" ]; then - IFS=, LIBS_LIST=("$OPTION_LIBS") - - for lib in ${LIBS_LIST[@]}; do - cp -r $IGNITE_HOME/libs/optional/"$lib"/* \ - $IGNITE_HOME/libs/ - done -fi - -if [ ! -z "$EXTERNAL_LIBS" ]; then - IFS=, LIBS_LIST=("$EXTERNAL_LIBS") - - for lib in ${LIBS_LIST[@]}; do - echo $lib >> temp - done - - wget -i temp -P $IGNITE_HOME/libs - - rm temp -fi - -QUIET="" - -if [ "$IGNITE_QUIET" = "false" ]; then - QUIET="-v" -fi - -if [ -z $CONFIG_URI ]; then - $IGNITE_HOME/bin/ignite.sh $QUIET -else - $IGNITE_HOME/bin/ignite.sh $QUIET $CONFIG_URI -fi diff --git a/modules/docker/1.6.0/Dockerfile b/modules/docker/1.6.0/Dockerfile deleted file mode 100644 index 959771b2baf5f..0000000000000 --- a/modules/docker/1.6.0/Dockerfile +++ /dev/null @@ -1,44 +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. -# - -# Start from a Java image. -FROM java:8 - -# Ignite version -ENV IGNITE_VERSION 1.6.0 - -# Ignite home -ENV IGNITE_HOME /opt/ignite/apache-ignite-fabric-${IGNITE_VERSION}-bin - -# Do not rely on anything provided by base image(s), but be explicit, if they are installed already it is noop then -RUN apt-get update && apt-get install -y --no-install-recommends \ - unzip \ - curl \ - && rm -rf /var/lib/apt/lists/* - -WORKDIR /opt/ignite - -RUN curl http://www.us.apache.org/dist/ignite/${IGNITE_VERSION}/apache-ignite-fabric-${IGNITE_VERSION}-bin.zip -o ignite.zip \ - && unzip ignite.zip \ - && rm ignite.zip - -# Copy sh files and set permission -COPY ./run.sh $IGNITE_HOME/ - -RUN chmod +x $IGNITE_HOME/run.sh - -CMD $IGNITE_HOME/run.sh \ No newline at end of file diff --git a/modules/docker/1.6.0/run.sh b/modules/docker/1.6.0/run.sh deleted file mode 100644 index 3aafc30ba0b9e..0000000000000 --- a/modules/docker/1.6.0/run.sh +++ /dev/null @@ -1,51 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -if [ ! -z "$OPTION_LIBS" ]; then - IFS=, LIBS_LIST=("$OPTION_LIBS") - - for lib in ${LIBS_LIST[@]}; do - cp -r $IGNITE_HOME/libs/optional/"$lib"/* \ - $IGNITE_HOME/libs/ - done -fi - -if [ ! -z "$EXTERNAL_LIBS" ]; then - IFS=, LIBS_LIST=("$EXTERNAL_LIBS") - - for lib in ${LIBS_LIST[@]}; do - echo $lib >> temp - done - - wget -i temp -P $IGNITE_HOME/libs - - rm temp -fi - -QUIET="" - -if [ "$IGNITE_QUIET" = "false" ]; then - QUIET="-v" -fi - -if [ -z $CONFIG_URI ]; then - $IGNITE_HOME/bin/ignite.sh $QUIET -else - $IGNITE_HOME/bin/ignite.sh $QUIET $CONFIG_URI -fi - diff --git a/modules/docker/1.7.0/Dockerfile b/modules/docker/1.7.0/Dockerfile deleted file mode 100644 index 5565df6145f96..0000000000000 --- a/modules/docker/1.7.0/Dockerfile +++ /dev/null @@ -1,44 +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. -# - -# Start from a Java image. -FROM java:8 - -# Ignite version -ENV IGNITE_VERSION 1.7.0 - -# Ignite home -ENV IGNITE_HOME /opt/ignite/apache-ignite-fabric-${IGNITE_VERSION}-bin - -# Do not rely on anything provided by base image(s), but be explicit, if they are installed already it is noop then -RUN apt-get update && apt-get install -y --no-install-recommends \ - unzip \ - curl \ - && rm -rf /var/lib/apt/lists/* - -WORKDIR /opt/ignite - -RUN curl http://www.us.apache.org/dist/ignite/${IGNITE_VERSION}/apache-ignite-fabric-${IGNITE_VERSION}-bin.zip -o ignite.zip \ - && unzip ignite.zip \ - && rm ignite.zip - -# Copy sh files and set permission -COPY ./run.sh $IGNITE_HOME/ - -RUN chmod +x $IGNITE_HOME/run.sh - -CMD $IGNITE_HOME/run.sh \ No newline at end of file diff --git a/modules/docker/1.7.0/run.sh b/modules/docker/1.7.0/run.sh deleted file mode 100644 index 3aafc30ba0b9e..0000000000000 --- a/modules/docker/1.7.0/run.sh +++ /dev/null @@ -1,51 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -if [ ! -z "$OPTION_LIBS" ]; then - IFS=, LIBS_LIST=("$OPTION_LIBS") - - for lib in ${LIBS_LIST[@]}; do - cp -r $IGNITE_HOME/libs/optional/"$lib"/* \ - $IGNITE_HOME/libs/ - done -fi - -if [ ! -z "$EXTERNAL_LIBS" ]; then - IFS=, LIBS_LIST=("$EXTERNAL_LIBS") - - for lib in ${LIBS_LIST[@]}; do - echo $lib >> temp - done - - wget -i temp -P $IGNITE_HOME/libs - - rm temp -fi - -QUIET="" - -if [ "$IGNITE_QUIET" = "false" ]; then - QUIET="-v" -fi - -if [ -z $CONFIG_URI ]; then - $IGNITE_HOME/bin/ignite.sh $QUIET -else - $IGNITE_HOME/bin/ignite.sh $QUIET $CONFIG_URI -fi - diff --git a/modules/docker/1.8.0/Dockerfile b/modules/docker/1.8.0/Dockerfile deleted file mode 100644 index 13fb3defb61ec..0000000000000 --- a/modules/docker/1.8.0/Dockerfile +++ /dev/null @@ -1,46 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Start from a Java image. -FROM java:8 - -# Ignite version -ENV IGNITE_VERSION 1.8.0 - -# Ignite home -ENV IGNITE_HOME /opt/ignite/apache-ignite-fabric-${IGNITE_VERSION}-bin - -# Do not rely on anything provided by base image(s), but be explicit, if they are installed already it is noop then -RUN apt-get update && apt-get install -y --no-install-recommends \ - unzip \ - curl \ - && rm -rf /var/lib/apt/lists/* - -WORKDIR /opt/ignite - -RUN curl http://www.us.apache.org/dist/ignite/${IGNITE_VERSION}/apache-ignite-fabric-${IGNITE_VERSION}-bin.zip -o ignite.zip \ - && unzip ignite.zip \ - && rm ignite.zip - -# Copy sh files and set permission -COPY ./run.sh $IGNITE_HOME/ - -RUN chmod +x $IGNITE_HOME/run.sh - -CMD $IGNITE_HOME/run.sh - -EXPOSE 11211 47100 47500 49112 \ No newline at end of file diff --git a/modules/docker/1.8.0/run.sh b/modules/docker/1.8.0/run.sh deleted file mode 100644 index 3aafc30ba0b9e..0000000000000 --- a/modules/docker/1.8.0/run.sh +++ /dev/null @@ -1,51 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -if [ ! -z "$OPTION_LIBS" ]; then - IFS=, LIBS_LIST=("$OPTION_LIBS") - - for lib in ${LIBS_LIST[@]}; do - cp -r $IGNITE_HOME/libs/optional/"$lib"/* \ - $IGNITE_HOME/libs/ - done -fi - -if [ ! -z "$EXTERNAL_LIBS" ]; then - IFS=, LIBS_LIST=("$EXTERNAL_LIBS") - - for lib in ${LIBS_LIST[@]}; do - echo $lib >> temp - done - - wget -i temp -P $IGNITE_HOME/libs - - rm temp -fi - -QUIET="" - -if [ "$IGNITE_QUIET" = "false" ]; then - QUIET="-v" -fi - -if [ -z $CONFIG_URI ]; then - $IGNITE_HOME/bin/ignite.sh $QUIET -else - $IGNITE_HOME/bin/ignite.sh $QUIET $CONFIG_URI -fi - diff --git a/modules/docker/1.9.0/Dockerfile b/modules/docker/1.9.0/Dockerfile deleted file mode 100644 index 48ceba12cd692..0000000000000 --- a/modules/docker/1.9.0/Dockerfile +++ /dev/null @@ -1,46 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Start from a Java image. -FROM java:8 - -# Ignite version -ENV IGNITE_VERSION 1.9.0 - -# Ignite home -ENV IGNITE_HOME /opt/ignite/apache-ignite-fabric-${IGNITE_VERSION}-bin - -# Do not rely on anything provided by base image(s), but be explicit, if they are installed already it is noop then -RUN apt-get update && apt-get install -y --no-install-recommends \ - unzip \ - curl \ - && rm -rf /var/lib/apt/lists/* - -WORKDIR /opt/ignite - -RUN curl https://dist.apache.org/repos/dist/release/ignite/${IGNITE_VERSION}/apache-ignite-fabric-${IGNITE_VERSION}-bin.zip -o ignite.zip \ - && unzip ignite.zip \ - && rm ignite.zip - -# Copy sh files and set permission -COPY ./run.sh $IGNITE_HOME/ - -RUN chmod +x $IGNITE_HOME/run.sh - -CMD $IGNITE_HOME/run.sh - -EXPOSE 11211 47100 47500 49112 \ No newline at end of file diff --git a/modules/docker/1.9.0/run.sh b/modules/docker/1.9.0/run.sh deleted file mode 100644 index 3aafc30ba0b9e..0000000000000 --- a/modules/docker/1.9.0/run.sh +++ /dev/null @@ -1,51 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -if [ ! -z "$OPTION_LIBS" ]; then - IFS=, LIBS_LIST=("$OPTION_LIBS") - - for lib in ${LIBS_LIST[@]}; do - cp -r $IGNITE_HOME/libs/optional/"$lib"/* \ - $IGNITE_HOME/libs/ - done -fi - -if [ ! -z "$EXTERNAL_LIBS" ]; then - IFS=, LIBS_LIST=("$EXTERNAL_LIBS") - - for lib in ${LIBS_LIST[@]}; do - echo $lib >> temp - done - - wget -i temp -P $IGNITE_HOME/libs - - rm temp -fi - -QUIET="" - -if [ "$IGNITE_QUIET" = "false" ]; then - QUIET="-v" -fi - -if [ -z $CONFIG_URI ]; then - $IGNITE_HOME/bin/ignite.sh $QUIET -else - $IGNITE_HOME/bin/ignite.sh $QUIET $CONFIG_URI -fi - diff --git a/modules/docker/2.0.0/Dockerfile b/modules/docker/2.0.0/Dockerfile deleted file mode 100644 index 229298184f155..0000000000000 --- a/modules/docker/2.0.0/Dockerfile +++ /dev/null @@ -1,46 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Start from a Java image. -FROM java:8 - -# Ignite version -ENV IGNITE_VERSION 2.0.0 - -# Ignite home -ENV IGNITE_HOME /opt/ignite/apache-ignite-fabric-${IGNITE_VERSION}-bin - -# Do not rely on anything provided by base image(s), but be explicit, if they are installed already it is noop then -RUN apt-get update && apt-get install -y --no-install-recommends \ - unzip \ - curl \ - && rm -rf /var/lib/apt/lists/* - -WORKDIR /opt/ignite - -RUN curl https://dist.apache.org/repos/dist/release/ignite/${IGNITE_VERSION}/apache-ignite-fabric-${IGNITE_VERSION}-bin.zip -o ignite.zip \ - && unzip ignite.zip \ - && rm ignite.zip - -# Copy sh files and set permission -COPY ./run.sh $IGNITE_HOME/ - -RUN chmod +x $IGNITE_HOME/run.sh - -CMD $IGNITE_HOME/run.sh - -EXPOSE 11211 47100 47500 49112 \ No newline at end of file diff --git a/modules/docker/2.0.0/run.sh b/modules/docker/2.0.0/run.sh deleted file mode 100644 index 3aafc30ba0b9e..0000000000000 --- a/modules/docker/2.0.0/run.sh +++ /dev/null @@ -1,51 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -if [ ! -z "$OPTION_LIBS" ]; then - IFS=, LIBS_LIST=("$OPTION_LIBS") - - for lib in ${LIBS_LIST[@]}; do - cp -r $IGNITE_HOME/libs/optional/"$lib"/* \ - $IGNITE_HOME/libs/ - done -fi - -if [ ! -z "$EXTERNAL_LIBS" ]; then - IFS=, LIBS_LIST=("$EXTERNAL_LIBS") - - for lib in ${LIBS_LIST[@]}; do - echo $lib >> temp - done - - wget -i temp -P $IGNITE_HOME/libs - - rm temp -fi - -QUIET="" - -if [ "$IGNITE_QUIET" = "false" ]; then - QUIET="-v" -fi - -if [ -z $CONFIG_URI ]; then - $IGNITE_HOME/bin/ignite.sh $QUIET -else - $IGNITE_HOME/bin/ignite.sh $QUIET $CONFIG_URI -fi - diff --git a/modules/docker/2.1.0/Dockerfile b/modules/docker/2.1.0/Dockerfile deleted file mode 100644 index 6a0eecd4a043f..0000000000000 --- a/modules/docker/2.1.0/Dockerfile +++ /dev/null @@ -1,46 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Start from a Java image. -FROM java:8 - -# Ignite version -ENV IGNITE_VERSION 2.1.0 - -# Ignite home -ENV IGNITE_HOME /opt/ignite/apache-ignite-fabric-${IGNITE_VERSION}-bin - -# Do not rely on anything provided by base image(s), but be explicit, if they are installed already it is noop then -RUN apt-get update && apt-get install -y --no-install-recommends \ - unzip \ - curl \ - && rm -rf /var/lib/apt/lists/* - -WORKDIR /opt/ignite - -RUN curl https://dist.apache.org/repos/dist/release/ignite/${IGNITE_VERSION}/apache-ignite-fabric-${IGNITE_VERSION}-bin.zip -o ignite.zip \ - && unzip ignite.zip \ - && rm ignite.zip - -# Copy sh files and set permission -COPY ./run.sh $IGNITE_HOME/ - -RUN chmod +x $IGNITE_HOME/run.sh - -CMD $IGNITE_HOME/run.sh - -EXPOSE 11211 47100 47500 49112 \ No newline at end of file diff --git a/modules/docker/2.1.0/run.sh b/modules/docker/2.1.0/run.sh deleted file mode 100644 index 3aafc30ba0b9e..0000000000000 --- a/modules/docker/2.1.0/run.sh +++ /dev/null @@ -1,51 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -if [ ! -z "$OPTION_LIBS" ]; then - IFS=, LIBS_LIST=("$OPTION_LIBS") - - for lib in ${LIBS_LIST[@]}; do - cp -r $IGNITE_HOME/libs/optional/"$lib"/* \ - $IGNITE_HOME/libs/ - done -fi - -if [ ! -z "$EXTERNAL_LIBS" ]; then - IFS=, LIBS_LIST=("$EXTERNAL_LIBS") - - for lib in ${LIBS_LIST[@]}; do - echo $lib >> temp - done - - wget -i temp -P $IGNITE_HOME/libs - - rm temp -fi - -QUIET="" - -if [ "$IGNITE_QUIET" = "false" ]; then - QUIET="-v" -fi - -if [ -z $CONFIG_URI ]; then - $IGNITE_HOME/bin/ignite.sh $QUIET -else - $IGNITE_HOME/bin/ignite.sh $QUIET $CONFIG_URI -fi - diff --git a/modules/docker/2.2.0/Dockerfile b/modules/docker/2.2.0/Dockerfile deleted file mode 100644 index 4c09653270f52..0000000000000 --- a/modules/docker/2.2.0/Dockerfile +++ /dev/null @@ -1,46 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Start from a Java image. -FROM openjdk:8 - -# Ignite version -ENV IGNITE_VERSION 2.2.0 - -# Ignite home -ENV IGNITE_HOME /opt/ignite/apache-ignite-fabric-${IGNITE_VERSION}-bin - -# Do not rely on anything provided by base image(s), but be explicit, if they are installed already it is noop then -RUN apt-get update && apt-get install -y --no-install-recommends \ - unzip \ - curl \ - && rm -rf /var/lib/apt/lists/* - -WORKDIR /opt/ignite - -RUN curl https://dist.apache.org/repos/dist/release/ignite/${IGNITE_VERSION}/apache-ignite-fabric-${IGNITE_VERSION}-bin.zip -o ignite.zip \ - && unzip ignite.zip \ - && rm ignite.zip - -# Copy sh files and set permission -COPY ./run.sh $IGNITE_HOME/ - -RUN chmod +x $IGNITE_HOME/run.sh - -CMD $IGNITE_HOME/run.sh - -EXPOSE 11211 47100 47500 49112 \ No newline at end of file diff --git a/modules/docker/2.2.0/run.sh b/modules/docker/2.2.0/run.sh deleted file mode 100644 index 3aafc30ba0b9e..0000000000000 --- a/modules/docker/2.2.0/run.sh +++ /dev/null @@ -1,51 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -if [ ! -z "$OPTION_LIBS" ]; then - IFS=, LIBS_LIST=("$OPTION_LIBS") - - for lib in ${LIBS_LIST[@]}; do - cp -r $IGNITE_HOME/libs/optional/"$lib"/* \ - $IGNITE_HOME/libs/ - done -fi - -if [ ! -z "$EXTERNAL_LIBS" ]; then - IFS=, LIBS_LIST=("$EXTERNAL_LIBS") - - for lib in ${LIBS_LIST[@]}; do - echo $lib >> temp - done - - wget -i temp -P $IGNITE_HOME/libs - - rm temp -fi - -QUIET="" - -if [ "$IGNITE_QUIET" = "false" ]; then - QUIET="-v" -fi - -if [ -z $CONFIG_URI ]; then - $IGNITE_HOME/bin/ignite.sh $QUIET -else - $IGNITE_HOME/bin/ignite.sh $QUIET $CONFIG_URI -fi - diff --git a/modules/docker/2.3.0/Dockerfile b/modules/docker/2.3.0/Dockerfile deleted file mode 100644 index 88ec8ba090848..0000000000000 --- a/modules/docker/2.3.0/Dockerfile +++ /dev/null @@ -1,46 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Start from a Java image. -FROM openjdk:8 - -# Ignite version -ENV IGNITE_VERSION 2.3.0 - -# Ignite home -ENV IGNITE_HOME /opt/ignite/apache-ignite-fabric-${IGNITE_VERSION}-bin - -# Do not rely on anything provided by base image(s), but be explicit, if they are installed already it is noop then -RUN apt-get update && apt-get install -y --no-install-recommends \ - unzip \ - curl \ - && rm -rf /var/lib/apt/lists/* - -WORKDIR /opt/ignite - -RUN curl https://dist.apache.org/repos/dist/release/ignite/${IGNITE_VERSION}/apache-ignite-fabric-${IGNITE_VERSION}-bin.zip -o ignite.zip \ - && unzip ignite.zip \ - && rm ignite.zip - -# Copy sh files and set permission -COPY ./run.sh $IGNITE_HOME/ - -RUN chmod +x $IGNITE_HOME/run.sh - -CMD $IGNITE_HOME/run.sh - -EXPOSE 11211 47100 47500 49112 \ No newline at end of file diff --git a/modules/docker/2.3.0/run.sh b/modules/docker/2.3.0/run.sh deleted file mode 100644 index 3aafc30ba0b9e..0000000000000 --- a/modules/docker/2.3.0/run.sh +++ /dev/null @@ -1,51 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -if [ ! -z "$OPTION_LIBS" ]; then - IFS=, LIBS_LIST=("$OPTION_LIBS") - - for lib in ${LIBS_LIST[@]}; do - cp -r $IGNITE_HOME/libs/optional/"$lib"/* \ - $IGNITE_HOME/libs/ - done -fi - -if [ ! -z "$EXTERNAL_LIBS" ]; then - IFS=, LIBS_LIST=("$EXTERNAL_LIBS") - - for lib in ${LIBS_LIST[@]}; do - echo $lib >> temp - done - - wget -i temp -P $IGNITE_HOME/libs - - rm temp -fi - -QUIET="" - -if [ "$IGNITE_QUIET" = "false" ]; then - QUIET="-v" -fi - -if [ -z $CONFIG_URI ]; then - $IGNITE_HOME/bin/ignite.sh $QUIET -else - $IGNITE_HOME/bin/ignite.sh $QUIET $CONFIG_URI -fi - diff --git a/modules/docker/Dockerfile b/modules/docker/Dockerfile index 88ec8ba090848..2cfa85f902391 100644 --- a/modules/docker/Dockerfile +++ b/modules/docker/Dockerfile @@ -15,32 +15,24 @@ # limitations under the License. # -# Start from a Java image. -FROM openjdk:8 - -# Ignite version -ENV IGNITE_VERSION 2.3.0 - -# Ignite home -ENV IGNITE_HOME /opt/ignite/apache-ignite-fabric-${IGNITE_VERSION}-bin - -# Do not rely on anything provided by base image(s), but be explicit, if they are installed already it is noop then -RUN apt-get update && apt-get install -y --no-install-recommends \ - unzip \ - curl \ - && rm -rf /var/lib/apt/lists/* +# Start from Java 8 based on Alpine Linux image (~5Mb) +FROM openjdk:8-jre-alpine +# Settings +ENV IGNITE_HOME /opt/ignite/apache-ignite-fabric WORKDIR /opt/ignite -RUN curl https://dist.apache.org/repos/dist/release/ignite/${IGNITE_VERSION}/apache-ignite-fabric-${IGNITE_VERSION}-bin.zip -o ignite.zip \ - && unzip ignite.zip \ - && rm ignite.zip +# Add missing software +RUN apk --update --no-cache add bash -# Copy sh files and set permission -COPY ./run.sh $IGNITE_HOME/ +# Copy main binary archive +COPY apache-ignite-fabric* apache-ignite-fabric -RUN chmod +x $IGNITE_HOME/run.sh +# Copy sh files and set permission +COPY run.sh $IGNITE_HOME/ +# Entry point CMD $IGNITE_HOME/run.sh -EXPOSE 11211 47100 47500 49112 \ No newline at end of file +# Container port exposure +EXPOSE 11211 47100 47500 49112 diff --git a/modules/docker/README.txt b/modules/docker/README.txt index 0299d4cc6a1b8..6912bc4745e4a 100644 --- a/modules/docker/README.txt +++ b/modules/docker/README.txt @@ -1,11 +1,26 @@ Apache Ignite Docker module ------------------------- +=========================== +Apache Ignite Docker module provides Dockerfile and accompanying files for building docker image. -Apache Ignite Docker module provides Dockerfile and bash scripts for building docker image. Build image -------------------------------------- +=========== +1) Build Apache Ignite binary archive as described in DEVNOTES.txt. -cd to ./modules/docker +2) Goto Apache Ignite's Docker module directory -sudo docker build . \ No newline at end of file + cd modules/docker + +3) Copy Apache Ignite's binary archive to Docker module directory + + cp -rfv ../../target/bin/apache-ignite-fabric-*.zip + +4) Unpack Apache Ignite's binary archive + + unzip apache-ignite-fabric-*.zip + +5) Build docker image + + docker build . -t apacheignite/ignite[:] + + Prepared image will be available issuing `docker images` command diff --git a/modules/docker/run.sh b/modules/docker/run.sh old mode 100644 new mode 100755 diff --git a/modules/web-console/docker/standalone/.dockerignore b/modules/web-console/docker/standalone/.dockerignore index c59189ec92ce3..0951480111120 100644 --- a/modules/web-console/docker/standalone/.dockerignore +++ b/modules/web-console/docker/standalone/.dockerignore @@ -1,7 +1,2 @@ -build/frontend/build -build/frontend/node_modules -build/frontend/ignite_modules_temp build/frontend/test -build/backend/config/*.json -build/backend/node_modules build/backend/test diff --git a/modules/web-console/docker/standalone/Dockerfile b/modules/web-console/docker/standalone/Dockerfile index bfb79016e2897..2fd5f56908848 100644 --- a/modules/web-console/docker/standalone/Dockerfile +++ b/modules/web-console/docker/standalone/Dockerfile @@ -20,37 +20,19 @@ FROM ubuntu:14.04 ENV NPM_CONFIG_LOGLEVEL info ENV NODE_VERSION 8.11.1 -# Before package list update. -RUN set -ex && \ - for key in \ - 9554F04D7259F04124DE6B476D5A82AC7E37093B \ - 94AE36675C464D64BAFA68DD7434390BDBE9B9C5 \ - FD3A5288F042B6850C66B31F09FE44734EB7990E \ - 71DCFD284A79C3B38668286BC97EC7A07EDE3FC1 \ - DD8F2338BAE7501E3DD5AC78C273792F7D83545D \ - B9AE9905FFD7803F25714661B63B535A4C206CA9 \ - C4F0DFFF4E8C1A8236409D08E73BC641CC11F4C8 \ - 56730D5401028683275BD23C23EFEFE93C4CFFFE \ - ; do \ - gpg --keyserver ha.pool.sks-keyservers.net --recv-keys "$key" || \ - gpg --keyserver pgp.mit.edu --recv-keys "$key" || \ - gpg --keyserver keyserver.pgp.com --recv-keys "$key" ; \ - done - RUN apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv EA312927 && \ - echo "deb http://repo.mongodb.org/apt/ubuntu trusty/mongodb-org/3.2 multiverse" | sudo tee /etc/apt/sources.list.d/mongodb-org-3.2.list + echo "deb http://repo.mongodb.org/apt/ubuntu trusty/mongodb-org/3.2 multiverse" | sudo tee /etc/apt/sources.list.d/mongodb-org-3.2.list # Update package list & install. RUN apt-get update && \ apt-get install -y nginx-light mongodb-org-server curl xz-utils git # Install Node JS. -RUN curl -SLO "https://nodejs.org/dist/v$NODE_VERSION/node-v$NODE_VERSION-linux-x64.tar.xz" && \ - curl -SLO "https://nodejs.org/dist/v$NODE_VERSION/SHASUMS256.txt.asc" && \ - gpg --batch --decrypt --output SHASUMS256.txt SHASUMS256.txt.asc && \ - grep " node-v$NODE_VERSION-linux-x64.tar.xz\$" SHASUMS256.txt | sha256sum -c - && \ - tar -xJf "node-v$NODE_VERSION-linux-x64.tar.xz" -C /usr/local --strip-components=1 && \ - rm "node-v$NODE_VERSION-linux-x64.tar.xz" SHASUMS256.txt.asc SHASUMS256.txt +RUN curl -SLO "https://nodejs.org/dist/v$NODE_VERSION/node-v$NODE_VERSION-linux-x64.tar.xz" && \ + curl -SLO "https://nodejs.org/dist/v$NODE_VERSION/SHASUMS256.txt" && \ + grep " node-v$NODE_VERSION-linux-x64.tar.xz\$" SHASUMS256.txt | sha256sum -c - && \ + tar -xJf "node-v$NODE_VERSION-linux-x64.tar.xz" -C /usr/local --strip-components=1 && \ + rm -rf "node-v$NODE_VERSION-linux-x64.tar.xz" SHASUMS256.txt # Install global node packages. RUN npm install -g pm2 @@ -62,23 +44,22 @@ RUN mkdir -p /opt/web-console WORKDIR /opt/web-console COPY build . -# Install node modules. -RUN cd /opt/web-console/frontend && npm install --no-optional --prod && npm run build -RUN cd /opt/web-console/backend && npm install --only=production --no-optional - -# Returns to base path. -WORKDIR /opt/web-console +# Install node modules for frontend and backend modules. +RUN cd /opt/web-console/frontend && \ + npm install --no-optional --prod && \ + npm run build && \ + cd /opt/web-console/backend && \ + npm install --no-optional --prod # Copy nginx config. -COPY ./nginx/nginx.conf /etc/nginx/nginx.conf -COPY ./nginx/web-console.conf /etc/nginx/web-console.conf +COPY nginx/* /etc/nginx/ -# Setup entrypoint. -COPY ./entrypoint.sh . -RUN chmod 755 /opt/web-console/entrypoint.sh +# Copy entrypoint. +COPY entrypoint.sh . # Clean up. -RUN apt-get clean && rm -rf /var/lib/apt/lists/* /tmp/* /var/tmp/* +RUN apt-get clean && \ + rm -rf /var/lib/apt/lists/* /tmp/* /var/tmp/* VOLUME ["/etc/nginx"] VOLUME ["/var/lib/mongodb"] @@ -87,3 +68,4 @@ VOLUME ["/opt/web-console/serve/agent_dists"] EXPOSE 80 ENTRYPOINT ["/opt/web-console/entrypoint.sh"] + diff --git a/modules/web-console/docker/standalone/README.txt b/modules/web-console/docker/standalone/README.txt new file mode 100644 index 0000000000000..68cfa8e98da58 --- /dev/null +++ b/modules/web-console/docker/standalone/README.txt @@ -0,0 +1,24 @@ +Web Console Docker module +========================= +Web Console Docker module provides Dockerfile and accompanying files for building docker image. + + +Build image +=========== +1) Build Apache Ignite binary archive as described in DEVNOTES.txt. + +2) Goto Web Console's Docker module directory + + cd modules/web-console/docker/standalone + +3) Copy build-related necessary files + + mkdir -pv build + cp -rf ../../frontend ../../backend build + cp -rfv ../../web-agent/target/ignite-web-agent-*.zip build/backend/agent_dists/ + +4) Build docker image + + docker build . -t apacheignite/web-console-standalone[:] + + Prepared image will be available issuing `docker images` command diff --git a/modules/web-console/docker/standalone/build.sh b/modules/web-console/docker/standalone/build.sh deleted file mode 100755 index c32dc403ba92d..0000000000000 --- a/modules/web-console/docker/standalone/build.sh +++ /dev/null @@ -1,59 +0,0 @@ -#!/bin/bash -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -if [ -z "$IGNITE_HOME" ]; then - echo "Ignite source folder is not found or IGNITE_HOME environment variable is not valid." - - exit 1 -fi - -WORK_DIR=`cd "$(dirname "$0")"; pwd` - -BUILD_DIR="$WORK_DIR/build" - -IGNITE_WEB_CONSOLE_DIR="$IGNITE_HOME/modules/web-console" -DOCKER_IMAGE_NAME="apacheignite/web-console-standalone" - -echo "Receiving version..." -VERSION=`cd $IGNITE_HOME && mvn org.apache.maven.plugins:maven-help-plugin:evaluate -Dexpression=project.version| grep -Ev '(^\[|Download\w+:)'` -RELEASE_VERSION=${VERSION%-SNAPSHOT} - -echo "Building $DOCKER_IMAGE_NAME:$RELEASE_VERSION" -echo "Step 1. Prepare build temp paths." -cd $WORK_DIR -rm -Rf $BUILD_DIR -docker rmi -f $DOCKER_IMAGE_NAME:$RELEASE_VERSION -mkdir -p $BUILD_DIR/frontend $BUILD_DIR/backend - -echo "Step 2. Build ignite web agent." -cd $IGNITE_HOME -mvn versions:set -DnewVersion=$RELEASE_VERSION -DgenerateBackupPoms=false -Pweb-console -DartifactId='*' -mvn clean package -pl :ignite-web-agent -am -P web-console -DskipTests=true -mvn versions:set -DnewVersion=$VERSION -DgenerateBackupPoms=false -Pweb-console -DartifactId='*' - -echo "Step 3. Copy sources." -cd $WORK_DIR -cp -r $IGNITE_WEB_CONSOLE_DIR/frontend/. $BUILD_DIR/frontend -cp -r $IGNITE_WEB_CONSOLE_DIR/backend/. $BUILD_DIR/backend -cp $IGNITE_HOME/modules/web-console/web-agent/target/ignite-web-agent*.zip $BUILD_DIR/backend/agent_dists/. - -echo "Step 4. Build docker image." -docker build -f=./Dockerfile -t $DOCKER_IMAGE_NAME:$RELEASE_VERSION -t $DOCKER_IMAGE_NAME:latest . - -echo "Step 5. Cleanup." -rm -Rf $BUILD_DIR From 55c50a1c4309246e9cd254ed1f25b41b89f9a959 Mon Sep 17 00:00:00 2001 From: Evgeny Stanilovskiy Date: Wed, 25 Apr 2018 20:46:47 +0300 Subject: [PATCH 0127/1463] IGNITE-8066 AssertionError while trying to archive wal segment. - Fixes #3837. Signed-off-by: dpavlov (cherry picked from commit 1fb2369) --- .../GridCacheDatabaseSharedManager.java | 11 +++-- .../wal/IgniteWalHistoryReservationsTest.java | 49 +++++++++++++++++-- 2 files changed, 52 insertions(+), 8 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 d5da19998d09f..42c7090a857f8 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 @@ -747,6 +747,9 @@ private void unRegistrateMetricsMBean() { WALPointer restore = restoreMemory(status); + if (restore == null && status.endPtr != CheckpointStatus.NULL_PTR) + throw new StorageException("Restore wal pointer = " + restore + ", while status.endPtr = " + status.endPtr + "."); + // First, bring memory to the last consistent checkpoint state if needed. // This method should return a pointer to the last valid record in the WAL. @@ -767,7 +770,7 @@ private void unRegistrateMetricsMBean() { catch (StorageException e) { cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e)); - throw new IgniteCheckedException(e); + throw e; } finally { checkpointReadUnlock(); @@ -1915,7 +1918,7 @@ private WALPointer readPointer(File cpMarkerFile, ByteBuffer buf) throws IgniteC * @throws IgniteCheckedException If failed. * @throws StorageException In case I/O error occurred during operations with storage. */ - private WALPointer restoreMemory(CheckpointStatus status) throws IgniteCheckedException { + private @Nullable WALPointer restoreMemory(CheckpointStatus status) throws IgniteCheckedException { return restoreMemory(status, false, (PageMemoryEx)metaStorage.pageMemory()); } @@ -1926,7 +1929,7 @@ private WALPointer restoreMemory(CheckpointStatus status) throws IgniteCheckedEx * @throws IgniteCheckedException If failed. * @throws StorageException In case I/O error occurred during operations with storage. */ - private WALPointer restoreMemory(CheckpointStatus status, boolean storeOnly, + private @Nullable WALPointer restoreMemory(CheckpointStatus status, boolean storeOnly, PageMemoryEx storePageMem) throws IgniteCheckedException { assert !storeOnly || storePageMem != null; @@ -2079,7 +2082,7 @@ else if (!F.eq(cpRec.checkpointId(), status.cpEndId)) if (status.needRestoreMemory()) { if (apply) - throw new IgniteCheckedException("Failed to restore memory state (checkpoint marker is present " + + throw new StorageException("Failed to restore memory state (checkpoint marker is present " + "on disk, but checkpoint record is missed in WAL) " + "[cpStatus=" + status + ", lastRead=" + lastRead + "]"); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java index 5ae16d2ce722a..e2c8bd1fce568 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java @@ -18,10 +18,10 @@ package org.apache.ignite.internal.processors.cache.persistence.db.wal; import java.util.Map; +import java.util.concurrent.Callable; import java.util.concurrent.locks.Lock; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteDataStreamer; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheWriteSynchronizationMode; @@ -31,12 +31,11 @@ import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.WALMode; +import org.apache.ignite.failure.StopNodeFailureHandler; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; -import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager; import org.apache.ignite.internal.util.lang.GridAbsPredicate; -import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -59,12 +58,15 @@ public class IgniteWalHistoryReservationsTest extends GridCommonAbstractTest { cfg.setConsistentId("NODE$" + gridName.charAt(gridName.length() - 1)); + cfg.setFailureHandler(new StopNodeFailureHandler()); + DataStorageConfiguration memCfg = new DataStorageConfiguration() .setDefaultDataRegionConfiguration( new DataRegionConfiguration() .setMaxSize(200 * 1024 * 1024) .setPersistenceEnabled(true)) - .setWalMode(WALMode.LOG_ONLY); + .setWalMode(WALMode.LOG_ONLY) + .setWalSegmentSize(512 * 1024); cfg.setDataStorageConfiguration(memCfg); @@ -352,6 +354,45 @@ public void testNodeIsClearedIfHistoryIsUnavailable() throws Exception { } } + /** + * @throws Exception If failed. + */ + public void testWalHistoryPartiallyRemoved() throws Exception { + int entryCnt = 10_000; + + IgniteEx ig0 = (IgniteEx) startGrids(2); + + ig0.cluster().active(true); + + IgniteCache cache = ig0.cache("cache1"); + + for (int k = 0; k < entryCnt; k++) + cache.put(k, k); + + GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + forceCheckpoint(); + + return null; + } + }); + + String nodeId0 = U.maskForFileName(ig0.localNode().consistentId().toString()); + + String walArchPath = ig0.configuration().getDataStorageConfiguration().getWalArchivePath(); + + stopAllGrids(); + + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), walArchPath + "/" + + nodeId0, false)); + + startGrid(0); + + Ignite ig1 = startGrid(1); + + ig1.cluster().active(true); + } + /** * @throws Exception If failed. */ From b329071afbd03362735c9e054cc844dc9ffb2f2d Mon Sep 17 00:00:00 2001 From: Sergey Chugunov Date: Wed, 25 Apr 2018 20:59:25 +0300 Subject: [PATCH 0128/1463] IGNITE-8372 ZookeeperClusterNode was made Externalizable to preserve serializing local node's metrics - Fixes #3907. Signed-off-by: Alexey Goncharuk --- .../zk/internal/ZookeeperClusterNode.java | 56 ++++++++++++++++++- .../zk/internal/ZookeeperDiscoveryImpl.java | 5 -- 2 files changed, 55 insertions(+), 6 deletions(-) diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java index 3cb5fad3129cf..2fe3052fa9d23 100644 --- a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java @@ -17,6 +17,10 @@ package org.apache.ignite.spi.discovery.zk.internal; +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; import java.io.Serializable; import java.util.Collection; import java.util.Collections; @@ -25,6 +29,7 @@ import java.util.UUID; import org.apache.ignite.cache.CacheMetrics; import org.apache.ignite.cluster.ClusterMetrics; +import org.apache.ignite.internal.ClusterMetricsSnapshot; import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.internal.managers.discovery.IgniteClusterNode; import org.apache.ignite.internal.util.tostring.GridToStringExclude; @@ -41,7 +46,7 @@ /** * Zookeeper Cluster Node. */ -public class ZookeeperClusterNode implements IgniteClusterNode, Serializable, Comparable { +public class ZookeeperClusterNode implements IgniteClusterNode, Externalizable, Comparable { /** */ private static final long serialVersionUID = 0L; @@ -99,6 +104,11 @@ public class ZookeeperClusterNode implements IgniteClusterNode, Serializable, Co @GridToStringExclude private transient volatile boolean daemonInit; + /** */ + public ZookeeperClusterNode() { + //No-op + } + /** * @param id Node ID. * @param addrs Node addresses. @@ -325,6 +335,50 @@ public void local(boolean loc) { return (CLIENT_NODE_MASK & flags) != 0; } + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + U.writeUuid(out, id); + out.writeObject(consistentId); + out.writeLong(internalId); + out.writeLong(order); + out.writeObject(ver); + U.writeMap(out, attrs); + U.writeCollection(out, addrs); + U.writeCollection(out, hostNames); + out.writeLong(sesTimeout); + out.writeByte(flags); + + // Cluster metrics + byte[] mtr = null; + + ClusterMetrics metrics = this.metrics; + + if (metrics != null) + mtr = ClusterMetricsSnapshot.serialize(metrics); + + U.writeByteArray(out, mtr); + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + id = U.readUuid(in); + consistentId = (Serializable) in.readObject(); + internalId = in.readLong(); + order = in.readLong(); + ver = (IgniteProductVersion) in.readObject(); + attrs = U.sealMap(U.readMap(in)); + addrs = U.readCollection(in); + hostNames = U.readCollection(in); + sesTimeout = in.readLong(); + flags = in.readByte(); + + // Cluster metrics + byte[] mtr = U.readByteArray(in); + + if (mtr != null) + metrics = ClusterMetricsSnapshot.deserialize(mtr, 0); + } + /** {@inheritDoc} */ @Override public int compareTo(@Nullable ZookeeperClusterNode node) { if (node == null) 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 74670acfc9647..0604458e461b6 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 @@ -51,7 +51,6 @@ import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CommunicationFailureResolver; import org.apache.ignite.events.EventType; -import org.apache.ignite.internal.ClusterMetricsSnapshot; import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; @@ -2955,8 +2954,6 @@ private void processLocalJoin(ZkDiscoveryEventsData evtsData, if (node.order() >= locNode.order()) break; - node.setMetrics(new ClusterMetricsSnapshot()); - rtState.top.addNode(node); } @@ -3448,8 +3445,6 @@ private void notifyNodeJoin(ZkJoinedNodeEvtData joinedEvtData, ZkJoiningNodeData joinedNode.order(joinedEvtData.topVer); joinedNode.internalId(joinedEvtData.joinedInternalId); - joinedNode.setMetrics(new ClusterMetricsSnapshot()); - rtState.top.addNode(joinedNode); final List topSnapshot = rtState.top.topologySnapshot(); From 9de5fce4470e63e19be1882c021672bf87b4cb44 Mon Sep 17 00:00:00 2001 From: shtykh_roman Date: Thu, 26 Apr 2018 12:27:44 +0300 Subject: [PATCH 0129/1463] IGNITE-8342: SQL: Fixed parameter parsing in CREATE INDEX command leading to infinite loop. This closes #3893. --- .../sql/command/SqlCreateIndexCommand.java | 38 +++++++++---------- .../sql/SqlParserCreateIndexSelfTest.java | 2 + 2 files changed, 19 insertions(+), 21 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlCreateIndexCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlCreateIndexCommand.java index f3f38d470a5a7..2f3197395bfbc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlCreateIndexCommand.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlCreateIndexCommand.java @@ -21,7 +21,6 @@ import org.apache.ignite.internal.sql.SqlLexer; import org.apache.ignite.internal.sql.SqlLexerTokenType; import org.apache.ignite.internal.sql.SqlLexerToken; -import org.apache.ignite.internal.sql.SqlParserUtils; 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; @@ -175,7 +174,7 @@ public Collection columns() { } /** - * Pasrse index name. + * Parse index name. * * @param lex Lexer. * @return Index name. @@ -249,34 +248,31 @@ private void parseIndexProperties(SqlLexer lex) { while (true) { SqlLexerToken token = lex.lookAhead(); - if (token.tokenType() == SqlLexerTokenType.EOF) - return; + if (token.tokenType() != SqlLexerTokenType.DEFAULT) + break; - if (token.tokenType() == SqlLexerTokenType.DEFAULT) { - switch (token.token()) { - case PARALLEL: - parallel = getIntProperty(lex, PARALLEL, foundProps); + switch (token.token()) { + case PARALLEL: + parallel = getIntProperty(lex, PARALLEL, foundProps); - if (parallel < 0) - throw error(lex, "Illegal " + PARALLEL + " value. Should be positive: " + parallel); + if (parallel < 0) + throw error(lex, "Illegal " + PARALLEL + " value. Should be positive: " + parallel); - break; + break; - case INLINE_SIZE: - inlineSize = getIntProperty(lex, INLINE_SIZE, foundProps); + case INLINE_SIZE: + inlineSize = getIntProperty(lex, INLINE_SIZE, foundProps); - if (inlineSize < 0) - throw error(lex, "Illegal " + INLINE_SIZE + - " value. Should be positive: " + inlineSize); + if (inlineSize < 0) + throw error(lex, "Illegal " + INLINE_SIZE + + " value. Should be positive: " + inlineSize); - break; + break; - default: - return; - } + default: + return; } } - } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserCreateIndexSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserCreateIndexSelfTest.java index 80328ab0dfd50..465e8d15c9204 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserCreateIndexSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserCreateIndexSelfTest.java @@ -46,8 +46,10 @@ public class SqlParserCreateIndexSelfTest extends SqlParserAbstractSelfTest { public void testCreateIndex() throws Exception { // Base. parseValidate(null, "CREATE INDEX idx ON tbl(a)", null, "TBL", "IDX", DEFAULT_PROPS, "A", false); + parseValidate(null, "CREATE INDEX idx ON tbl(a);", null, "TBL", "IDX", DEFAULT_PROPS, "A", false); parseValidate(null, "CREATE INDEX idx ON tbl(a ASC)", null, "TBL", "IDX", DEFAULT_PROPS, "A", false); parseValidate(null, "CREATE INDEX idx ON tbl(a DESC)", null, "TBL", "IDX", DEFAULT_PROPS, "A", true); + assertParseError(null, "CREATE INDEX idx ON tbl(a) ,", "Unexpected token: \",\""); // Case (in)sensitivity. parseValidate(null, "CREATE INDEX IDX ON TBL(COL)", null, "TBL", "IDX", DEFAULT_PROPS, "COL", false); From 88b5742977564bd5a18b5ffff7bea25f6efa9ed2 Mon Sep 17 00:00:00 2001 From: shtykh_roman Date: Thu, 26 Apr 2018 12:55:29 +0300 Subject: [PATCH 0130/1463] IGNITE-8052: SQL: clear error message when using a non-existing column name for CREATE TABLE primary key. This closes #3701. --- .../query/h2/sql/GridSqlQueryParser.java | 4 +++- .../cache/index/H2DynamicTableSelfTest.java | 18 ++++++++++++++++++ 2 files changed, 21 insertions(+), 1 deletion(-) 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 2d2c25ce643f0..133333e265c27 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 @@ -1041,7 +1041,9 @@ private GridSqlCreateTable parseCreateTable(CreateTable createTbl) { for (IndexColumn pkIdxCol : pkIdxCols) { GridSqlColumn gridCol = cols.get(pkIdxCol.columnName); - assert gridCol != null; + if (gridCol == null) + throw new IgniteSQLException("PRIMARY KEY column is not defined: " + pkIdxCol.columnName, + IgniteQueryErrorCode.PARSING); pkCols.add(gridCol.columnName()); } 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 82247114fc062..cd41419a17ca8 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 @@ -72,6 +72,7 @@ /** * Tests for CREATE/DROP TABLE. */ +@SuppressWarnings("ThrowableNotThrown") public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest { /** Client node index. */ private static final int CLIENT = 2; @@ -719,6 +720,23 @@ public void testCreateExistingTable() throws Exception { }, IgniteSQLException.class, "Table already exists: Person"); } + /** + * Test that attempting to use a non-existing column name for the primary key when {@code CREATE TABLE} + * yields an error. + * @throws Exception if failed. + */ + public void testCreateTableWithWrongColumnNameAsKey() throws Exception { + GridTestUtils.assertThrows(null, new Callable() { + @Override public Object call() throws Exception { + execute("CREATE TABLE \"Person\" (\"id\" int, \"city\" varchar" + + ", \"name\" varchar, \"surname\" varchar, \"age\" int, PRIMARY KEY (\"id\", \"c_ity\")) WITH " + + "\"template=cache\""); + + return null; + } + }, IgniteSQLException.class, "PRIMARY KEY column is not defined: c_ity"); + } + /** * Test that {@code DROP TABLE} actually removes specified cache and type descriptor on all nodes. * @throws Exception if failed. From edd506ae913f778fe2c9a9abb0e95e3e9890acee Mon Sep 17 00:00:00 2001 From: vk Date: Thu, 26 Apr 2018 13:39:10 +0300 Subject: [PATCH 0131/1463] IGNITE-6133: Added clearNodeLocalMap() method for IgniteMXBean. This closes #2582. --- .../apache/ignite/internal/IgniteKernal.java | 5 ++++ .../apache/ignite/mxbean/IgniteMXBean.java | 6 +++++ .../internal/GridNodeLocalSelfTest.java | 23 +++++++++++++++++++ 3 files changed, 34 insertions(+) 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 dc1307cd18dc5..8305d4ee239ed 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 @@ -4334,6 +4334,11 @@ private boolean unregisterMBean(ObjectName mbean) { new ArrayList(ctx.cluster().get().forServers().forRemotes().nodes())); } + /** {@inheritDoc} */ + @Override public void clearNodeLocalMap() { + ctx.cluster().get().clearNodeMap(); + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(IgniteKernal.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java index 308c148d11e8c..77ea035309f79 100644 --- a/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java +++ b/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java @@ -475,4 +475,10 @@ void runIoTest( int payLoadSize, boolean procFromNioThread ); + + /** + * Clears node local map. + */ + @MXBeanDescription("Clears local node map.") + void clearNodeLocalMap(); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridNodeLocalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridNodeLocalSelfTest.java index dccd225887a97..8d8b59f476041 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridNodeLocalSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridNodeLocalSelfTest.java @@ -23,6 +23,7 @@ import org.apache.ignite.internal.util.lang.GridTuple3; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.mxbean.IgniteMXBean; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.ignite.testframework.junits.common.GridCommonTest; @@ -63,4 +64,26 @@ public void testNodeLocal() throws Exception { assert "Hello world!".equals(nl.get(keyStr)); assert (Integer)nl.get(key) == 12; } + + /** + * Test that node local map is cleared via {@link IgniteMXBean#clearNodeLocalMap()}. + * + * @throws Exception if test failed. + */ + public void testClearNodeLocalMap() throws Exception { + final String key = "key"; + final String value = "value"; + + Ignite grid = G.ignite(getTestIgniteInstanceName()); + + ConcurrentMap nodeLocalMap = grid.cluster().nodeLocalMap(); + nodeLocalMap.put(key, value); + + assert !nodeLocalMap.isEmpty() : "Empty node local map"; + assert nodeLocalMap.containsKey(key); + + IgniteMXBean igniteMXBean = (IgniteMXBean)grid; + igniteMXBean.clearNodeLocalMap(); + assert nodeLocalMap.isEmpty() : "Not empty node local map"; + } } \ No newline at end of file From 05f9280a240882ef10cde4986f8a28dd10b84eaf Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Thu, 26 Apr 2018 16:06:52 +0300 Subject: [PATCH 0132/1463] IGNITE-8390 Correct assertion for historical rebalance - Fixes #3917. Signed-off-by: Alexey Goncharuk --- .../preloader/GridDhtPartitionSupplier.java | 2 +- .../GridDhtPartitionSupplyMessage.java | 5 +- .../db/wal/IgniteWalRebalanceTest.java | 63 +++++++++++++++++-- 3 files changed, 62 insertions(+), 8 deletions(-) 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 a3ee305406207..84e6828df4dc3 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 @@ -378,7 +378,7 @@ public void handleDemandMessage(int topicId, UUID nodeId, GridDhtPartitionDemand info.cacheId(row.cacheId()); if (preloadPred == null || preloadPred.apply(info)) - s.addEntry0(part, info, grp.shared(), grp.cacheObjectContext()); + 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 " + 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 4ae5acdbaf2c9..77baa38cc52c3 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 @@ -209,15 +209,16 @@ int messageSize() { /** * @param p Partition. + * @param historical {@code True} if partition rebalancing using WAL history. * @param info Entry to add. * @param ctx Cache shared context. * @param cacheObjCtx Cache object context. * @throws IgniteCheckedException If failed. */ - void addEntry0(int p, GridCacheEntryInfo info, GridCacheSharedContext ctx, CacheObjectContext cacheObjCtx) throws IgniteCheckedException { + void addEntry0(int p, boolean historical, GridCacheEntryInfo info, GridCacheSharedContext ctx, CacheObjectContext cacheObjCtx) throws IgniteCheckedException { assert info != null; assert info.key() != null : info; - assert info.value() != null : info; + assert info.value() != null || historical : info; // Need to call this method to initialize info properly. marshalInfo(info, ctx, cacheObjCtx); 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 6387dac1a3ef0..23dda265fb5eb 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,7 +17,7 @@ package org.apache.ignite.internal.processors.cache.persistence.db.wal; -import java.util.concurrent.TimeUnit; +import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; @@ -30,6 +30,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.WALMode; import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -48,6 +49,8 @@ public class IgniteWalRebalanceTest extends GridCommonAbstractTest { IgniteConfiguration cfg = super.getConfiguration(gridName); + cfg.setConsistentId(gridName); + CacheConfiguration ccfg = new CacheConfiguration<>(CACHE_NAME); ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); @@ -87,9 +90,11 @@ public class IgniteWalRebalanceTest extends GridCommonAbstractTest { } /** + * Test simple WAL historical rebalance. + * * @throws Exception if failed. */ - public void test() throws Exception { + public void testSimple() throws Exception { IgniteEx ig0 = startGrid(0); IgniteEx ig1 = startGrid(1); final int entryCnt = 10_000; @@ -112,12 +117,60 @@ public void test() throws Exception { ig1 = startGrid(1); - IgniteCache cache1 = ig1.cache(CACHE_NAME); + awaitPartitionMapExchange(); + + for (Ignite ig : G.allGrids()) { + IgniteCache cache1 = ig.cache(CACHE_NAME); + + for (int k = 0; k < entryCnt; k++) + assertEquals(new IndexedObject(k + 1), cache1.get(k)); + } + } + + /** + * Test that cache entry removes are rebalanced properly using WAL. + * + * @throws Exception If failed. + */ + public void testRebalanceRemoves() throws Exception { + IgniteEx ig0 = startGrid(0); + IgniteEx ig1 = startGrid(1); + final int entryCnt = 10_000; + + ig0.cluster().active(true); - cache1.rebalance().get(2, TimeUnit.MINUTES); + IgniteCache cache = ig0.cache(CACHE_NAME); for (int k = 0; k < entryCnt; k++) - assertEquals(new IndexedObject(k + 1), cache.get(k)); + cache.put(k, new IndexedObject(k)); + + forceCheckpoint(); + + stopGrid(1, false); + + for (int k = 0; k < entryCnt; k++) { + if (k % 3 != 2) + cache.put(k, new IndexedObject(k + 1)); + else // Spread removes across all partitions. + cache.remove(k); + } + + forceCheckpoint(); + + ig1 = startGrid(1); + + awaitPartitionMapExchange(); + + for (Ignite ig : G.allGrids()) { + IgniteCache cache1 = ig.cache(CACHE_NAME); + + for (int k = 0; k < entryCnt; k++) { + if (k % 3 != 2) + assertEquals(new IndexedObject(k + 1), cache1.get(k)); + else + assertNull(cache1.get(k)); + } + } } /** From e190e35a97ffe1f7a1cbc8a2803bb9b9f301a417 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Thu, 26 Apr 2018 19:36:25 +0300 Subject: [PATCH 0133/1463] IGNITE-8394: ODBC: Fixed async establishing of SSL connection (cherry picked from commit 1840f75) --- .../cpp/odbc-test/src/queries_ssl_test.cpp | 1 + .../ignite/odbc/ssl/secure_socket_client.h | 57 +++- .../include/ignite/odbc/ssl/ssl_bindings.h | 146 ++++++--- .../include/ignite/odbc/ssl/ssl_gateway.h | 16 +- .../os/win/src/system/tcp_socket_client.cpp | 1 - .../cpp/odbc/src/ssl/secure_socket_client.cpp | 298 +++++++++++------- .../cpp/odbc/src/ssl/ssl_gateway.cpp | 37 ++- 7 files changed, 378 insertions(+), 178 deletions(-) diff --git a/modules/platforms/cpp/odbc-test/src/queries_ssl_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_ssl_test.cpp index 47eb5dc33695a..861bef1890412 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_ssl_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_ssl_test.cpp @@ -19,6 +19,7 @@ #include #include +#include #ifndef _MSC_VER # define BOOST_TEST_DYN_LINK diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/ssl/secure_socket_client.h b/modules/platforms/cpp/odbc/include/ignite/odbc/ssl/secure_socket_client.h index f6da30d7551dd..f8ca0be3d81e0 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/ssl/secure_socket_client.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/ssl/secure_socket_client.h @@ -55,7 +55,7 @@ namespace ignite * * @param hostname Host name or address. * @param port TCP port. - * @param timeout Timeout. + * @param timeout Timeout in seconds. * @param diag Diagnostics collector to use for error-reporting. * @return @c true on success and @c false on fail. */ @@ -71,7 +71,7 @@ namespace ignite * Send data using connection. * @param data Data to send. * @param size Number of bytes to send. - * @param timeout Timeout. + * @param timeout Timeout in seconds. * @return Number of bytes that have been sent on success, * WaitResult::TIMEOUT on timeout and -errno on failure. */ @@ -82,7 +82,7 @@ namespace ignite * * @param buffer Pointer to data buffer. * @param size Size of the buffer in bytes. - * @param timeout Timeout. + * @param timeout Timeout in seconds. * @return Number of bytes that have been received on success, * WaitResult::TIMEOUT on timeout and -errno on failure. */ @@ -106,12 +106,13 @@ namespace ignite * This function uses poll to achive timeout functionality * for every separate socket operation. * - * @param timeout Timeout. + * @param ssl SSL instance. + * @param timeout Timeout in seconds. * @param rd Wait for read if @c true, or for write if @c false. * @return -errno on error, WaitResult::TIMEOUT on timeout and * WaitResult::SUCCESS on success. */ - int WaitOnSocket(int32_t timeout, bool rd); + static int WaitOnSocket(void* ssl, int32_t timeout, bool rd); /** * Make new context instance. @@ -120,11 +121,51 @@ namespace ignite * @param keyPath Private key file path. * @param caPath Certificate authority file path. * @param diag Diagnostics collector to use for error-reporting. - * @return New context instance on success and null-opinter on fail. + * @return New context instance on success and null-pointer on fail. */ static void* MakeContext(const std::string& certPath, const std::string& keyPath, const std::string& caPath, diagnostic::Diagnosable& diag); + /** + * Make new SSL instance. + * + * @param context SSL context. + * @param hostname Host name or address. + * @param port TCP port. + * @param blocking Indicates if the resulted SSL is blocking or not. + * @param diag Diagnostics collector to use for error-reporting. + * @return New SSL instance on success and null-pointer on fail. + */ + static void* MakeSsl(void* context, const char* hostname, uint16_t port, + bool& blocking, diagnostic::Diagnosable& diag); + + /** + * Complete async connect. + * + * @param ssl SSL instance. + * @param timeout Timeout in seconds. + * @param diag Diagnostics collector to use for error-reporting. + * @return @c true on success. + */ + static bool CompleteConnectInternal(void* ssl, int timeout, diagnostic::Diagnosable& diag); + + /** + * Get SSL error. + * + * @param ssl SSL instance. + * @param ret Return value of the pervious operation. + * @return Error string. + */ + static std::string GetSslError(void* ssl, int ret); + + /** + * Check if a actual error occured. + * + * @param err SSL error code. + * @return @true if a actual error occured + */ + static bool IsActualError(int err); + /** Certificate file path. */ std::string certPath; @@ -137,8 +178,8 @@ namespace ignite /** SSL context. */ void* context; - /** OpenSSL I/O stream abstraction */ - void* sslBio; + /** OpenSSL instance */ + void* ssl; /** Blocking flag. */ bool blocking; diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/ssl/ssl_bindings.h b/modules/platforms/cpp/odbc/include/ignite/odbc/ssl/ssl_bindings.h index b23533a9062a0..9a1740dc41f7c 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/ssl/ssl_bindings.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/ssl/ssl_bindings.h @@ -20,6 +20,7 @@ #include #include +#include #include "ignite/odbc/ssl/ssl_gateway.h" @@ -173,84 +174,133 @@ namespace ignite TLSEXT_NAMETYPE_host_name, const_cast(name)); } + inline void SSL_set_connect_state_(SSL* s) + { + typedef void(FuncType)(SSL*); + + FuncType* fp = reinterpret_cast( + SslGateway::GetInstance().GetFunctions().fpSSL_set_connect_state); + return fp(s); + } - inline const SSL_METHOD *SSLv23_method() + inline int SSL_connect_(SSL* s) { - typedef const SSL_METHOD*(FuncType)(); + typedef int(FuncType)(SSL*); - FuncType* fp = reinterpret_cast(SslGateway::GetInstance().GetFunctions().fpSSLv23_method); + FuncType* fp = reinterpret_cast(SslGateway::GetInstance().GetFunctions().fpSSL_connect); - return fp(); + return fp(s); } - inline void OPENSSL_config(const char *configName) + inline int SSL_get_error_(const SSL *s, int ret) { - typedef void(FuncType)(const char*); + typedef int(FuncType)(const SSL*, int); - FuncType* fp = reinterpret_cast(SslGateway::GetInstance().GetFunctions().fpOPENSSL_config); + FuncType* fp = reinterpret_cast(SslGateway::GetInstance().GetFunctions().fpSSL_get_error); - fp(configName); + return fp(s, ret); } - inline void X509_free(X509 *a) + inline int SSL_want_(const SSL *s) { - typedef void(FuncType)(X509*); + typedef int(FuncType)(const SSL*); - FuncType* fp = reinterpret_cast(SslGateway::GetInstance().GetFunctions().fpX509_free); + FuncType* fp = reinterpret_cast(SslGateway::GetInstance().GetFunctions().fpSSL_want); - fp(a); + return fp(s); } - inline BIO *BIO_new_ssl_connect(SSL_CTX *ctx) + inline int SSL_write_(SSL *s, const void *buf, int num) { - typedef BIO*(FuncType)(SSL_CTX*); + typedef int(FuncType)(SSL*, const void*, int); - FuncType* fp = reinterpret_cast( - SslGateway::GetInstance().GetFunctions().fpBIO_new_ssl_connect); + FuncType* fp = reinterpret_cast(SslGateway::GetInstance().GetFunctions().fpSSL_write); - return fp(ctx); + return fp(s, buf, num); } - inline int BIO_write(BIO *b, const void *data, int len) + inline int SSL_read_(SSL *s, void *buf, int num) { - typedef int(FuncType)(BIO*, const void*, int); + typedef int(FuncType)(SSL*, void*, int); - FuncType* fp = reinterpret_cast(SslGateway::GetInstance().GetFunctions().fpBIO_write); + FuncType* fp = reinterpret_cast(SslGateway::GetInstance().GetFunctions().fpSSL_read); - return fp(b, data, len); + return fp(s, buf, num); } - inline int BIO_read(BIO *b, void *data, int len) + inline int SSL_pending_(const SSL *ssl) { - typedef int(FuncType)(BIO*, const void*, int); + typedef int(FuncType)(const SSL*); - FuncType* fp = reinterpret_cast(SslGateway::GetInstance().GetFunctions().fpBIO_read); + FuncType* fp = reinterpret_cast(SslGateway::GetInstance().GetFunctions().fpSSL_pending); - return fp(b, data, len); + return fp(ssl); } - inline void BIO_free_all(BIO *a) + inline int SSL_get_fd_(const SSL *ssl) { - typedef void(FuncType)(BIO*); + typedef int(FuncType)(const SSL*); - FuncType* fp = reinterpret_cast(SslGateway::GetInstance().GetFunctions().fpBIO_free_all); + FuncType* fp = reinterpret_cast(SslGateway::GetInstance().GetFunctions().fpSSL_get_fd); + + return fp(ssl); + } + + inline void SSL_free_(SSL *ssl) + { + typedef void(FuncType)(SSL*); + + FuncType* fp = reinterpret_cast(SslGateway::GetInstance().GetFunctions().fpSSL_free); + + fp(ssl); + } + + inline const SSL_METHOD *SSLv23_client_method_() + { + typedef const SSL_METHOD*(FuncType)(); + + FuncType* fp = reinterpret_cast( + SslGateway::GetInstance().GetFunctions().fpSSLv23_client_method); + + return fp(); + } + + inline void OPENSSL_config(const char *configName) + { + typedef void(FuncType)(const char*); + + FuncType* fp = reinterpret_cast(SslGateway::GetInstance().GetFunctions().fpOPENSSL_config); + + fp(configName); + } + + inline void X509_free(X509 *a) + { + typedef void(FuncType)(X509*); + + FuncType* fp = reinterpret_cast(SslGateway::GetInstance().GetFunctions().fpX509_free); fp(a); } - inline int BIO_test_flags(const BIO *b, int flags) + inline BIO *BIO_new_ssl_connect(SSL_CTX *ctx) { - typedef int(FuncType)(const BIO*, int); + typedef BIO*(FuncType)(SSL_CTX*); - FuncType* fp = reinterpret_cast(SslGateway::GetInstance().GetFunctions().fpBIO_test_flags); + FuncType* fp = reinterpret_cast( + SslGateway::GetInstance().GetFunctions().fpBIO_new_ssl_connect); - return fp(b, flags); + return fp(ctx); } - inline int BIO_should_retry_(const BIO *b) + inline void BIO_free_all(BIO *a) { - return ssl::BIO_test_flags(b, BIO_FLAGS_SHOULD_RETRY); + typedef void(FuncType)(BIO*); + + FuncType* fp = reinterpret_cast(SslGateway::GetInstance().GetFunctions().fpBIO_free_all); + + fp(a); } inline long BIO_ctrl(BIO *bp, int cmd, long larg, void *parg) @@ -267,16 +317,6 @@ namespace ignite return ssl::BIO_ctrl(bp, BIO_C_GET_FD, 0, reinterpret_cast(fd)); } - inline long BIO_do_handshake_(BIO *bp) - { - return ssl::BIO_ctrl(bp, BIO_C_DO_STATE_MACHINE, 0, NULL); - } - - inline long BIO_do_connect_(BIO *bp) - { - return ssl::BIO_do_handshake_(bp); - } - inline long BIO_get_ssl_(BIO *bp, SSL** ssl) { return ssl::BIO_ctrl(bp, BIO_C_GET_SSL, 0, reinterpret_cast(ssl)); @@ -292,9 +332,23 @@ namespace ignite return ssl::BIO_ctrl(bp, BIO_C_SET_CONNECT, 0, const_cast(name)); } - inline long BIO_pending_(BIO *bp) + inline unsigned long ERR_get_error_() + { + typedef unsigned long(FuncType)(); + + FuncType* fp = reinterpret_cast(SslGateway::GetInstance().GetFunctions().fpERR_get_error); + + return fp(); + } + + inline void ERR_error_string_n_(unsigned long e, char *buf, size_t len) { - return ssl::BIO_ctrl(bp, BIO_CTRL_PENDING, 0, NULL); + typedef void(FuncType)(unsigned long, char*, size_t); + + FuncType* fp = reinterpret_cast( + SslGateway::GetInstance().GetFunctions().fpERR_error_string_n); + + fp(e, buf, len); } } } diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/ssl/ssl_gateway.h b/modules/platforms/cpp/odbc/include/ignite/odbc/ssl/ssl_gateway.h index b131228f01fad..4b102ad28b972 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/ssl/ssl_gateway.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/ssl/ssl_gateway.h @@ -46,15 +46,23 @@ namespace ignite void *fpSSL_load_error_strings; void *fpSSL_get_peer_certificate; void *fpSSL_ctrl; - void *fpSSLv23_method; + void *fpSSLv23_client_method; + void *fpSSL_set_connect_state; + void *fpSSL_connect; + void *fpSSL_get_error; + void *fpSSL_want; + void *fpSSL_write; + void *fpSSL_read; + void *fpSSL_pending; + void *fpSSL_get_fd; + void *fpSSL_free; void *fpOPENSSL_config; void *fpX509_free; void *fpBIO_new_ssl_connect; - void *fpBIO_write; - void *fpBIO_read; void *fpBIO_free_all; - void *fpBIO_test_flags; void *fpBIO_ctrl; + void *fpERR_get_error; + void *fpERR_error_string_n; }; /** diff --git a/modules/platforms/cpp/odbc/os/win/src/system/tcp_socket_client.cpp b/modules/platforms/cpp/odbc/os/win/src/system/tcp_socket_client.cpp index e891ebf7b330a..38e8b977392d8 100644 --- a/modules/platforms/cpp/odbc/os/win/src/system/tcp_socket_client.cpp +++ b/modules/platforms/cpp/odbc/os/win/src/system/tcp_socket_client.cpp @@ -97,7 +97,6 @@ namespace ignite { namespace system { - TcpSocketClient::TcpSocketClient() : socketHandle(INVALID_SOCKET), blocking(true) diff --git a/modules/platforms/cpp/odbc/src/ssl/secure_socket_client.cpp b/modules/platforms/cpp/odbc/src/ssl/secure_socket_client.cpp index 602dec6116162..84eb1e8867f82 100644 --- a/modules/platforms/cpp/odbc/src/ssl/secure_socket_client.cpp +++ b/modules/platforms/cpp/odbc/src/ssl/secure_socket_client.cpp @@ -23,6 +23,7 @@ #include "ignite/odbc/system/tcp_socket_client.h" #include "ignite/odbc/ssl/secure_socket_client.h" #include "ignite/odbc/ssl/ssl_bindings.h" +#include "ignite/common/utils.h" #ifndef SOCKET_ERROR # define SOCKET_ERROR (-1) @@ -40,7 +41,7 @@ namespace ignite keyPath(keyPath), caPath(caPath), context(0), - sslBio(0), + ssl(0), blocking(true) { // No-op. @@ -72,115 +73,60 @@ namespace ignite } } - BIO* bio = ssl::BIO_new_ssl_connect(reinterpret_cast(context)); - if (!bio) - { - diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, "Can not create SSL connection."); - - return false; - } - - blocking = false; - long res = ssl::BIO_set_nbio_(bio, 1); - if (res != OPERATION_SUCCESS) - { - blocking = true; - - diag.AddStatusRecord(SqlState::S01S02_OPTION_VALUE_CHANGED, - "Can not set up non-blocking mode. Timeouts are not available."); - } - - std::stringstream stream; - stream << hostname << ":" << port; - - std::string address = stream.str(); - - res = ssl::BIO_set_conn_hostname_(bio, address.c_str()); - if (res != OPERATION_SUCCESS) - { - diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, "Can not set SSL connection hostname."); - - ssl::BIO_free_all(bio); - - return false; - } - - SSL* ssl = 0; - ssl::BIO_get_ssl_(bio, &ssl); - if (!ssl) - { - diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, "Can not get SSL instance from BIO."); - - ssl::BIO_free_all(bio); - + SSL* ssl0 = reinterpret_cast(MakeSsl(context, hostname, port, blocking, diag)); + if (!ssl0) return false; - } - res = ssl::SSL_set_tlsext_host_name_(ssl, hostname); + int res = ssl::SSL_set_tlsext_host_name_(ssl0, hostname); if (res != OPERATION_SUCCESS) { - diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, "Can not set host name for secure connection"); + diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, + "Can not set host name for secure connection: " + GetSslError(ssl0, res)); - ssl::BIO_free_all(bio); + ssl::SSL_free_(ssl0); return false; } - do - { - res = ssl::BIO_do_connect_(bio); - } while (ssl::BIO_should_retry_(bio)); + ssl::SSL_set_connect_state_(ssl0); - if (res != OPERATION_SUCCESS) - { - diag.AddStatusRecord(SqlState::S08001_CANNOT_CONNECT, - "Failed to establish secure connection with the host."); - - ssl::BIO_free_all(bio); - - return false; - } + bool connected = CompleteConnectInternal(ssl0, DEFALT_CONNECT_TIMEOUT, diag); - do + if (!connected) { - res = ssl::BIO_do_handshake_(bio); - } while (ssl::BIO_should_retry_(bio)); - - if (res != OPERATION_SUCCESS) - { - diag.AddStatusRecord(SqlState::S08001_CANNOT_CONNECT, "SSL handshake failed."); - - ssl::BIO_free_all(bio); + ssl::SSL_free_(ssl0); return false; } // Verify a server certificate was presented during the negotiation - X509* cert = ssl::SSL_get_peer_certificate(ssl); + X509* cert = ssl::SSL_get_peer_certificate(ssl0); if (cert) ssl::X509_free(cert); else { - diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, "Remote host did not provide certificate."); + diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, + "Remote host did not provide certificate: " + GetSslError(ssl0, res)); - ssl::BIO_free_all(bio); + ssl::SSL_free_(ssl0); return false; } // Verify the result of chain verification // Verification performed according to RFC 4158 - res = ssl::SSL_get_verify_result(ssl); + res = ssl::SSL_get_verify_result(ssl0); if (X509_V_OK != res) { - diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, "Certificate chain verification failed."); + diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, + "Certificate chain verification failed: " + GetSslError(ssl0, res)); - ssl::BIO_free_all(bio); + ssl::SSL_free_(ssl0); return false; } - sslBio = reinterpret_cast(bio); + ssl = reinterpret_cast(ssl0); return true; } @@ -194,22 +140,16 @@ namespace ignite { assert(SslGateway::GetInstance().Loaded()); - if (!sslBio) + if (!ssl) { LOG_MSG("Trying to send data using closed connection"); return -1; } - BIO* sslBio0 = reinterpret_cast(sslBio); + SSL* ssl0 = reinterpret_cast(ssl); - int res = 0; - - do - { - res = ssl::BIO_write(sslBio0, data, static_cast(size)); - } - while (ssl::BIO_should_retry_(sslBio0)); + int res = ssl::SSL_write_(ssl0, data, static_cast(size)); return res; } @@ -218,30 +158,26 @@ namespace ignite { assert(SslGateway::GetInstance().Loaded()); - if (!sslBio) + if (!ssl) { LOG_MSG("Trying to receive data using closed connection"); return -1; } - BIO* sslBio0 = reinterpret_cast(sslBio); + SSL* ssl0 = reinterpret_cast(ssl); int res = 0; - if (!blocking && BIO_pending_(sslBio0) == 0) + if (!blocking && ssl::SSL_pending_(ssl0) == 0) { - res = WaitOnSocket(timeout, true); + res = WaitOnSocket(ssl, timeout, true); if (res < 0 || res == WaitResult::TIMEOUT) return res; } - do - { - res = ssl::BIO_read(sslBio0, buffer, static_cast(size)); - } - while (ssl::BIO_should_retry_(sslBio0)); + res = ssl::SSL_read_(ssl0, buffer, static_cast(size)); return res; } @@ -277,10 +213,10 @@ namespace ignite } } - const SSL_METHOD* method = SSLv23_method(); + const SSL_METHOD* method = ssl::SSLv23_client_method_(); if (!method) { - diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, "Can not create new SSL method."); + diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, "Can not get SSL method."); return 0; } @@ -350,34 +286,180 @@ namespace ignite return ctx; } + void* SecureSocketClient::MakeSsl(void* context, const char* hostname, uint16_t port, + bool& blocking, diagnostic::Diagnosable& diag) + { + BIO* bio = ssl::BIO_new_ssl_connect(reinterpret_cast(context)); + if (!bio) + { + diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, "Can not create SSL connection."); + + return 0; + } + + blocking = false; + long res = ssl::BIO_set_nbio_(bio, 1); + if (res != OPERATION_SUCCESS) + { + blocking = true; + + diag.AddStatusRecord(SqlState::S01S02_OPTION_VALUE_CHANGED, + "Can not set up non-blocking mode. Timeouts are not available."); + } + + std::stringstream stream; + stream << hostname << ":" << port; + + std::string address = stream.str(); + + res = ssl::BIO_set_conn_hostname_(bio, address.c_str()); + if (res != OPERATION_SUCCESS) + { + diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, "Can not set SSL connection hostname."); + + ssl::BIO_free_all(bio); + + return 0; + } + + SSL* ssl = 0; + ssl::BIO_get_ssl_(bio, &ssl); + if (!ssl) + { + diag.AddStatusRecord(SqlState::SHY000_GENERAL_ERROR, "Can not get SSL instance from BIO."); + + ssl::BIO_free_all(bio); + + return 0; + } + + return ssl; + } + + bool SecureSocketClient::CompleteConnectInternal(void* ssl, int timeout, diagnostic::Diagnosable& diag) + { + SSL* ssl0 = reinterpret_cast(ssl); + + while (true) + { + int res = ssl::SSL_connect_(ssl0); + + if (res == OPERATION_SUCCESS) + return true; + + int sslError = ssl::SSL_get_error_(ssl0, res); + + LOG_MSG("wait res=" << res << ", sslError=" << sslError); + + if (IsActualError(sslError)) + { + diag.AddStatusRecord(SqlState::S08001_CANNOT_CONNECT, + "Can not establish secure connection: " + GetSslError(ssl0, res)); + + return false; + } + + int want = ssl::SSL_want_(ssl0); + + res = WaitOnSocket(ssl, timeout, want == SSL_READING); + + LOG_MSG("wait res=" << res << ", want=" << want); + + if (res == WaitResult::TIMEOUT) + { + diag.AddStatusRecord(SqlState::SHYT01_CONNECTION_TIMEOUT, + "Can not establish secure connection: Timeout expired (" + + common::LexicalCast(timeout) + " seconds)"); + + return false; + } + + if (res != WaitResult::SUCCESS) + { + diag.AddStatusRecord(SqlState::S08001_CANNOT_CONNECT, + "Can not establish secure connection due to internal error"); + + return false; + } + } + } + + std::string SecureSocketClient::GetSslError(void* ssl, int ret) + { + SSL* ssl0 = reinterpret_cast(ssl); + + int sslError = ssl::SSL_get_error_(ssl0, ret); + + LOG_MSG("ssl_error: " << sslError); + + switch (sslError) + { + case SSL_ERROR_NONE: + break; + + case SSL_ERROR_WANT_WRITE: + return std::string("SSL_connect wants write"); + + case SSL_ERROR_WANT_READ: + return std::string("SSL_connect wants read"); + + default: + return std::string("SSL error: ") + common::LexicalCast(sslError); + } + + long error = ssl::ERR_get_error_(); + + char errBuf[1024] = { 0 }; + + ssl::ERR_error_string_n_(error, errBuf, sizeof(errBuf)); + + return std::string(errBuf); + } + + bool SecureSocketClient::IsActualError(int err) + { + switch (err) + { + case SSL_ERROR_NONE: + case SSL_ERROR_WANT_WRITE: + case SSL_ERROR_WANT_READ: + case SSL_ERROR_WANT_CONNECT: + case SSL_ERROR_WANT_ACCEPT: + case SSL_ERROR_WANT_X509_LOOKUP: + return false; + + default: + return true; + } + } + void SecureSocketClient::CloseInteral() { assert(SslGateway::GetInstance().Loaded()); - if (sslBio) + if (ssl) { - ssl::BIO_free_all(reinterpret_cast(sslBio)); + ssl::SSL_free_(reinterpret_cast(ssl)); - sslBio = 0; + ssl = 0; } } - int SecureSocketClient::WaitOnSocket(int32_t timeout, bool rd) + int SecureSocketClient::WaitOnSocket(void* ssl, int32_t timeout, bool rd) { int ready = 0; int lastError = 0; - int fdSocket = 0; - BIO* sslBio0 = reinterpret_cast(sslBio); + SSL* ssl0 = reinterpret_cast(ssl); fd_set fds; - long res = ssl::BIO_get_fd_(sslBio0, &fdSocket); + int fd = ssl::SSL_get_fd_(ssl0); - if (res < 0) + if (fd < 0) { - LOG_MSG("Can not get file descriptor from the SSL socket: " << res); + LOG_MSG("Can not get file descriptor from the SSL socket: " << fd << ", " << GetSslError(ssl, fd)); - return res; + return fd; } do { @@ -385,7 +467,7 @@ namespace ignite tv.tv_sec = timeout; FD_ZERO(&fds); - FD_SET(static_cast(fdSocket), &fds); + FD_SET(static_cast(fd), &fds); fd_set* readFds = 0; fd_set* writeFds = 0; @@ -395,7 +477,7 @@ namespace ignite else writeFds = &fds; - ready = select(fdSocket + 1, readFds, writeFds, NULL, (timeout == 0 ? NULL : &tv)); + ready = select(fd + 1, readFds, writeFds, NULL, (timeout == 0 ? NULL : &tv)); if (ready == SOCKET_ERROR) lastError = system::TcpSocketClient::GetLastSocketError(); @@ -405,7 +487,7 @@ namespace ignite if (ready == SOCKET_ERROR) return -lastError; - lastError = system::TcpSocketClient::GetLastSocketError(fdSocket); + lastError = system::TcpSocketClient::GetLastSocketError(fd); if (lastError != 0) return -lastError; diff --git a/modules/platforms/cpp/odbc/src/ssl/ssl_gateway.cpp b/modules/platforms/cpp/odbc/src/ssl/ssl_gateway.cpp index 0fd7930dd4235..308302a069f8a 100644 --- a/modules/platforms/cpp/odbc/src/ssl/ssl_gateway.cpp +++ b/modules/platforms/cpp/odbc/src/ssl/ssl_gateway.cpp @@ -61,8 +61,6 @@ namespace ignite std::string home = GetEnv(ADDITIONAL_OPENSSL_HOME_ENV); - std::cout << ADDITIONAL_OPENSSL_HOME_ENV << std::endl; - if (home.empty()) home = GetEnv("OPENSSL_HOME"); @@ -147,18 +145,27 @@ namespace ignite functions.fpSSL_ctrl = LoadSslMethod("SSL_ctrl"); functions.fpSSL_CTX_ctrl = LoadSslMethod("SSL_CTX_ctrl"); - functions.fpSSLv23_method = LoadSslMethod("SSLv23_method"); + functions.fpSSLv23_client_method = LoadSslMethod("SSLv23_client_method"); + functions.fpSSL_set_connect_state = LoadSslMethod("SSL_set_connect_state"); + functions.fpSSL_connect = LoadSslMethod("SSL_connect"); + functions.fpSSL_get_error = LoadSslMethod("SSL_get_error"); + functions.fpSSL_want = LoadSslMethod("SSL_want"); + functions.fpSSL_write = LoadSslMethod("SSL_write"); + functions.fpSSL_read = LoadSslMethod("SSL_read"); + functions.fpSSL_pending = LoadSslMethod("SSL_pending"); + functions.fpSSL_get_fd = LoadSslMethod("SSL_get_fd"); + functions.fpSSL_free = LoadSslMethod("SSL_free"); functions.fpBIO_new_ssl_connect = LoadSslMethod("BIO_new_ssl_connect"); functions.fpOPENSSL_config = LoadSslMethod("OPENSSL_config"); functions.fpX509_free = LoadSslMethod("X509_free"); - functions.fpBIO_write = LoadSslMethod("BIO_write"); - functions.fpBIO_read = LoadSslMethod("BIO_read"); functions.fpBIO_free_all = LoadSslMethod("BIO_free_all"); - functions.fpBIO_test_flags = LoadSslMethod("BIO_test_flags"); functions.fpBIO_ctrl = LoadSslMethod("BIO_ctrl"); + functions.fpERR_get_error = LoadSslMethod("ERR_get_error"); + functions.fpERR_error_string_n = LoadSslMethod("ERR_error_string_n"); + bool allLoaded = functions.fpSSL_CTX_new != 0 && functions.fpSSL_CTX_free != 0 && @@ -174,15 +181,23 @@ namespace ignite functions.fpSSL_get_peer_certificate != 0 && functions.fpSSL_ctrl != 0 && functions.fpSSL_CTX_ctrl != 0 && - functions.fpSSLv23_method != 0 && + functions.fpSSLv23_client_method != 0 && + functions.fpSSL_set_connect_state != 0 && + functions.fpSSL_connect != 0 && + functions.fpSSL_get_error != 0 && + functions.fpSSL_want != 0 && + functions.fpSSL_write != 0 && + functions.fpSSL_read != 0 && + functions.fpSSL_pending != 0 && + functions.fpSSL_get_fd != 0 && + functions.fpSSL_free != 0 && functions.fpBIO_new_ssl_connect != 0 && functions.fpOPENSSL_config != 0 && functions.fpX509_free != 0 && - functions.fpBIO_write != 0 && - functions.fpBIO_read != 0 && functions.fpBIO_free_all != 0 && - functions.fpBIO_test_flags != 0 && - functions.fpBIO_ctrl != 0; + functions.fpBIO_ctrl != 0 && + functions.fpERR_get_error != 0 && + functions.fpERR_error_string_n != 0; if (!allLoaded) { From 4296fdd8f0731c5a839a45e0d8a142dcf5829ffa Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Thu, 26 Apr 2018 18:57:38 +0300 Subject: [PATCH 0134/1463] IGNITE-8277 Added utilities to check and display cache info --- .../internal/commandline/Arguments.java | 21 +- .../ignite/internal/commandline/Command.java | 5 +- .../internal/commandline/CommandHandler.java | 433 +++++++++++++++++- .../commandline/cache/CacheArguments.java | 163 +++++++ .../commandline/cache/CacheCommand.java | 93 ++++ .../processors/cache/verify/CacheInfo.java | 322 +++++++++++++ .../cache/verify/ContentionClosure.java | 162 +++++++ .../cache/verify/ContentionInfo.java | 72 +++ .../cache/verify/ViewCacheClosure.java | 185 ++++++++ .../visor/verify/IndexValidationIssue.java | 88 ++++ .../ValidateIndexesPartitionResult.java | 145 ++++++ .../verify/VisorContentionJobResult.java | 80 ++++ .../visor/verify/VisorContentionTask.java | 100 ++++ .../visor/verify/VisorContentionTaskArg.java | 84 ++++ .../verify/VisorContentionTaskResult.java | 100 ++++ .../visor/verify/VisorIdleAnalyzeTask.java | 124 +++++ .../visor/verify/VisorIdleAnalyzeTaskArg.java | 88 ++++ .../verify/VisorIdleAnalyzeTaskResult.java | 76 +++ .../visor/verify/VisorIdleVerifyTask.java | 97 ++++ .../visor/verify/VisorIdleVerifyTaskArg.java | 74 +++ .../verify/VisorIdleVerifyTaskResult.java | 76 +++ .../verify/VisorValidateIndexesJobResult.java | 73 +++ .../verify/VisorValidateIndexesTaskArg.java | 74 +++ .../VisorValidateIndexesTaskResult.java | 88 ++++ .../visor/verify/VisorViewCacheCmd.java | 47 ++ .../visor/verify/VisorViewCacheTask.java | 75 +++ .../visor/verify/VisorViewCacheTaskArg.java | 86 ++++ .../verify/VisorViewCacheTaskResult.java | 74 +++ .../resources/META-INF/classnames.properties | 114 ++++- .../ignite/util/GridCommandHandlerTest.java | 312 +++++++++++-- .../visor/verify/ValidateIndexesClosure.java | 356 ++++++++++++++ .../verify/VisorValidateIndexesTask.java | 99 ++++ .../util/GridCommandHandlerIndexingTest.java | 121 +++++ 33 files changed, 4026 insertions(+), 81 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheArguments.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommand.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/CacheInfo.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/ContentionClosure.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/ContentionInfo.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/ViewCacheClosure.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/IndexValidationIssue.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesPartitionResult.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorContentionJobResult.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorContentionTask.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorContentionTaskArg.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorContentionTaskResult.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleAnalyzeTask.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleAnalyzeTaskArg.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleAnalyzeTaskResult.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTask.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskArg.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskResult.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesJobResult.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTaskArg.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTaskResult.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorViewCacheCmd.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorViewCacheTask.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorViewCacheTaskArg.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorViewCacheTaskResult.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTask.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/Arguments.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/Arguments.java index 83a272b6ec12f..ce7269378fc20 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/Arguments.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/Arguments.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.commandline; import org.apache.ignite.internal.client.GridClientConfiguration; +import org.apache.ignite.internal.commandline.cache.CacheArguments; import org.apache.ignite.internal.visor.tx.VisorTxTaskArg; /** @@ -61,6 +62,11 @@ public class Arguments { /** Transaction arguments. */ private final VisorTxTaskArg txArg; + /** + * Arguments for --cache subcommand. + */ + private CacheArguments cacheArgs; + /** * @param cmd Command. * @param host Host. @@ -73,10 +79,11 @@ public class Arguments { * @param force Force flag. * @param pingTimeout Ping timeout. See {@link GridClientConfiguration#pingTimeout}. * @param pingInterval Ping interval. See {@link GridClientConfiguration#pingInterval}. + * @param cacheArgs --cache subcommand arguments. */ - public Arguments(Command cmd, String host, String port, String user, String pwd, - String baselineAct, String baselineArgs, long pingTimeout, - long pingInterval, VisorTxTaskArg txArg, boolean force) { + public Arguments(Command cmd, String host, String port, String user, String pwd, String baselineAct, + String baselineArgs, long pingTimeout, long pingInterval, VisorTxTaskArg txArg, boolean force, + CacheArguments cacheArgs) { this.cmd = cmd; this.host = host; this.port = port; @@ -88,6 +95,7 @@ public Arguments(Command cmd, String host, String port, String user, String pwd, this.pingInterval = pingInterval; this.force = force; this.txArg = txArg; + this.cacheArgs = cacheArgs; } /** @@ -170,4 +178,11 @@ public VisorTxTaskArg transactionArguments() { public boolean force() { return force; } + + /** + * @return Arguments for --cache subcommand. + */ + public CacheArguments cacheArgs() { + return cacheArgs; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/Command.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/Command.java index c8c7db5cafad5..52098d629fb0d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/Command.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/Command.java @@ -34,7 +34,10 @@ public enum Command { BASELINE("--baseline"), /** */ - TX("--tx"); + TX("--tx"), + + /** */ + CACHE("--cache"); /** */ private final String text; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java index 08488eacd4192..ff05684bb47a8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java @@ -19,15 +19,19 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Scanner; +import java.util.Set; import java.util.UUID; import java.util.logging.Logger; import java.util.regex.Pattern; import java.util.regex.PatternSyntaxException; +import java.util.stream.Collectors; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.client.GridClient; import org.apache.ignite.internal.client.GridClientAuthenticationException; @@ -42,6 +46,12 @@ import org.apache.ignite.internal.client.GridClientNode; import org.apache.ignite.internal.client.GridServerUnreachableException; import org.apache.ignite.internal.client.impl.connection.GridClientConnectionResetException; +import org.apache.ignite.internal.commandline.cache.CacheArguments; +import org.apache.ignite.internal.commandline.cache.CacheCommand; +import org.apache.ignite.internal.processors.cache.verify.CacheInfo; +import org.apache.ignite.internal.processors.cache.verify.ContentionInfo; +import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecord; +import org.apache.ignite.internal.processors.cache.verify.PartitionKey; 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; @@ -58,6 +68,20 @@ import org.apache.ignite.internal.visor.tx.VisorTxTask; import org.apache.ignite.internal.visor.tx.VisorTxTaskArg; import org.apache.ignite.internal.visor.tx.VisorTxTaskResult; +import org.apache.ignite.internal.visor.verify.IndexValidationIssue; +import org.apache.ignite.internal.visor.verify.ValidateIndexesPartitionResult; +import org.apache.ignite.internal.visor.verify.VisorContentionTask; +import org.apache.ignite.internal.visor.verify.VisorContentionTaskArg; +import org.apache.ignite.internal.visor.verify.VisorContentionTaskResult; +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.VisorValidateIndexesJobResult; +import org.apache.ignite.internal.visor.verify.VisorValidateIndexesTaskArg; +import org.apache.ignite.internal.visor.verify.VisorValidateIndexesTaskResult; +import org.apache.ignite.internal.visor.verify.VisorViewCacheTask; +import org.apache.ignite.internal.visor.verify.VisorViewCacheTaskArg; +import org.apache.ignite.internal.visor.verify.VisorViewCacheTaskResult; import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.plugin.security.SecurityCredentials; import org.apache.ignite.plugin.security.SecurityCredentialsBasicProvider; @@ -66,6 +90,7 @@ import static org.apache.ignite.internal.IgniteVersionUtils.COPYRIGHT; import static org.apache.ignite.internal.commandline.Command.ACTIVATE; import static org.apache.ignite.internal.commandline.Command.BASELINE; +import static org.apache.ignite.internal.commandline.Command.CACHE; import static org.apache.ignite.internal.commandline.Command.DEACTIVATE; import static org.apache.ignite.internal.commandline.Command.STATE; import static org.apache.ignite.internal.commandline.Command.TX; @@ -74,6 +99,8 @@ import static org.apache.ignite.internal.visor.baseline.VisorBaselineOperation.REMOVE; import static org.apache.ignite.internal.visor.baseline.VisorBaselineOperation.SET; import static org.apache.ignite.internal.visor.baseline.VisorBaselineOperation.VERSION; +import static org.apache.ignite.internal.visor.verify.VisorViewCacheCmd.GROUPS; +import static org.apache.ignite.internal.visor.verify.VisorViewCacheCmd.SEQ; /** * Class that execute several commands passed via command line. @@ -103,6 +130,23 @@ public class CommandHandler { /** */ private static final String CMD_USER = "--user"; + /** Force option is used for auto confirmation. */ + private static final String CMD_FORCE = "--force"; + + /** List of optional auxiliary commands. */ + private static final Set AUX_COMMANDS = new HashSet<>(); + static { + AUX_COMMANDS.add(CMD_HELP); + AUX_COMMANDS.add(CMD_HOST); + AUX_COMMANDS.add(CMD_PORT); + AUX_COMMANDS.add(CMD_PASSWORD); + AUX_COMMANDS.add(CMD_USER); + AUX_COMMANDS.add(CMD_FORCE); + } + + /** Broadcast uuid. */ + private static final UUID BROADCAST_UUID = UUID.randomUUID(); + /** */ protected static final String CMD_PING_INTERVAL = "--ping-interval"; @@ -130,9 +174,6 @@ public class CommandHandler { /** */ private static final String DELIM = "--------------------------------------------------------------------------------"; - /** Force option is used for auto confirmation. */ - private static final String CMD_FORCE = "--force"; - /** */ public static final int EXIT_CODE_OK = 0; @@ -157,6 +198,9 @@ public class CommandHandler { /** */ private static final Scanner IN = new Scanner(System.in); + /** Validate indexes task name. */ + private static final String VALIDATE_INDEXES_TASK = "org.apache.ignite.internal.visor.verify.VisorValidateIndexesTask"; + /** */ private static final String TX_LIMIT = "limit"; @@ -194,7 +238,7 @@ public class CommandHandler { private String peekedArg; /** */ - private Object lastOperationResult; + private Object lastOperationRes; /** * Output specified string to console. @@ -279,16 +323,22 @@ private String confirmationPrompt(Arguments args) { switch (args.command()) { case DEACTIVATE: str = "Warning: the command will deactivate a cluster."; + break; case BASELINE: if (!BASELINE_COLLECT.equals(args.baselineAction())) str = "Warning: the command will perform changes in baseline."; + break; case TX: if (args.transactionArguments().getOperation() == VisorTxOperation.KILL) str = "Warning: the command will kill some transactions."; + + break; + + default: break; } @@ -392,12 +442,52 @@ private Map executeTransactionsTask(GridClient client, * @throws GridClientException If failed to execute task. */ private R executeTask(GridClient client, Class taskCls, Object taskArgs) throws GridClientException { + return executeTaskByNameOnNode(client, taskCls.getName(), taskArgs, null); + } + + /** + * @param client Client + * @param taskClsName Task class name. + * @param taskArgs Task args. + * @param nodeId Node ID to execute task at (if null, random node will be chosen by balancer). + * @return Task result. + * @throws GridClientException If failed to execute task. + */ + private R executeTaskByNameOnNode(GridClient client, String taskClsName, Object taskArgs, UUID nodeId + ) throws GridClientException { GridClientCompute compute = client.compute(); - GridClientNode node = getBalancedNode(compute); + if (nodeId == BROADCAST_UUID) { + Collection nodes = compute.nodes(GridClientNode::connectable); + + if (F.isEmpty(nodes)) + throw new GridClientDisconnectedException("Connectable nodes not found", null); + + List nodeIds = nodes.stream() + .map(GridClientNode::nodeId) + .collect(Collectors.toList()); + + return client.compute().execute(taskClsName, new VisorTaskArgument<>(nodeIds, taskArgs, false)); + } + + GridClientNode node = null; + + if (nodeId == null) + node = getBalancedNode(compute); + else { + for (GridClientNode n : compute.nodes()) { + if (n.connectable() && nodeId.equals(n.nodeId())) { + node = n; - return compute.execute(taskCls.getName(), - new VisorTaskArgument<>(node.nodeId(), taskArgs, false)); + break; + } + } + + if (node == null) + throw new IllegalArgumentException("Node with id=" + nodeId + " not found"); + } + + return compute.projection(node).execute(taskClsName, new VisorTaskArgument<>(node.nodeId(), taskArgs, false)); } /** @@ -405,11 +495,7 @@ private R executeTask(GridClient client, Class taskCls, Object taskArgs) * @return balanced node */ private GridClientNode getBalancedNode(GridClientCompute compute) throws GridClientException { - List nodes = new ArrayList<>(); - - for (GridClientNode node : compute.nodes()) - if (node.connectable()) - nodes.add(node); + Collection nodes = compute.nodes(GridClientNode::connectable); if (F.isEmpty(nodes)) throw new GridClientDisconnectedException("Connectable node not found", null); @@ -417,6 +503,175 @@ private GridClientNode getBalancedNode(GridClientCompute compute) throws GridCli return compute.balancer().balancedNode(nodes); } + /** + * Executes --cache subcommand. + * + * @param client Client. + * @param cacheArgs Cache args. + */ + private void cache(GridClient client, CacheArguments cacheArgs) throws Throwable { + switch (cacheArgs.command()) { + case HELP: + printCacheHelp(); + + break; + + case IDLE_VERIFY: + cacheIdleVerify(client, cacheArgs); + + break; + + case VALIDATE_INDEXES: + cacheValidateIndexes(client, cacheArgs); + + break; + + case CONTENTION: + cacheContention(client, cacheArgs); + + break; + + default: + cacheView(client, cacheArgs); + + break; + } + } + + /** + * + */ + private void printCacheHelp() { + log("--cache subcommand allows to do the following operations:"); + + usage(" Show information about caches, groups or sequences that match a regex:", CACHE, " list regexPattern [groups|seq] [nodeId]"); + usage(" Show hot keys that are point of contention for multiple transactions:", CACHE, " contention minQueueSize [nodeId] [maxPrint]"); + usage(" Verify partition counters and hashes between primary and backups on idle cluster:", CACHE, " idle_verify [cache1,...,cacheN]"); + usage(" Validate custom indexes on idle cluster:", CACHE, " validate_indexes [cache1,...,cacheN] [nodeId]"); + + log(" If [nodeId] is not specified, cont and validate_indexes commands will be broadcasted to all server nodes."); + log(" Another commands where [nodeId] is optional will run on a random server node."); + nl(); + } + + /** + * @param client Client. + * @param cacheArgs Cache args. + */ + private void cacheContention(GridClient client, CacheArguments cacheArgs) throws GridClientException { + VisorContentionTaskArg taskArg = new VisorContentionTaskArg( + cacheArgs.minQueueSize(), cacheArgs.maxPrint()); + + UUID nodeId = cacheArgs.nodeId() == null ? BROADCAST_UUID : cacheArgs.nodeId(); + + VisorContentionTaskResult res = executeTaskByNameOnNode( + client, VisorContentionTask.class.getName(), taskArg, nodeId); + + if (!F.isEmpty(res.exceptions())) { + log("Contention check failed on nodes:"); + + for (Map.Entry e : res.exceptions().entrySet()) { + log("Node ID = " + e.getKey()); + + log("Exception message:"); + log(e.getValue().getMessage()); + nl(); + } + } + + for (ContentionInfo info : res.getInfos()) + info.print(); + } + + /** + * @param client Client. + * @param cacheArgs Cache args. + */ + private void cacheValidateIndexes(GridClient client, CacheArguments cacheArgs) throws GridClientException { + VisorValidateIndexesTaskArg taskArg = new VisorValidateIndexesTaskArg(cacheArgs.caches()); + + UUID nodeId = cacheArgs.nodeId() == null ? BROADCAST_UUID : cacheArgs.nodeId(); + + VisorValidateIndexesTaskResult taskRes = executeTaskByNameOnNode( + client, VALIDATE_INDEXES_TASK, taskArg, nodeId); + + if (!F.isEmpty(taskRes.exceptions())) { + log("Index validation failed on nodes:"); + + for (Map.Entry e : taskRes.exceptions().entrySet()) { + log("Node ID = " + e.getKey()); + + log("Exception message:"); + log(e.getValue().getMessage()); + nl(); + } + } + + boolean errors = false; + + for (Map.Entry nodeEntry : taskRes.results().entrySet()) { + Map map = nodeEntry.getValue().response(); + + for (Map.Entry e : map.entrySet()) { + ValidateIndexesPartitionResult res = e.getValue(); + + if (!res.issues().isEmpty()) { + errors = true; + + log(e.getKey().toString() + " " + e.getValue().toString()); + + for (IndexValidationIssue is : res.issues()) + log(is.toString()); + } + } + } + + if (!errors) + log("validate_indexes has finished, no issues found."); + else + log("validate_indexes has finished with errors (listed above)."); + } + + /** + * @param client Client. + * @param cacheArgs Cache args. + */ + private void cacheView(GridClient client, CacheArguments cacheArgs) throws GridClientException { + VisorViewCacheTaskArg taskArg = new VisorViewCacheTaskArg(cacheArgs.regex(), cacheArgs.cacheCommand()); + + VisorViewCacheTaskResult res = executeTaskByNameOnNode( + client, VisorViewCacheTask.class.getName(), taskArg, cacheArgs.nodeId()); + + for (CacheInfo info : res.cacheInfos()) + info.print(cacheArgs.cacheCommand()); + } + + /** + * @param client Client. + * @param cacheArgs Cache args. + */ + private void cacheIdleVerify(GridClient client, CacheArguments cacheArgs) throws GridClientException { + VisorIdleVerifyTaskResult res = executeTask( + client, VisorIdleVerifyTask.class, new VisorIdleVerifyTaskArg(cacheArgs.caches())); + + Map> conflicts = res.getConflicts(); + + if (conflicts.isEmpty()) { + log("idle_verify check has finished, no conflicts have been found."); + nl(); + } + else { + log ("idle_verify check has finished, found " + conflicts.size() + " conflict partitions."); + nl(); + + for (Map.Entry> entry : conflicts.entrySet()) { + log("Conflict partition: " + entry.getKey()); + + log("Partition instances: " + entry.getValue()); + } + } + } + /** * Change baseline. * @@ -508,7 +763,7 @@ private void baselinePrint0(VisorBaselineTaskResult res) { Map baseline = res.getBaseline(); - Map servers = res.getServers(); + Map srvs = res.getServers(); if (F.isEmpty(baseline)) log("Baseline nodes not found."); @@ -517,7 +772,7 @@ private void baselinePrint0(VisorBaselineTaskResult res) { for(VisorBaselineNode node : baseline.values()) { log(" ConsistentID=" + node.getConsistentId() + ", STATE=" + - (servers.containsKey(node.getConsistentId()) ? "ONLINE" : "OFFLINE")); + (srvs.containsKey(node.getConsistentId()) ? "ONLINE" : "OFFLINE")); } log(DELIM); @@ -527,7 +782,7 @@ private void baselinePrint0(VisorBaselineTaskResult res) { List others = new ArrayList<>(); - for (VisorBaselineNode node : servers.values()) { + for (VisorBaselineNode node : srvs.values()) { if (!baseline.containsKey(node.getConsistentId())) others.add(node); } @@ -645,7 +900,7 @@ private void transactions(GridClient client, VisorTxTaskArg arg) throws GridClie try { Map res = executeTask(client, VisorTxTask.class, arg); - lastOperationResult = res; + lastOperationRes = res; if (res.isEmpty()) log("Nothing found."); @@ -778,6 +1033,8 @@ Arguments parseAndValidate(List rawArgs) { boolean force = false; + CacheArguments cacheArgs = null; + List commands = new ArrayList<>(); initArgIterator(rawArgs); @@ -824,6 +1081,13 @@ Arguments parseAndValidate(List rawArgs) { break; + case CACHE: + commands.add(CACHE); + + cacheArgs = parseAndValidateCacheArgs(); + + break; + default: throw new IllegalArgumentException("Unexpected command: " + str); } @@ -862,14 +1126,17 @@ Arguments parseAndValidate(List rawArgs) { case CMD_USER: user = nextArg("Expected user name"); + break; case CMD_PASSWORD: pwd = nextArg("Expected password"); + break; case CMD_FORCE: force = true; + break; default: @@ -895,7 +1162,118 @@ Arguments parseAndValidate(List rawArgs) { throw new IllegalArgumentException("Both user and password should be specified"); return new Arguments(cmd, host, port, user, pwd, baselineAct, baselineArgs, - pingTimeout, pingInterval, txArgs, force); + pingTimeout, pingInterval, txArgs, force, cacheArgs); + } + + /** + * Parses and validates cache arguments. + * + * @return --cache subcommand arguments in case validation is successful. + */ + private CacheArguments parseAndValidateCacheArgs() { + if (!hasNextCacheArg()) { + throw new IllegalArgumentException("Arguments are expected for --cache subcommand, " + + "run --cache help for more info."); + } + + CacheArguments cacheArgs = new CacheArguments(); + + String str = nextArg("").toLowerCase(); + + CacheCommand cmd = CacheCommand.of(str); + + if (cmd == null) + cmd = CacheCommand.HELP; + + cacheArgs.command(cmd); + + switch (cmd) { + case HELP: + break; + + case IDLE_VERIFY: + parseCacheNamesIfPresent(cacheArgs); + + break; + + case CONTENTION: + cacheArgs.minQueueSize(Integer.parseInt(nextArg("Min queue size expected"))); + + if (hasNextCacheArg()) + cacheArgs.nodeId(UUID.fromString(nextArg(""))); + + if (hasNextCacheArg()) + cacheArgs.maxPrint(Integer.parseInt(nextArg(""))); + else + cacheArgs.maxPrint(10); + + break; + + case VALIDATE_INDEXES: + parseCacheNamesIfPresent(cacheArgs); + + if (hasNextCacheArg()) + cacheArgs.nodeId(UUID.fromString(nextArg(""))); + + break; + + default: + cacheArgs.regex(nextArg("Regex is expected")); + + if (hasNextCacheArg()) { + String tmp = nextArg(""); + + switch (tmp) { + case "groups": + cacheArgs.cacheCommand(GROUPS); + + break; + + case "seq": + cacheArgs.cacheCommand(SEQ); + + break; + + default: + cacheArgs.nodeId(UUID.fromString(tmp)); + } + } + + break; + } + + if (hasNextCacheArg()) + throw new IllegalArgumentException("Unexpected argument of --cache subcommand: " + peekNextArg()); + + return cacheArgs; + } + + /** + * @return true if there's next argument for --cache subcommand. + */ + private boolean hasNextCacheArg() { + return hasNextArg() && Command.of(peekNextArg()) == null && !AUX_COMMANDS.contains(peekNextArg()); + } + + /** + * @param cacheArgs Cache args. + */ + private void parseCacheNamesIfPresent(CacheArguments cacheArgs) { + if (hasNextCacheArg()) { + String cacheNames = nextArg(""); + + String[] cacheNamesArr = cacheNames.split(","); + Set cacheNamesSet = new HashSet<>(); + + for (String cacheName : cacheNamesArr) { + if (F.isEmpty(cacheName)) + throw new IllegalArgumentException("Non-empty cache names expected."); + + cacheNamesSet.add(cacheName.trim()); + } + + cacheArgs.caches(cacheNamesSet); + } } /** @@ -1002,7 +1380,7 @@ private VisorTxTaskArg parseTransactionArguments() { try { Pattern.compile(lbRegex); } - catch (PatternSyntaxException e) { + catch (PatternSyntaxException ignored) { throw new IllegalArgumentException("Illegal regex syntax"); } @@ -1046,7 +1424,7 @@ private long nextLongArg(String lb) { return val; } - catch (NumberFormatException e) { + catch (NumberFormatException ignored) { throw new IllegalArgumentException("Invalid value for " + lb + ": " + str); } } @@ -1079,6 +1457,10 @@ public int execute(List rawArgs) { "[minSize SIZE] [label PATTERN_REGEX] [servers|clients] " + "[nodes consistentId1[,consistentId2,....,consistentIdN] [limit NUMBER] [order DURATION|SIZE] [kill] [--force]"); + log("The utility has --cache subcommand to view and control state of caches in cluster."); + log(" More info: control.sh --cache help"); + nl(); + log("By default commands affecting the cluster require interactive confirmation. "); log(" --force option can be used to execute commands without prompting for confirmation."); nl(); @@ -1122,26 +1504,35 @@ public int execute(List rawArgs) { } try (GridClient client = GridClientFactory.start(cfg)) { - switch (args.command()) { case ACTIVATE: activate(client); + break; case DEACTIVATE: deactivate(client); + break; case STATE: state(client); + break; case BASELINE: baseline(client, args.baselineAction(), args.baselineArguments()); + break; case TX: transactions(client, args.transactionArguments()); + + break; + + case CACHE: + cache(client, args.cacheArgs()); + break; } } @@ -1177,7 +1568,7 @@ public static void main(String[] args) { */ @SuppressWarnings("unchecked") public T getLastOperationResult() { - return (T)lastOperationResult; + return (T)lastOperationRes; } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheArguments.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheArguments.java new file mode 100644 index 0000000000000..6f315efe6a732 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheArguments.java @@ -0,0 +1,163 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ +package org.apache.ignite.internal.commandline.cache; + +import java.util.Set; +import java.util.UUID; +import org.apache.ignite.internal.visor.verify.VisorViewCacheCmd; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public class CacheArguments { + /** Command. */ + private CacheCommand cmd; + + /** Caches. */ + private Set caches; + + /** Partition id. */ + private int partId; + + /** Regex. */ + private String regex; + + /** Node id. */ + private UUID nodeId; + + /** Min queue size. */ + private int minQueueSize; + + /** Max print. */ + private int maxPrint; + + /** Cache view command. */ + private @Nullable VisorViewCacheCmd cacheCmd; + + /** + * @return Command. + */ + public CacheCommand command() { + return cmd; + } + + /** + * @return Cache view command. + */ + @Nullable public VisorViewCacheCmd cacheCommand() { + return cacheCmd; + } + + /** + * @param cmd Cache view command. + */ + public void cacheCommand(VisorViewCacheCmd cmd) { + this.cacheCmd = cmd; + } + + /** + * @param cmd New command. + */ + public void command(CacheCommand cmd) { + this.cmd = cmd; + } + + /** + * @return Caches. + */ + public Set caches() { + return caches; + } + + /** + * @param caches New caches. + */ + public void caches(Set caches) { + this.caches = caches; + } + + /** + * @return Partition id. + */ + public int partitionId() { + return partId; + } + + /** + * @param partId New partition id. + */ + public void partitionId(int partId) { + this.partId = partId; + } + + /** + * @return Regex. + */ + public String regex() { + return regex; + } + + /** + * @param regex New regex. + */ + public void regex(String regex) { + this.regex = regex; + } + + /** + * @return Node id. + */ + public UUID nodeId() { + return nodeId; + } + + /** + * @param nodeId New node id. + */ + public void nodeId(UUID nodeId) { + this.nodeId = nodeId; + } + + /** + * @return Min queue size. + */ + public int minQueueSize() { + return minQueueSize; + } + + /** + * @param minQueueSize New min queue size. + */ + public void minQueueSize(int minQueueSize) { + this.minQueueSize = minQueueSize; + } + + /** + * @return Max print. + */ + public int maxPrint() { + return maxPrint; + } + + /** + * @param maxPrint New max print. + */ + public void maxPrint(int maxPrint) { + this.maxPrint = maxPrint; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommand.java new file mode 100644 index 0000000000000..6aec6d7e9cfa1 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheCommand.java @@ -0,0 +1,93 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.ignite.internal.commandline.cache; + +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public enum CacheCommand { + /** + * Prints out help for the cache command. + */ + HELP("help"), + + /** + * Checks consistency of primary and backup partitions assuming no concurrent updates are happening in the cluster. + */ + IDLE_VERIFY("idle_verify"), + + /** + * Prints info regarding caches, groups or sequences. + */ + LIST("list"), + + /** + * Validates indexes attempting to read each indexed entry. + */ + VALIDATE_INDEXES("validate_indexes"), + + /** + * Prints info about contended keys (the keys concurrently locked from multiple transactions). + */ + CONTENTION("contention"); + + /** Enumerated values. */ + private static final CacheCommand[] VALS = values(); + + /** Name. */ + private final String name; + + /** + * @param name Name. + */ + CacheCommand(String name) { + this.name = name; + } + + /** + * @param text Command text. + * @return Command for the text. + */ + public static CacheCommand of(String text) { + for (CacheCommand cmd : CacheCommand.values()) { + if (cmd.text().equalsIgnoreCase(text)) + return cmd; + } + + return null; + } + + /** + * @return Name. + */ + public String text() { + return name; + } + + /** + * Efficiently gets enumerated value from its ordinal. + * + * @param ord Ordinal value. + * @return Enumerated value or {@code null} if ordinal out of range. + */ + @Nullable public static CacheCommand fromOrdinal(int ord) { + return ord >= 0 && ord < VALS.length ? VALS[ord] : null; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/CacheInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/CacheInfo.java new file mode 100644 index 0000000000000..9a090a042e961 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/CacheInfo.java @@ -0,0 +1,322 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.verify; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; +import org.apache.ignite.internal.visor.verify.VisorViewCacheCmd; + +/** + * Cache info DTO. + */ +public class CacheInfo extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Sequence name. */ + private String seqName; + + /** Sequence value. */ + private long seqVal; + + /** Cache name. */ + private String cacheName; + + /** Cache id. */ + private int cacheId; + + /** Group name. */ + private String grpName; + + /** Group id. */ + private int grpId; + + /** Caches count. */ + private int cachesCnt; + + /** Partitions. */ + private int partitions; + + /** Mapped. */ + private int mapped; + + /** Topology version. */ + public AffinityTopologyVersion topVer; + + /** Mode. */ + private CacheMode mode; + + /** Backups count. */ + private int backupsCnt; + + /** Affinity class name. */ + private String affinityClsName; + + /** */ + public String getSeqName() { + return seqName; + } + + /** + * @param seqName Sequence name. + */ + public void setSeqName(String seqName) { + this.seqName = seqName; + } + + /** + * + */ + public String getCacheName() { + return cacheName; + } + + /** + * @param cacheName Cache name. + */ + public void setCacheName(String cacheName) { + this.cacheName = cacheName; + } + + /** + * + */ + public int getCacheId() { + return cacheId; + } + + /** + * @param cacheId Cache id. + */ + public void setCacheId(int cacheId) { + this.cacheId = cacheId; + } + + /** + * + */ + public String getGrpName() { + return grpName; + } + + /** + * @param grpName Group name. + */ + public void setGrpName(String grpName) { + this.grpName = grpName; + } + + /** + * + */ + public int getGrpId() { + return grpId; + } + + /** + * @param grpId Group id. + */ + public void setGrpId(int grpId) { + this.grpId = grpId; + } + + /** + * + */ + public int getCachesCnt() { + return cachesCnt; + } + + /** + * @param cachesCnt Caches count. + */ + public void setCachesCnt(int cachesCnt) { + this.cachesCnt = cachesCnt; + } + + /** + * + */ + public int getPartitions() { + return partitions; + } + + /** + * @param partitions Partitions. + */ + public void setPartitions(int partitions) { + this.partitions = partitions; + } + + /** + * + */ + public int getMapped() { + return mapped; + } + + /** + * @param mapped Mapped. + */ + public void setMapped(int mapped) { + this.mapped = mapped; + } + + /** + * + */ + public AffinityTopologyVersion getTopologyVersion() { + return topVer; + } + + /** + * @param topologyVersion Topology version. + */ + public void setTopologyVersion(AffinityTopologyVersion topologyVersion) { + this.topVer = topologyVersion; + } + + /** + * @param seqVal Sequence value. + */ + public void setSeqVal(long seqVal) { + this.seqVal = seqVal; + } + + /** + * + */ + public long getSeqVal() { + return seqVal; + } + + /** + * + */ + public CacheMode getMode() { + return mode; + } + + /** + * @param mode Mode. + */ + public void setMode(CacheMode mode) { + this.mode = mode; + } + + /** + * + */ + public int getBackupsCnt() { + return backupsCnt; + } + + /** + * @param backupsCnt Backups count. + */ + public void setBackupsCnt(int backupsCnt) { + this.backupsCnt = backupsCnt; + } + + /** + * + */ + public String getAffinityClsName() { + return affinityClsName; + } + + /** + * @param affinityClsName Affinity class name. + */ + public void setAffinityClsName(String affinityClsName) { + this.affinityClsName = affinityClsName; + } + + /** + * @param cmd Command. + */ + public void print(VisorViewCacheCmd cmd) { + if (cmd == null) + cmd = VisorViewCacheCmd.CACHES; + + switch (cmd) { + case SEQ: + System.out.println("[seqName=" + getSeqName() + ", curVal=" + seqVal + ']'); + + break; + + case GROUPS: + System.out.println("[grpName=" + getGrpName() + ", grpId=" + getGrpId() + ", cachesCnt=" + getCachesCnt() + + ", prim=" + getPartitions() + ", mapped=" + getMapped() + ", mode=" + getMode() + + ", backups=" + getBackupsCnt() + ", affCls=" + getAffinityClsName() + ']'); + + break; + + default: + System.out.println("[cacheName=" + getCacheName() + ", cacheId=" + getCacheId() + + ", grpName=" + getGrpName() + ", grpId=" + getGrpId() + ", prim=" + getPartitions() + + ", mapped=" + getMapped() + ", mode=" + getMode() + + ", backups=" + getBackupsCnt() + ", affCls=" + getAffinityClsName() + ']'); + } + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + U.writeString(out, seqName); + out.writeLong(seqVal); + U.writeString(out, cacheName); + out.writeInt(cacheId); + U.writeString(out, grpName); + out.writeInt(grpId); + out.writeInt(partitions); + out.writeInt(mapped); + out.writeObject(topVer); + U.writeEnum(out, mode); + out.writeInt(backupsCnt); + U.writeString(out, affinityClsName); + out.writeInt(cachesCnt); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + seqName = U.readString(in); + seqVal = in.readLong(); + cacheName = U.readString(in); + cacheId = in.readInt(); + grpName = U.readString(in); + grpId = in.readInt(); + partitions = in.readInt(); + mapped = in.readInt(); + topVer = (AffinityTopologyVersion)in.readObject(); + mode = CacheMode.fromOrdinal(in.readByte()); + backupsCnt = in.readInt(); + affinityClsName = U.readString(in); + cachesCnt = in.readInt(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(CacheInfo.class, this); + } +} \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/ContentionClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/ContentionClosure.java new file mode 100644 index 0000000000000..e97378e9870d6 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/ContentionClosure.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.verify; + +import java.util.ArrayList; +import java.util.Collection; +import org.apache.ignite.internal.IgniteEx; +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.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.IgniteTxLocalState; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxState; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxStateImpl; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.resources.IgniteInstanceResource; + +/** + */ +public class ContentionClosure implements IgniteCallable { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Ignite. */ + @IgniteInstanceResource + protected transient IgniteEx ignite; + + /** */ + private int minQueueSize; + + /** */ + private int maxPrint; + + /** + * @param minQueueSize Min candidate queue size to account. + * @param maxPrint Max entries to print. + */ + public ContentionClosure(int minQueueSize, int maxPrint) { + this.minQueueSize = minQueueSize; + this.maxPrint = maxPrint; + } + + /** {@inheritDoc} */ + @Override public ContentionInfo call() throws Exception { + final IgniteTxManager tm = ignite.context().cache().context().tm(); + + final Collection activeTxs = tm.activeTransactions(); + + ContentionInfo ci = new ContentionInfo(); + + ci.setNode(ignite.localNode()); + ci.setEntries(new ArrayList<>()); + + for (IgniteInternalTx tx : activeTxs) { + if (ci.getEntries().size() == maxPrint) + break; + + // Show only primary txs. + if (tx.local()) { + IgniteTxLocalAdapter tx0 = (IgniteTxLocalAdapter)tx; + + final IgniteTxLocalState state0 = tx0.txState(); + + if (!(state0 instanceof IgniteTxStateImpl)) + continue; + + final IgniteTxStateImpl state = (IgniteTxStateImpl)state0; + + final Collection entries = state.allEntriesCopy(); + + IgniteTxEntry bad = null; + + int qSize = 0; + + for (IgniteTxEntry entry : entries) { + Collection locs; + + GridCacheEntryEx cached = entry.cached(); + + while(true) { + try { + locs = cached.localCandidates(); + + break; + } + catch (GridCacheEntryRemovedException ignored) { + cached = entry.context().cache().entryEx(entry.key()); + } + } + + if (locs != null) + qSize += locs.size(); + + final Collection rmts = cached.remoteMvccSnapshot(); + + if (rmts != null) + qSize += rmts.size(); + + if (qSize >= minQueueSize) { + bad = entry; + + break; + } + else + qSize = 0; + } + + if (bad != null) { + StringBuilder b = new StringBuilder(); + + b.append("TxEntry [cacheId=").append(bad.cacheId()). + append(", key=").append(bad.key()). + append(", queue=").append(qSize). + append(", op=").append(bad.op()). + append(", val=").append(bad.value()). + append(", tx=").append(CU.txString(tx)). + append(", other=["); + + final IgniteTxState st = tx.txState(); + + if (st instanceof IgniteTxStateImpl) { + IgniteTxStateImpl st0 = (IgniteTxStateImpl)st; + + final Collection cp = st0.allEntriesCopy(); + + for (IgniteTxEntry entry : cp) { + if (entry == bad) + continue; + + b.append(entry.toString()).append('\n'); + } + } + + b.append("]]"); + + ci.getEntries().add(b.toString()); + } + } + } + + return ci; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/ContentionInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/ContentionInfo.java new file mode 100644 index 0000000000000..c7bfbebef4316 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/ContentionInfo.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.internal.processors.cache.verify; + +import java.io.Serializable; +import java.util.List; +import org.apache.ignite.cluster.ClusterNode; + +/** + */ +public class ContentionInfo implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private ClusterNode node; + + /** */ + private List entries; + + /** + * @return Node. + */ + public ClusterNode getNode() { + return node; + } + + /** + * @param node Node. + */ + public void setNode(ClusterNode node) { + this.node = node; + } + + /** + * @return Entries. + */ + public List getEntries() { + return entries; + } + + /** + * @param entries Entries. + */ + public void setEntries(List entries) { + this.entries = entries; + } + + /** */ + public void print() { + System.out.println("[node=" + node + ']'); + + for (String entry : entries) + System.out.println(" " + entry); + } +} + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/ViewCacheClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/ViewCacheClosure.java new file mode 100644 index 0000000000000..1f363f3643db7 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/ViewCacheClosure.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.verify; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; +import javax.cache.Cache; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cluster.ClusterGroup; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor; +import org.apache.ignite.internal.processors.cache.IgniteInternalCache; +import org.apache.ignite.internal.processors.datastructures.AtomicDataStructureValue; +import org.apache.ignite.internal.processors.datastructures.DataStructureType; +import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor; +import org.apache.ignite.internal.processors.datastructures.GridCacheAtomicSequenceValue; +import org.apache.ignite.internal.processors.datastructures.GridCacheInternalKey; +import org.apache.ignite.internal.visor.verify.VisorViewCacheCmd; +import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.resources.IgniteInstanceResource; + +/** + * View cache closure. + */ +public class ViewCacheClosure implements IgniteCallable> { + /** */ + private static final long serialVersionUID = 0L; + + /** Regex. */ + private String regex; + + /** {@code true} to skip cache destroying. */ + private VisorViewCacheCmd cmd; + + @IgniteInstanceResource + private Ignite ignite; + + /** + * @param regex Regex name for stopping caches. + * @param cmd Command. + */ + public ViewCacheClosure(String regex, VisorViewCacheCmd cmd) { + this.regex = regex; + this.cmd = cmd; + } + + /** {@inheritDoc} */ + @Override public List call() throws Exception { + Pattern compiled = Pattern.compile(regex); + + List cacheInfo = new ArrayList<>(); + + IgniteKernal k = (IgniteKernal)ignite; + + if (cmd == null) + cmd = VisorViewCacheCmd.CACHES; + + switch (cmd) { + case SEQ: + collectSequences(k.context(), compiled, cacheInfo); + + return cacheInfo; + + case GROUPS: + Collection contexts = k.context().cache().cacheGroups(); + + for (CacheGroupContext context : contexts) { + if (!compiled.matcher(context.cacheOrGroupName()).find()) + continue; + + CacheInfo ci = new CacheInfo(); + ci.setGrpName(context.cacheOrGroupName()); + ci.setGrpId(context.groupId()); + ci.setCachesCnt(context.caches().size()); + ci.setPartitions(context.config().getAffinity().partitions()); + ci.setBackupsCnt(context.config().getBackups()); + ci.setAffinityClsName(context.config().getAffinity().getClass().getSimpleName()); + ci.setMode(context.config().getCacheMode()); + ci.setMapped(mapped(context.caches().iterator().next().name())); + + cacheInfo.add(ci); + } + + return cacheInfo; + + default: + Map descMap = k.context().cache().cacheDescriptors(); + + for (Map.Entry entry : descMap.entrySet()) { + + DynamicCacheDescriptor desc = entry.getValue(); + + if (!compiled.matcher(desc.cacheName()).find()) + continue; + + CacheInfo ci = new CacheInfo(); + + ci.setCacheName(desc.cacheName()); + ci.setCacheId(desc.cacheId()); + ci.setGrpName(desc.groupDescriptor().groupName()); + ci.setGrpId(desc.groupDescriptor().groupId()); + ci.setPartitions(desc.cacheConfiguration().getAffinity().partitions()); + ci.setBackupsCnt(desc.cacheConfiguration().getBackups()); + ci.setAffinityClsName(desc.cacheConfiguration().getAffinity().getClass().getSimpleName()); + ci.setMode(desc.cacheConfiguration().getCacheMode()); + ci.setMapped(mapped(desc.cacheName())); + + cacheInfo.add(ci); + } + + return cacheInfo; + } + } + + /** + * @param cacheName Cache name. + */ + private int mapped(String cacheName) { + int mapped = 0; + + ClusterGroup srvs = ignite.cluster().forServers(); + + Collection nodes = srvs.forDataNodes(cacheName).nodes(); + + for (ClusterNode node : nodes) + mapped += ignite.affinity(cacheName).primaryPartitions(node).length; + + return mapped; + } + + /** + * @param ctx Context. + * @param compiled Compiled pattern. + * @param cacheInfo Cache info. + */ + private void collectSequences(GridKernalContext ctx, Pattern compiled, List cacheInfo) throws IgniteCheckedException { + String dsCacheName = DataStructuresProcessor.ATOMICS_CACHE_NAME + "@default-ds-group"; + + IgniteInternalCache cache0 = ctx.cache().cache(dsCacheName); + + final Iterator> iter = cache0.scanIterator(false, null); + + while (iter.hasNext()) { + Cache.Entry entry = iter.next(); + + final AtomicDataStructureValue val = entry.getValue(); + + if (val.type() == DataStructureType.ATOMIC_SEQ) { + final String name = entry.getKey().name(); + + if (compiled.matcher(name).find()) { + CacheInfo ci = new CacheInfo(); + ci.setSeqName(name); + ci.setSeqVal(((GridCacheAtomicSequenceValue)val).get()); + + cacheInfo.add(ci); + } + + } + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/IndexValidationIssue.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/IndexValidationIssue.java new file mode 100644 index 0000000000000..37f936088b439 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/IndexValidationIssue.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.internal.visor.verify; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * + */ +public class IndexValidationIssue extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Key. */ + private String key; + + /** Cache name. */ + private String cacheName; + + /** Index name. */ + private String idxName; + + /** T. */ + @GridToStringExclude + private Throwable t; + + /** + * + */ + public IndexValidationIssue() { + // Default constructor required for Externalizable. + } + + /** + * @param key Key. + * @param cacheName Cache name. + * @param idxName Index name. + * @param t T. + */ + public IndexValidationIssue(String key, String cacheName, String idxName, Throwable t) { + this.key = key; + this.cacheName = cacheName; + this.idxName = idxName; + this.t = t; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + U.writeString(out, key); + U.writeString(out, cacheName); + U.writeString(out, idxName); + out.writeObject(t); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + key = U.readString(in); + cacheName = U.readString(in); + idxName = U.readString(in); + t = (Throwable)in.readObject(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(IndexValidationIssue.class, this) + ", " + t.getClass() + ": " + t.getMessage(); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesPartitionResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesPartitionResult.java new file mode 100644 index 0000000000000..18899607110fc --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesPartitionResult.java @@ -0,0 +1,145 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.ignite.internal.visor.verify; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.ArrayList; +import java.util.List; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * + */ +public class ValidateIndexesPartitionResult extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Update counter. */ + private long updateCntr; + + /** Size. */ + private long size; + + /** Is primary. */ + private boolean isPrimary; + + /** Consistent id. */ + @GridToStringInclude + private Object consistentId; + + /** Issues. */ + @GridToStringExclude + private List issues = new ArrayList<>(10); + + /** + * + */ + public ValidateIndexesPartitionResult() { + // Empty constructor required for Externalizable. + } + + /** + * @param updateCntr Update counter. + * @param size Size. + * @param isPrimary Is primary. + * @param consistentId Consistent id. + */ + public ValidateIndexesPartitionResult(long updateCntr, long size, boolean isPrimary, Object consistentId) { + this.updateCntr = updateCntr; + this.size = size; + this.isPrimary = isPrimary; + this.consistentId = consistentId; + } + + /** + * + */ + public long updateCntr() { + return updateCntr; + } + + /** + * + */ + public long size() { + return size; + } + + /** + * + */ + public boolean primary() { + return isPrimary; + } + + /** + * + */ + public Object consistentId() { + return consistentId; + } + + /** + * + */ + public List issues() { + return issues; + } + + /** + * @param t Issue. + * @return True if there are already enough issues. + */ + public boolean reportIssue(IndexValidationIssue t) { + if (issues.size() >= 10) + return true; + + issues.add(t); + + return false; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + out.writeLong(updateCntr); + out.writeLong(size); + out.writeBoolean(isPrimary); + out.writeObject(consistentId); + U.writeCollection(out, issues); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + updateCntr = in.readLong(); + size = in.readLong(); + isPrimary = in.readBoolean(); + consistentId = in.readObject(); + issues = U.readList(in); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ValidateIndexesPartitionResult.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorContentionJobResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorContentionJobResult.java new file mode 100644 index 0000000000000..427224423c61a --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorContentionJobResult.java @@ -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. +*/ + +package org.apache.ignite.internal.visor.verify; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.List; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.processors.cache.verify.ContentionInfo; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * + */ +public class VisorContentionJobResult extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Info. */ + private ContentionInfo info; + + /** + * @param info Info. + */ + public VisorContentionJobResult(ContentionInfo info) { + this.info = info; + } + + /** + * For externalization only. + */ + public VisorContentionJobResult() { + } + + /** + * @return Contention info. + */ + public ContentionInfo info() { + return info; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + out.writeObject(info.getNode()); + U.writeCollection(out, info.getEntries()); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + Object node = in.readObject(); + List entries = U.readList(in); + + info = new ContentionInfo(); + info.setNode((ClusterNode)node); + info.setEntries(entries); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorContentionJobResult.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorContentionTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorContentionTask.java new file mode 100644 index 0000000000000..4e7e12c2ff102 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorContentionTask.java @@ -0,0 +1,100 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.ignite.internal.visor.verify; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.IgniteException; +import org.apache.ignite.compute.ComputeJobResult; +import org.apache.ignite.internal.processors.cache.verify.ContentionClosure; +import org.apache.ignite.internal.processors.cache.verify.ContentionInfo; +import org.apache.ignite.internal.processors.task.GridInternal; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.visor.VisorJob; +import org.apache.ignite.internal.visor.VisorMultiNodeTask; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +@GridInternal +public class VisorContentionTask extends VisorMultiNodeTask { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Nullable @Override protected VisorContentionTaskResult reduce0(List list) throws IgniteException { + Map exceptions = new HashMap<>(); + List infos = new ArrayList<>(); + + for (ComputeJobResult res : list) { + if (res.getException() != null) + exceptions.put(res.getNode().id(), res.getException()); + else + infos.add(res.getData()); + } + + return new VisorContentionTaskResult(infos, exceptions); + } + + /** {@inheritDoc} */ + @Override protected VisorJob job(VisorContentionTaskArg arg) { + return new VisorContentionJob(arg, debug); + } + + /** + * + */ + private static class VisorContentionJob extends VisorJob { + /** */ + private static final long serialVersionUID = 0L; + + /** + * @param arg Argument. + * @param debug Debug. + */ + protected VisorContentionJob(@Nullable VisorContentionTaskArg arg, boolean debug) { + super(arg, debug); + } + + /** {@inheritDoc} */ + @Override protected VisorContentionJobResult run(@Nullable VisorContentionTaskArg arg) throws IgniteException { + try { + ContentionClosure clo = new ContentionClosure(arg.minQueueSize(), arg.maxPrint()); + + ignite.context().resource().injectGeneric(clo); + + ContentionInfo info = clo.call(); + + return new VisorContentionJobResult(info); + } + catch (Exception e) { + throw new IgniteException(e); + } + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorContentionJob.class, this); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorContentionTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorContentionTaskArg.java new file mode 100644 index 0000000000000..ecfc9d8340145 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorContentionTaskArg.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.internal.visor.verify; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * + */ +public class VisorContentionTaskArg extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Min queue size. */ + private int minQueueSize; + + /** Max print size. */ + private int maxPrint; + + /** + * @param minQueueSize Min queue size. + * @param maxPrint Max print. + */ + public VisorContentionTaskArg(int minQueueSize, int maxPrint) { + this.minQueueSize = minQueueSize; + this.maxPrint = maxPrint; + } + + /** + * For externalization only. + */ + public VisorContentionTaskArg() { + } + + /** + * @return Min queue size. + */ + public int minQueueSize() { + return minQueueSize; + } + + /** + * @return Max print size. + */ + public int maxPrint() { + return maxPrint; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + out.writeInt(minQueueSize); + out.writeInt(maxPrint); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + minQueueSize = in.readInt(); + maxPrint = in.readInt(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorContentionTaskArg.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorContentionTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorContentionTaskResult.java new file mode 100644 index 0000000000000..5c452caf0e9ce --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorContentionTaskResult.java @@ -0,0 +1,100 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.ignite.internal.visor.verify; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.ignite.internal.processors.cache.verify.ContentionInfo; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * + */ +public class VisorContentionTaskResult extends VisorDataTransferObject { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Cluster infos. */ + private List clusterInfos; + + /** Exceptions. */ + private Map exceptions; + + /** + * @param clusterInfos Cluster infos. + * @param exceptions Exceptions. + */ + public VisorContentionTaskResult(List clusterInfos, + Map exceptions) { + this.clusterInfos = clusterInfos; + this.exceptions = exceptions; + } + + /** + * For externalization only. + */ + public VisorContentionTaskResult() { + } + + /** + * @return Cluster infos. + */ + public Collection jobResults() { + return clusterInfos; + } + + /** + * @return Collection of {@link ContentionInfo} collected during task execution. + */ + public Collection getInfos() { + return clusterInfos.stream().map(VisorContentionJobResult::info).collect(Collectors.toList()); + } + + /** + * @return Exceptions. + */ + public Map exceptions() { + return exceptions; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + U.writeCollection(out, clusterInfos); + U.writeMap(out, exceptions); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in + ) throws IOException, ClassNotFoundException { + clusterInfos = U.readList(in); + exceptions = U.readMap(in); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorContentionTaskResult.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleAnalyzeTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleAnalyzeTask.java new file mode 100644 index 0000000000000..2d86a428d38bd --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleAnalyzeTask.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.visor.verify; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.ignite.IgniteException; +import org.apache.ignite.compute.ComputeJobContext; +import org.apache.ignite.compute.ComputeTaskFuture; +import org.apache.ignite.internal.processors.cache.verify.CollectConflictPartitionKeysTask; +import org.apache.ignite.internal.processors.cache.verify.PartitionEntryHashRecord; +import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecord; +import org.apache.ignite.internal.processors.cache.verify.RetrieveConflictPartitionValuesTask; +import org.apache.ignite.internal.processors.task.GridInternal; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.visor.VisorJob; +import org.apache.ignite.internal.visor.VisorOneNodeTask; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.resources.JobContextResource; + +/** + * Task to find diverged keys of conflict partition. + */ +@GridInternal +public class VisorIdleAnalyzeTask extends VisorOneNodeTask { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override protected VisorJob job(VisorIdleAnalyzeTaskArg arg) { + return new VisorIdleVerifyJob(arg, debug); + } + + /** + * + */ + private static class VisorIdleVerifyJob extends VisorJob { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private ComputeTaskFuture>> conflictKeysFut; + + /** */ + private ComputeTaskFuture>> conflictValsFut; + + /** Auto-inject job context. */ + @JobContextResource + protected transient ComputeJobContext jobCtx; + + /** + * @param arg Argument. + * @param debug Debug. + */ + private VisorIdleVerifyJob(VisorIdleAnalyzeTaskArg arg, boolean debug) { + super(arg, debug); + } + + /** {@inheritDoc} */ + @Override protected VisorIdleAnalyzeTaskResult run(VisorIdleAnalyzeTaskArg arg) throws IgniteException { + if (conflictKeysFut == null) { + conflictKeysFut = ignite.compute() + .executeAsync(CollectConflictPartitionKeysTask.class, arg.getPartitionKey()); + + if (!conflictKeysFut.isDone()) { + jobCtx.holdcc(); + + conflictKeysFut.listen(new IgniteInClosure>>>() { + @Override public void apply(IgniteFuture>> f) { + jobCtx.callcc(); + } + }); + + return null; + } + } + + Map> conflictKeys = conflictKeysFut.get(); + + if (conflictKeys.isEmpty()) + return new VisorIdleAnalyzeTaskResult(Collections.emptyMap()); + + if (conflictValsFut == null) { + conflictValsFut = ignite.compute().executeAsync(RetrieveConflictPartitionValuesTask.class, conflictKeys); + + if (!conflictValsFut.isDone()) { + jobCtx.holdcc(); + + conflictKeysFut.listen(new IgniteInClosure>>>() { + @Override public void apply(IgniteFuture>> f) { + jobCtx.callcc(); + } + }); + + return null; + } + } + + return new VisorIdleAnalyzeTaskResult(conflictValsFut.get()); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorIdleVerifyJob.class, this); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleAnalyzeTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleAnalyzeTaskArg.java new file mode 100644 index 0000000000000..884f9610c42a7 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleAnalyzeTaskArg.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.internal.visor.verify; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import org.apache.ignite.internal.processors.cache.verify.PartitionKey; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * Arguments for task {@link VisorIdleAnalyzeTask} + */ +public class VisorIdleAnalyzeTaskArg extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Partition key. */ + private PartitionKey partKey; + + /** + * Default constructor. + */ + public VisorIdleAnalyzeTaskArg() { + // No-op. + } + + /** + * @param partKey Partition key. + */ + public VisorIdleAnalyzeTaskArg(PartitionKey partKey) { + this.partKey = partKey; + } + + /** + * @param grpId Group id. + * @param partId Partition id. + * @param grpName Group name. + */ + public VisorIdleAnalyzeTaskArg(int grpId, int partId, String grpName) { + this(new PartitionKey(grpId, partId, grpName)); + } + + /** + * @return Partition key. + */ + public PartitionKey getPartitionKey() { + return partKey; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + out.writeInt(partKey.groupId()); + out.writeInt(partKey.partitionId()); + U.writeString(out, partKey.groupName()); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException { + int grpId = in.readInt(); + int partId = in.readInt(); + String grpName = U.readString(in); + + partKey = new PartitionKey(grpId, partId, grpName); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorIdleAnalyzeTaskArg.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleAnalyzeTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleAnalyzeTaskResult.java new file mode 100644 index 0000000000000..e8c129033ef8d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleAnalyzeTaskResult.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.visor.verify; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.List; +import java.util.Map; +import org.apache.ignite.internal.processors.cache.verify.PartitionEntryHashRecord; +import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecord; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * Result for task {@link VisorIdleAnalyzeTask} + */ +public class VisorIdleAnalyzeTaskResult extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Results. */ + private Map> divergedEntries; + + /** + * Default constructor. + */ + public VisorIdleAnalyzeTaskResult() { + // No-op. + } + + /** + * @param divergedEntries Result. + */ + public VisorIdleAnalyzeTaskResult(Map> divergedEntries) { + this.divergedEntries = divergedEntries; + } + + /** + * @return Results. + */ + public Map> getDivergedEntries() { + return divergedEntries; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + U.writeMap(out, divergedEntries); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + divergedEntries = U.readMap(in); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorIdleAnalyzeTaskResult.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTask.java new file mode 100644 index 0000000000000..05f2621310366 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTask.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.visor.verify; + +import java.util.List; +import java.util.Map; +import org.apache.ignite.IgniteException; +import org.apache.ignite.compute.ComputeJobContext; +import org.apache.ignite.compute.ComputeTaskFuture; +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.processors.task.GridInternal; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.visor.VisorJob; +import org.apache.ignite.internal.visor.VisorOneNodeTask; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.resources.JobContextResource; + +/** + * Task to verify checksums of backup partitions. + */ +@GridInternal +public class VisorIdleVerifyTask extends VisorOneNodeTask { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override protected VisorJob job(VisorIdleVerifyTaskArg arg) { + return new VisorIdleVerifyJob(arg, debug); + } + + /** + * + */ + private static class VisorIdleVerifyJob extends VisorJob { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private ComputeTaskFuture>> fut; + + /** Auto-inject job context. */ + @JobContextResource + protected transient ComputeJobContext jobCtx; + + /** + * @param arg Argument. + * @param debug Debug. + */ + private VisorIdleVerifyJob(VisorIdleVerifyTaskArg arg, boolean debug) { + super(arg, debug); + } + + /** {@inheritDoc} */ + @Override protected VisorIdleVerifyTaskResult run(VisorIdleVerifyTaskArg arg) throws IgniteException { + if (fut == null) { + fut = ignite.compute().executeAsync(VerifyBackupPartitionsTask.class, arg.getCaches()); + + if (!fut.isDone()) { + jobCtx.holdcc(); + + fut.listen(new IgniteInClosure>>>() { + @Override public void apply(IgniteFuture>> f) { + jobCtx.callcc(); + } + }); + + return null; + } + } + + return new VisorIdleVerifyTaskResult(fut.get()); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorIdleVerifyJob.class, this); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskArg.java new file mode 100644 index 0000000000000..c82af5878ebb6 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskArg.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.visor.verify; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.Set; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * Arguments for task {@link VisorIdleVerifyTask} + */ +public class VisorIdleVerifyTaskArg extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Caches. */ + private Set caches; + + /** + * Default constructor. + */ + public VisorIdleVerifyTaskArg() { + // No-op. + } + + /** + * @param caches Caches. + */ + public VisorIdleVerifyTaskArg(Set caches) { + this.caches = caches; + } + + + /** + * @return Caches. + */ + public Set getCaches() { + return caches; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + U.writeCollection(out, caches); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + caches = U.readSet(in); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorIdleVerifyTaskArg.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskResult.java new file mode 100644 index 0000000000000..7ef542f019e62 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskResult.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.visor.verify; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.List; +import java.util.Map; +import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecord; +import org.apache.ignite.internal.processors.cache.verify.PartitionKey; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * Result for task {@link VisorIdleVerifyTask} + */ +public class VisorIdleVerifyTaskResult extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Results. */ + private Map> conflicts; + + /** + * Default constructor. + */ + public VisorIdleVerifyTaskResult() { + // No-op. + } + + /** + * @param conflicts Result. + */ + public VisorIdleVerifyTaskResult(Map> conflicts) { + this.conflicts = conflicts; + } + + /** + * @return Results. + */ + public Map> getConflicts() { + return conflicts; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + U.writeMap(out, conflicts); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + conflicts = U.readMap(in); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorIdleVerifyTaskResult.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesJobResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesJobResult.java new file mode 100644 index 0000000000000..25c97b651a4d9 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesJobResult.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.visor.verify; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.Map; +import org.apache.ignite.internal.processors.cache.verify.PartitionKey; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * + */ +public class VisorValidateIndexesJobResult extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Results of indexes validation from node. */ + private Map res; + + /** + * @param res Results of indexes validation from node. + */ + public VisorValidateIndexesJobResult(Map res) { + this.res = res; + } + + /** + * For externalization only. + */ + public VisorValidateIndexesJobResult() { + } + + /** + * @return Results of indexes validation from node. + */ + public Map response() { + return res; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + U.writeMap(out, res); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + res = U.readMap(in); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorValidateIndexesJobResult.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTaskArg.java new file mode 100644 index 0000000000000..cf9aff5ac2033 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTaskArg.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.visor.verify; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.Set; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * Arguments for task {@link VisorIdleVerifyTask} + */ +public class VisorValidateIndexesTaskArg extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Caches. */ + private Set caches; + + /** + * Default constructor. + */ + public VisorValidateIndexesTaskArg() { + // No-op. + } + + /** + * @param caches Caches. + */ + public VisorValidateIndexesTaskArg(Set caches) { + this.caches = caches; + } + + + /** + * @return Caches. + */ + public Set getCaches() { + return caches; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + U.writeCollection(out, caches); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + caches = U.readSet(in); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorValidateIndexesTaskArg.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTaskResult.java new file mode 100644 index 0000000000000..e206448c43ccc --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTaskResult.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.internal.visor.verify; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * + */ +public class VisorValidateIndexesTaskResult extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Exceptions. */ + private Map exceptions; + + /** Results from cluster. */ + private Map results; + + /** + * @param results Results. + * @param exceptions Exceptions. + */ + public VisorValidateIndexesTaskResult(Map results, + Map exceptions) { + this.exceptions = exceptions; + this.results = results; + } + + /** + * For externalization only. + */ + public VisorValidateIndexesTaskResult() { + } + + /** + * @return Exceptions. + */ + public Map exceptions() { + return exceptions; + } + + /** + * @return Results from cluster. + */ + public Map results() { + return results; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + U.writeMap(out, exceptions); + U.writeMap(out, results); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + exceptions = U.readMap(in); + results = U.readMap(in); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorValidateIndexesTaskResult.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorViewCacheCmd.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorViewCacheCmd.java new file mode 100644 index 0000000000000..0d9ce3ae20ad8 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorViewCacheCmd.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.visor.verify; + +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public enum VisorViewCacheCmd { + /** Caches. */ + CACHES, + + /** Groups. */ + GROUPS, + + /** Sequence. */ + SEQ; + + /** Enumerated values. */ + private static final VisorViewCacheCmd[] VALS = values(); + + /** + * Efficiently gets enumerated value from its ordinal. + * + * @param ord Ordinal value. + * @return Enumerated value or {@code null} if ordinal out of range. + */ + @Nullable public static VisorViewCacheCmd fromOrdinal(int ord) { + return ord >= 0 && ord < VALS.length ? VALS[ord] : null; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorViewCacheTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorViewCacheTask.java new file mode 100644 index 0000000000000..86931e60c64ae --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorViewCacheTask.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.visor.verify; + +import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.processors.cache.verify.ViewCacheClosure; +import org.apache.ignite.internal.processors.task.GridInternal; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.visor.VisorJob; +import org.apache.ignite.internal.visor.VisorOneNodeTask; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +@GridInternal +public class VisorViewCacheTask extends VisorOneNodeTask { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override protected VisorJob job(VisorViewCacheTaskArg arg) { + return new VisorViewCacheJob(arg, debug); + } + + /** + * + */ + private static class VisorViewCacheJob extends VisorJob { + /** */ + private static final long serialVersionUID = 0L; + + /** + * @param arg Argument. + * @param debug Debug. + */ + protected VisorViewCacheJob(@Nullable VisorViewCacheTaskArg arg, boolean debug) { + super(arg, debug); + } + + /** {@inheritDoc} */ + @Override protected VisorViewCacheTaskResult run(@Nullable VisorViewCacheTaskArg arg) throws IgniteException { + try { + ViewCacheClosure clo = new ViewCacheClosure(arg.regex(), arg.command()); + + ignite.context().resource().injectGeneric(clo); + + return new VisorViewCacheTaskResult(clo.call()); + } + catch (Exception e) { + throw new IgniteException(e); + } + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorViewCacheJob.class, this); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorViewCacheTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorViewCacheTaskArg.java new file mode 100644 index 0000000000000..5fcd66d18d082 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorViewCacheTaskArg.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.internal.visor.verify; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +public class VisorViewCacheTaskArg extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Regex. */ + private String regex; + + /** Type. */ + private @Nullable VisorViewCacheCmd cmd; + + /** + * @param regex Regex. + * @param cmd Command. + */ + public VisorViewCacheTaskArg(String regex, @Nullable VisorViewCacheCmd cmd) { + this.regex = regex; + this.cmd = cmd; + } + + /** + * For externalization only. + */ + public VisorViewCacheTaskArg() { + } + + /** + * @return Regex. + */ + public String regex() { + return regex; + } + + /** + * @return Command. + */ + public VisorViewCacheCmd command() { + return cmd; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + U.writeString(out, regex); + U.writeEnum(out, cmd); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + regex = U.readString(in); + cmd = VisorViewCacheCmd.fromOrdinal(in.readByte()); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorViewCacheTaskArg.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorViewCacheTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorViewCacheTaskResult.java new file mode 100644 index 0000000000000..138bf064710b0 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorViewCacheTaskResult.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.visor.verify; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.Collection; +import java.util.List; +import org.apache.ignite.internal.processors.cache.verify.CacheInfo; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * + */ +public class VisorViewCacheTaskResult extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Cache infos. */ + private List cacheInfos; + + /** + * @param cacheInfos Cache infos. + */ + public VisorViewCacheTaskResult(List cacheInfos) { + this.cacheInfos = cacheInfos; + } + + /** + * For externalization only. + */ + public VisorViewCacheTaskResult() { + } + + /** + * @return Cache infos. + */ + public Collection cacheInfos() { + return cacheInfos; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + U.writeCollection(out, cacheInfos); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + cacheInfos = U.readList(in); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorViewCacheTaskResult.class, this); + } +} diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties index d939b02018bbb..8ca47c8430830 100644 --- a/modules/core/src/main/resources/META-INF/classnames.properties +++ b/modules/core/src/main/resources/META-INF/classnames.properties @@ -28,6 +28,7 @@ org.apache.ignite.IgniteState org.apache.ignite.binary.BinaryInvalidTypeException org.apache.ignite.binary.BinaryObject org.apache.ignite.binary.BinaryObjectException +org.apache.ignite.binary.BinaryTypeConfiguration org.apache.ignite.cache.CacheAtomicUpdateTimeoutException org.apache.ignite.cache.CacheAtomicityMode org.apache.ignite.cache.CacheEntryEventSerializableFilter @@ -112,6 +113,13 @@ org.apache.ignite.cache.store.jdbc.dialect.SQLServerDialect org.apache.ignite.cache.store.jdbc.dialect.SQLServerDialect$1 org.apache.ignite.cache.store.jdbc.dialect.SQLServerDialect$2 org.apache.ignite.cache.store.jdbc.dialect.SQLServerDialect$3 +org.apache.ignite.client.ClientAuthenticationException +org.apache.ignite.client.ClientAuthorizationException +org.apache.ignite.client.ClientCacheConfiguration +org.apache.ignite.client.ClientConnectionException +org.apache.ignite.client.ClientException +org.apache.ignite.client.SslMode +org.apache.ignite.client.SslProtocol org.apache.ignite.cluster.ClusterGroupEmptyException org.apache.ignite.cluster.ClusterTopologyException org.apache.ignite.compute.ComputeExecutionRejectedException @@ -135,9 +143,11 @@ org.apache.ignite.compute.gridify.GridifyTaskSplitAdapter org.apache.ignite.compute.gridify.aop.GridifyArgumentAdapter org.apache.ignite.compute.gridify.aop.GridifyDefaultRangeTask org.apache.ignite.compute.gridify.aop.GridifyDefaultTask +org.apache.ignite.configuration.BinaryConfiguration org.apache.ignite.configuration.CacheConfiguration org.apache.ignite.configuration.CacheConfiguration$IgniteAllNodesPredicate org.apache.ignite.configuration.CheckpointWriteOrder +org.apache.ignite.configuration.ClientConfiguration org.apache.ignite.configuration.CollectionConfiguration org.apache.ignite.configuration.DataPageEvictionMode org.apache.ignite.configuration.DataRegionConfiguration @@ -164,6 +174,7 @@ org.apache.ignite.events.IgfsEvent org.apache.ignite.events.JobEvent org.apache.ignite.events.TaskEvent org.apache.ignite.events.WalSegmentArchivedEvent +org.apache.ignite.failure.FailureType org.apache.ignite.hadoop.HadoopInputSplit org.apache.ignite.hadoop.HadoopMapReducePlan org.apache.ignite.igfs.IgfsConcurrentModificationException @@ -247,7 +258,7 @@ org.apache.ignite.internal.IgniteMessagingImpl org.apache.ignite.internal.IgniteNeedReconnectException org.apache.ignite.internal.IgniteSchedulerImpl org.apache.ignite.internal.IgniteServicesImpl -org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance$1 +org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance$4 org.apache.ignite.internal.NodeStoppingException org.apache.ignite.internal.binary.BinaryEnumObjectImpl org.apache.ignite.internal.binary.BinaryFieldMetadata @@ -286,6 +297,11 @@ org.apache.ignite.internal.client.impl.connection.GridClientNioTcpConnection$6 org.apache.ignite.internal.client.impl.connection.GridClientNioTcpConnection$7 org.apache.ignite.internal.client.impl.connection.GridClientTopology$1 org.apache.ignite.internal.client.impl.connection.GridConnectionIdleClosedException +org.apache.ignite.internal.client.thin.ClientError +org.apache.ignite.internal.client.thin.ClientOperation +org.apache.ignite.internal.client.thin.ClientProtocolError +org.apache.ignite.internal.client.thin.ClientServerError +org.apache.ignite.internal.client.thin.ClientUtils$CfgItem org.apache.ignite.internal.cluster.ClusterGroupAdapter org.apache.ignite.internal.cluster.ClusterGroupAdapter$AgeClusterGroup org.apache.ignite.internal.cluster.ClusterGroupAdapter$AttributeFilter @@ -305,6 +321,8 @@ org.apache.ignite.internal.cluster.IgniteKillTask org.apache.ignite.internal.cluster.IgniteKillTask$IgniteKillJob org.apache.ignite.internal.cluster.NodeOrderComparator org.apache.ignite.internal.cluster.NodeOrderLegacyComparator +org.apache.ignite.internal.commandline.Command +org.apache.ignite.internal.commandline.cache.CacheCommand org.apache.ignite.internal.compute.ComputeTaskCancelledCheckedException org.apache.ignite.internal.compute.ComputeTaskTimeoutCheckedException org.apache.ignite.internal.direct.DirectMessageReader$1 @@ -317,7 +335,6 @@ org.apache.ignite.internal.igfs.common.IgfsIpcCommand org.apache.ignite.internal.jdbc.thin.ConnectionPropertiesImpl org.apache.ignite.internal.jdbc.thin.ConnectionPropertiesImpl$BooleanProperty org.apache.ignite.internal.jdbc.thin.ConnectionPropertiesImpl$ConnectionProperty -org.apache.ignite.internal.jdbc.thin.ConnectionPropertiesImpl$EmptyStringValidator org.apache.ignite.internal.jdbc.thin.ConnectionPropertiesImpl$IntegerProperty org.apache.ignite.internal.jdbc.thin.ConnectionPropertiesImpl$NumberProperty org.apache.ignite.internal.jdbc.thin.ConnectionPropertiesImpl$PropertyValidator @@ -348,6 +365,7 @@ org.apache.ignite.internal.managers.discovery.DiscoCache$1 org.apache.ignite.internal.managers.discovery.DiscoCache$2 org.apache.ignite.internal.managers.discovery.DiscoCache$3 org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage +org.apache.ignite.internal.managers.discovery.DiscoveryServerOnlyCustomMessage org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$1 org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$2 org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$4$1 @@ -372,6 +390,19 @@ org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion org.apache.ignite.internal.processors.affinity.GridAffinityAssignment org.apache.ignite.internal.processors.affinity.GridAffinityMessage org.apache.ignite.internal.processors.affinity.GridAffinityUtils$AffinityJob +org.apache.ignite.internal.processors.authentication.IgniteAccessControlException +org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor$3 +org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor$InitialUsersData +org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor$RefreshUsersStorageWorker$1 +org.apache.ignite.internal.processors.authentication.User +org.apache.ignite.internal.processors.authentication.UserAcceptedMessage +org.apache.ignite.internal.processors.authentication.UserAuthenticateRequestMessage +org.apache.ignite.internal.processors.authentication.UserAuthenticateResponseMessage +org.apache.ignite.internal.processors.authentication.UserManagementException +org.apache.ignite.internal.processors.authentication.UserManagementOperation +org.apache.ignite.internal.processors.authentication.UserManagementOperation$OperationType +org.apache.ignite.internal.processors.authentication.UserManagementOperationFinishedMessage +org.apache.ignite.internal.processors.authentication.UserProposedMessage org.apache.ignite.internal.processors.bulkload.BulkLoadCacheWriter org.apache.ignite.internal.processors.bulkload.BulkLoadStreamerWriter org.apache.ignite.internal.processors.cache.CacheAffinityChangeMessage @@ -528,6 +559,7 @@ org.apache.ignite.internal.processors.cache.GridCacheLoaderWriterStore org.apache.ignite.internal.processors.cache.GridCacheLoaderWriterStoreFactory org.apache.ignite.internal.processors.cache.GridCacheLockTimeoutException org.apache.ignite.internal.processors.cache.GridCacheLogger +org.apache.ignite.internal.processors.cache.GridCacheMapEntry$1 org.apache.ignite.internal.processors.cache.GridCacheMessage org.apache.ignite.internal.processors.cache.GridCacheMultiTxFuture$1 org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate @@ -616,6 +648,7 @@ org.apache.ignite.internal.processors.cache.WalStateAbstractMessage org.apache.ignite.internal.processors.cache.WalStateAckMessage org.apache.ignite.internal.processors.cache.WalStateFinishMessage org.apache.ignite.internal.processors.cache.WalStateManager$2 +org.apache.ignite.internal.processors.cache.WalStateManager$3 org.apache.ignite.internal.processors.cache.WalStateProposeMessage org.apache.ignite.internal.processors.cache.affinity.GridCacheAffinityProxy org.apache.ignite.internal.processors.cache.binary.BinaryMetadataHolder @@ -803,17 +836,15 @@ org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CacheParti org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysFuture$1 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysRequest org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponse +org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandLegacyMessage org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$1 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$1$1 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$2 -org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$3 -org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$4$1 -org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$5$1 +org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$3$1 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap -org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplier$SupplyContextPhase org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsAbstractMessage org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture$2 @@ -837,6 +868,9 @@ org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtP org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtPartitionCountersMap2 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtPartitionHistorySuppliersMap org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtPartitionsToReloadMap +org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteHistoricalIterator +org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteRebalanceIteratorImpl +org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.LatchAckMessage org.apache.ignite.internal.processors.cache.distributed.near.CacheVersionedValue org.apache.ignite.internal.processors.cache.distributed.near.GridNearAtomicCache org.apache.ignite.internal.processors.cache.distributed.near.GridNearAtomicCache$1 @@ -903,6 +937,7 @@ org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal$20 org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal$21 org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal$22 org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal$23 +org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal$24 org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal$3 org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal$4 org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal$5 @@ -926,11 +961,11 @@ org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache$5 org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache$8 org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache$9 org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter$RowData -org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager$11 +org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager$12 org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager$6 -org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager$8 +org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager$9 org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager$CheckpointEntryType -org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager$RebalanceIteratorAdapter +org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager$WALHistoricalIterator org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager$1 org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIOFactory org.apache.ignite.internal.processors.cache.persistence.file.FileDownloader$1 @@ -942,6 +977,7 @@ org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImpl$T org.apache.ignite.internal.processors.cache.persistence.pagemem.PagesWriteSpeedBasedThrottle$ThrottleMode org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotDiscoveryMessage org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperation +org.apache.ignite.internal.processors.cache.persistence.snapshot.TrackingPageIsCorruptedException org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree$Bool org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree$DestroyBag org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree$Result @@ -1095,8 +1131,13 @@ org.apache.ignite.internal.processors.cache.transactions.TxLock org.apache.ignite.internal.processors.cache.transactions.TxLockList org.apache.ignite.internal.processors.cache.transactions.TxLocksRequest org.apache.ignite.internal.processors.cache.transactions.TxLocksResponse +org.apache.ignite.internal.processors.cache.verify.CacheInfo +org.apache.ignite.internal.processors.cache.verify.CacheInfo$1 +org.apache.ignite.internal.processors.cache.verify.CacheInfo$2 org.apache.ignite.internal.processors.cache.verify.CollectConflictPartitionKeysTask org.apache.ignite.internal.processors.cache.verify.CollectConflictPartitionKeysTask$CollectPartitionEntryHashesJob +org.apache.ignite.internal.processors.cache.verify.ContentionClosure +org.apache.ignite.internal.processors.cache.verify.ContentionInfo org.apache.ignite.internal.processors.cache.verify.PartitionEntryHashRecord org.apache.ignite.internal.processors.cache.verify.PartitionHashRecord org.apache.ignite.internal.processors.cache.verify.PartitionKey @@ -1104,6 +1145,8 @@ org.apache.ignite.internal.processors.cache.verify.RetrieveConflictPartitionValu org.apache.ignite.internal.processors.cache.verify.RetrieveConflictPartitionValuesTask$RetrieveConflictValuesJob org.apache.ignite.internal.processors.cache.verify.VerifyBackupPartitionsTask org.apache.ignite.internal.processors.cache.verify.VerifyBackupPartitionsTask$VerifyBackupPartitionsJob +org.apache.ignite.internal.processors.cache.verify.ViewCacheClosure +org.apache.ignite.internal.processors.cache.verify.ViewCacheClosure$1 org.apache.ignite.internal.processors.cache.version.GridCacheRawVersionedEntry org.apache.ignite.internal.processors.cache.version.GridCacheVersion org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext$State @@ -1130,15 +1173,16 @@ org.apache.ignite.internal.processors.closure.GridClosureProcessor$T8 org.apache.ignite.internal.processors.closure.GridClosureProcessor$T9 org.apache.ignite.internal.processors.closure.GridClosureProcessor$TaskNoReduceAdapter org.apache.ignite.internal.processors.closure.GridPeerDeployAwareTaskAdapter -org.apache.ignite.internal.processors.cluster.ClusterNodeMetrics org.apache.ignite.internal.processors.cluster.BaselineTopology org.apache.ignite.internal.processors.cluster.BaselineTopologyHistory org.apache.ignite.internal.processors.cluster.BaselineTopologyHistoryItem org.apache.ignite.internal.processors.cluster.BranchingPointType org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage -org.apache.ignite.internal.processors.cluster.ClusterProcessor$3 -org.apache.ignite.internal.processors.cluster.ClusterProcessor$3$1 +org.apache.ignite.internal.processors.cluster.ClusterMetricsUpdateMessage +org.apache.ignite.internal.processors.cluster.ClusterNodeMetrics +org.apache.ignite.internal.processors.cluster.ClusterProcessor$4 +org.apache.ignite.internal.processors.cluster.ClusterProcessor$4$1 org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor$1$1 org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor$2 @@ -1150,18 +1194,24 @@ org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor$Baseline org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor$CheckGlobalStateComputeRequest org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor$ClientChangeGlobalStateComputeRequest org.apache.ignite.internal.processors.continuous.AbstractContinuousMessage +org.apache.ignite.internal.processors.continuous.ContinuousRoutineInfo +org.apache.ignite.internal.processors.continuous.ContinuousRoutineStartResultMessage +org.apache.ignite.internal.processors.continuous.ContinuousRoutinesCommonDiscoveryData +org.apache.ignite.internal.processors.continuous.ContinuousRoutinesJoiningNodeDiscoveryData org.apache.ignite.internal.processors.continuous.GridContinuousHandler org.apache.ignite.internal.processors.continuous.GridContinuousHandler$RegisterStatus org.apache.ignite.internal.processors.continuous.GridContinuousMessage org.apache.ignite.internal.processors.continuous.GridContinuousMessageType -org.apache.ignite.internal.processors.continuous.GridContinuousProcessor$8 -org.apache.ignite.internal.processors.continuous.GridContinuousProcessor$9$1 +org.apache.ignite.internal.processors.continuous.GridContinuousProcessor$11$1 +org.apache.ignite.internal.processors.continuous.GridContinuousProcessor$9 org.apache.ignite.internal.processors.continuous.GridContinuousProcessor$DiscoveryData org.apache.ignite.internal.processors.continuous.GridContinuousProcessor$DiscoveryDataItem org.apache.ignite.internal.processors.continuous.GridContinuousProcessor$LocalRoutineInfo org.apache.ignite.internal.processors.continuous.StartRequestData +org.apache.ignite.internal.processors.continuous.StartRequestDataV2 org.apache.ignite.internal.processors.continuous.StartRoutineAckDiscoveryMessage org.apache.ignite.internal.processors.continuous.StartRoutineDiscoveryMessage +org.apache.ignite.internal.processors.continuous.StartRoutineDiscoveryMessageV2 org.apache.ignite.internal.processors.continuous.StopRoutineAckDiscoveryMessage org.apache.ignite.internal.processors.continuous.StopRoutineDiscoveryMessage org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor$3 @@ -1427,6 +1477,7 @@ org.apache.ignite.internal.processors.query.GridQueryProcessor$5 org.apache.ignite.internal.processors.query.GridQueryProcessor$6 org.apache.ignite.internal.processors.query.GridQueryProcessor$7 org.apache.ignite.internal.processors.query.GridQueryProcessor$8 +org.apache.ignite.internal.processors.query.GridQueryProcessor$9 org.apache.ignite.internal.processors.query.GridQueryProcessor$SchemaOperation$1 org.apache.ignite.internal.processors.query.IgniteSQLException org.apache.ignite.internal.processors.query.QueryEntityEx @@ -1543,6 +1594,7 @@ org.apache.ignite.internal.processors.service.GridServiceProcessor$ServiceDeploy org.apache.ignite.internal.processors.service.GridServiceProcessor$ServiceTopologyCallable org.apache.ignite.internal.processors.service.GridServiceProxy org.apache.ignite.internal.processors.service.GridServiceProxy$ServiceProxyCallable +org.apache.ignite.internal.processors.service.GridServiceProxy$ServiceProxyException org.apache.ignite.internal.processors.service.LazyServiceConfiguration org.apache.ignite.internal.processors.service.ServiceContextImpl org.apache.ignite.internal.processors.service.ServiceDescriptorImpl @@ -1622,6 +1674,7 @@ org.apache.ignite.internal.util.GridSnapshotLock$Sync org.apache.ignite.internal.util.GridSpiCloseableIteratorWrapper org.apache.ignite.internal.util.GridStringBuilder org.apache.ignite.internal.util.GridSynchronizedMap +org.apache.ignite.internal.util.HostAndPortRange org.apache.ignite.internal.util.IgniteExceptionRegistry$ExceptionInfo org.apache.ignite.internal.util.IgniteTree$OperationType org.apache.ignite.internal.util.IgniteUtils$10 @@ -1860,6 +1913,10 @@ org.apache.ignite.internal.visor.cache.VisorCacheJdbcTypeField org.apache.ignite.internal.visor.cache.VisorCacheLoadTask org.apache.ignite.internal.visor.cache.VisorCacheLoadTask$VisorCachesLoadJob org.apache.ignite.internal.visor.cache.VisorCacheLoadTaskArg +org.apache.ignite.internal.visor.cache.VisorCacheLostPartitionsTask +org.apache.ignite.internal.visor.cache.VisorCacheLostPartitionsTask$VisorCacheLostPartitionsJob +org.apache.ignite.internal.visor.cache.VisorCacheLostPartitionsTaskArg +org.apache.ignite.internal.visor.cache.VisorCacheLostPartitionsTaskResult org.apache.ignite.internal.visor.cache.VisorCacheMetadataTask org.apache.ignite.internal.visor.cache.VisorCacheMetadataTask$VisorCacheMetadataJob org.apache.ignite.internal.visor.cache.VisorCacheMetadataTaskArg @@ -1883,6 +1940,9 @@ org.apache.ignite.internal.visor.cache.VisorCacheRebalanceConfiguration org.apache.ignite.internal.visor.cache.VisorCacheRebalanceTask org.apache.ignite.internal.visor.cache.VisorCacheRebalanceTask$VisorCachesRebalanceJob org.apache.ignite.internal.visor.cache.VisorCacheRebalanceTaskArg +org.apache.ignite.internal.visor.cache.VisorCacheResetLostPartitionsTask +org.apache.ignite.internal.visor.cache.VisorCacheResetLostPartitionsTask$VisorCacheResetLostPartitionsJob +org.apache.ignite.internal.visor.cache.VisorCacheResetLostPartitionsTaskArg org.apache.ignite.internal.visor.cache.VisorCacheResetMetricsTask org.apache.ignite.internal.visor.cache.VisorCacheResetMetricsTask$VisorCacheResetMetricsJob org.apache.ignite.internal.visor.cache.VisorCacheResetMetricsTaskArg @@ -2076,6 +2136,31 @@ org.apache.ignite.internal.visor.tx.VisorTxSortOrder org.apache.ignite.internal.visor.tx.VisorTxTask org.apache.ignite.internal.visor.tx.VisorTxTaskArg org.apache.ignite.internal.visor.tx.VisorTxTaskResult +org.apache.ignite.internal.visor.verify.VisorViewCacheCmd +org.apache.ignite.internal.visor.verify.ValidateIndexesPartitionResult +org.apache.ignite.internal.visor.verify.VisorContentionJobResult +org.apache.ignite.internal.visor.verify.VisorContentionTask +org.apache.ignite.internal.visor.verify.VisorContentionTask$VisorContentionJob +org.apache.ignite.internal.visor.verify.VisorContentionTaskArg +org.apache.ignite.internal.visor.verify.VisorContentionTaskResult +org.apache.ignite.internal.visor.verify.VisorIdleAnalyzeTask +org.apache.ignite.internal.visor.verify.VisorIdleAnalyzeTask$VisorIdleVerifyJob +org.apache.ignite.internal.visor.verify.VisorIdleAnalyzeTask$VisorIdleVerifyJob$1 +org.apache.ignite.internal.visor.verify.VisorIdleAnalyzeTask$VisorIdleVerifyJob$2 +org.apache.ignite.internal.visor.verify.VisorIdleAnalyzeTaskArg +org.apache.ignite.internal.visor.verify.VisorIdleAnalyzeTaskResult +org.apache.ignite.internal.visor.verify.VisorIdleVerifyTask +org.apache.ignite.internal.visor.verify.VisorIdleVerifyTask$VisorIdleVerifyJob +org.apache.ignite.internal.visor.verify.VisorIdleVerifyTask$VisorIdleVerifyJob$1 +org.apache.ignite.internal.visor.verify.VisorIdleVerifyTaskArg +org.apache.ignite.internal.visor.verify.VisorIdleVerifyTaskResult +org.apache.ignite.internal.visor.verify.VisorValidateIndexesJobResult +org.apache.ignite.internal.visor.verify.VisorValidateIndexesTaskArg +org.apache.ignite.internal.visor.verify.VisorValidateIndexesTaskResult +org.apache.ignite.internal.visor.verify.VisorViewCacheTask +org.apache.ignite.internal.visor.verify.VisorViewCacheTask$VisorViewCacheJob +org.apache.ignite.internal.visor.verify.VisorViewCacheTaskArg +org.apache.ignite.internal.visor.verify.VisorViewCacheTaskResult org.apache.ignite.internal.websession.WebSessionAttributeProcessor org.apache.ignite.internal.websession.WebSessionEntity org.apache.ignite.lang.IgniteBiClosure @@ -2143,6 +2228,7 @@ org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$4 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeClosure org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeException org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeTimeoutException +org.apache.ignite.spi.communication.tcp.internal.TcpCommunicationConnectionCheckFuture$SingleAddressConnectFuture$1 org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage2 org.apache.ignite.spi.communication.tcp.messages.NodeIdMessage diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java index c2d9eec4bae58..dbd6107f737b1 100644 --- a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java @@ -22,13 +22,19 @@ import java.io.PrintStream; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; import java.util.Map; import java.util.TreeMap; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteAtomicSequence; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.ConnectorConfiguration; @@ -36,8 +42,10 @@ 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.IgniteInternalFuture; import org.apache.ignite.internal.commandline.CommandHandler; +import org.apache.ignite.internal.commandline.cache.CacheCommand; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.X; @@ -49,8 +57,8 @@ import org.apache.ignite.transactions.Transaction; import org.apache.ignite.transactions.TransactionRollbackException; -import static org.apache.ignite.cache.CacheAtomicityMode.*; -import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*; +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; import static org.apache.ignite.internal.commandline.CommandHandler.EXIT_CODE_OK; import static org.apache.ignite.internal.commandline.CommandHandler.EXIT_CODE_UNEXPECTED_ERROR; import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC; @@ -61,6 +69,12 @@ * Command line handler test. */ public class GridCommandHandlerTest extends GridCommonAbstractTest { + /** System out. */ + protected PrintStream sysOut; + + /** Test out - can be injected via {@link #injectTestSystemOut()} instead of System.out and analyzed in test. */ + protected ByteArrayOutputStream testOut; + /** * @return Folder in work directory. * @throws IgniteCheckedException If failed to resolve folder name. @@ -74,6 +88,10 @@ protected File folder(String folder) throws IgniteCheckedException { cleanPersistenceDir(); stopAllGrids(); + + sysOut = System.out; + + testOut = new ByteArrayOutputStream(128 * 1024); } /** {@inheritDoc} */ @@ -81,6 +99,18 @@ protected File folder(String folder) throws IgniteCheckedException { stopAllGrids(); cleanPersistenceDir(); + + System.setOut(sysOut); + + if (testOut != null) + System.out.println(testOut.toString()); + } + + /** + * + */ + protected void injectTestSystemOut() { + System.setOut(new PrintStream(testOut)); } /** {@inheritDoc} */ @@ -262,40 +292,6 @@ public void testBaselineAdd() throws Exception { assertEquals(2, ignite.cluster().currentBaselineTopology().size()); } - /** - * Test baseline add items works via control.sh - * - * @throws Exception If failed. - */ - public void testBaselineAddOnNotActiveCluster() throws Exception { - try { - Ignite ignite = startGrid(1); - - assertFalse(ignite.cluster().active()); - - String consistentIDs = getTestIgniteInstanceName(1); - - ByteArrayOutputStream out = new ByteArrayOutputStream(4096); - System.setOut(new PrintStream(out)); - - assertEquals(EXIT_CODE_UNEXPECTED_ERROR, execute("--baseline", "add", consistentIDs)); - - assertTrue(out.toString().contains("Changing BaselineTopology on inactive cluster is not allowed.")); - - consistentIDs = - getTestIgniteInstanceName(1) + ", " + - getTestIgniteInstanceName(2) + "," + - getTestIgniteInstanceName(3); - - assertEquals(EXIT_CODE_UNEXPECTED_ERROR, execute("--baseline", "add", consistentIDs)); - - assertTrue(out.toString().contains("Node not found for consistent ID: bltTest2")); - } - finally { - System.setOut(System.out); - } - } - /** * Test baseline remove works via control.sh * @@ -373,7 +369,7 @@ public void testActiveTransactions() throws Exception { Ignite client = startGrid("client"); - IgniteCache cache = client.getOrCreateCache(new CacheConfiguration<>(DEFAULT_CACHE_NAME) + client.getOrCreateCache(new CacheConfiguration<>(DEFAULT_CACHE_NAME) .setAtomicityMode(TRANSACTIONAL).setWriteSynchronizationMode(FULL_SYNC)); for (Ignite ig : G.allGrids()) @@ -401,9 +397,9 @@ public void testActiveTransactions() throws Exception { fail("Commit must fail"); } - catch (Exception ignored) { + catch (Exception e) { // No-op. - assertTrue(X.hasCause(ignored, TransactionRollbackException.class)); + assertTrue(X.hasCause(e, TransactionRollbackException.class)); } break; @@ -534,14 +530,246 @@ else if (entry.getKey().equals(node2)) { fut.get(); } + /** + * Test baseline add items works via control.sh + * + * @throws Exception If failed. + */ + public void testBaselineAddOnNotActiveCluster() throws Exception { + Ignite ignite = startGrid(1); + + assertFalse(ignite.cluster().active()); + + String consistentIDs = getTestIgniteInstanceName(1); + + injectTestSystemOut(); + + assertEquals(EXIT_CODE_UNEXPECTED_ERROR, execute("--baseline", "add", consistentIDs)); + + assertTrue(testOut.toString().contains("Changing BaselineTopology on inactive cluster is not allowed.")); + + consistentIDs = + getTestIgniteInstanceName(1) + ", " + + getTestIgniteInstanceName(2) + "," + + getTestIgniteInstanceName(3); + + assertEquals(EXIT_CODE_UNEXPECTED_ERROR, execute("--baseline", "add", consistentIDs)); + + assertTrue(testOut.toString().contains("Node not found for consistent ID: bltTest2")); + } + + /** + * @throws Exception If failed. + */ + public void testCacheHelp() throws Exception { + Ignite ignite = startGrids(1); + + ignite.cluster().active(true); + + injectTestSystemOut(); + + assertEquals(EXIT_CODE_OK, execute("--cache", "help")); + + for (CacheCommand cmd : CacheCommand.values()) { + if (cmd != CacheCommand.HELP) + assertTrue(cmd.text(), testOut.toString().contains(cmd.text())); + } + } + + /** + * @throws Exception If failed. + */ + public void testCacheIdleVerify() throws Exception { + Ignite ignite = startGrids(2); + + ignite.cluster().active(true); + + IgniteCache cache = ignite.createCache(new CacheConfiguration<>() + .setAffinity(new RendezvousAffinityFunction(false, 32)) + .setBackups(1) + .setName("cacheIV")); + + for (int i = 0; i < 100; i++) + cache.put(i, i); + + injectTestSystemOut(); + + assertEquals(EXIT_CODE_OK, execute("--cache", "idle_verify")); + + assertTrue(testOut.toString().contains("no conflicts have been found")); + + HashSet clearKeys = new HashSet<>(Arrays.asList(1, 2, 3, 4, 5, 6)); + + ((IgniteEx)ignite).context().cache().cache("cacheIV").clearLocallyAll(clearKeys, true, true, true); + + assertEquals(EXIT_CODE_OK, execute("--cache", "idle_verify")); + + assertTrue(testOut.toString().contains("conflict partitions")); + } + + /** + * + */ + public void testCacheContention() throws Exception { + int cnt = 10; + + final ExecutorService svc = Executors.newFixedThreadPool(cnt); + + try { + Ignite ignite = startGrids(2); + + ignite.cluster().active(true); + + final IgniteCache cache = ignite.createCache(new CacheConfiguration<>() + .setAffinity(new RendezvousAffinityFunction(false, 32)) + .setAtomicityMode(TRANSACTIONAL) + .setBackups(1) + .setName("cacheCont")); + + final CountDownLatch l = new CountDownLatch(1); + + final CountDownLatch l2 = new CountDownLatch(1); + + svc.submit(new Runnable() { + @Override public void run() { + try (final Transaction tx = ignite.transactions().txStart()) { + cache.put(0, 0); + + l.countDown(); + + U.awaitQuiet(l2); + + tx.commit(); + } + } + }); + + for (int i = 0; i < cnt - 1; i++) { + svc.submit(new Runnable() { + @Override public void run() { + U.awaitQuiet(l); + + try (final Transaction tx = ignite.transactions().txStart()) { + cache.get(0); + + tx.commit(); + } + } + }); + } + + U.awaitQuiet(l); + + Thread.sleep(300); + + injectTestSystemOut(); + + assertEquals(EXIT_CODE_OK, execute("--cache", "contention", "5")); + + l2.countDown(); + + assertTrue(testOut.toString().contains("TxEntry")); + assertTrue(testOut.toString().contains("op=READ")); + assertTrue(testOut.toString().contains("op=CREATE")); + assertTrue(testOut.toString().contains("id=" + ignite(0).cluster().localNode().id())); + assertTrue(testOut.toString().contains("id=" + ignite(1).cluster().localNode().id())); + } + finally { + svc.shutdown(); + svc.awaitTermination(100, TimeUnit.DAYS); + } + } + + /** + * + */ + public void testCacheSequence() throws Exception { + Ignite ignite = startGrid(); + + ignite.cluster().active(true); + + Ignite client = startGrid("client"); + + final IgniteAtomicSequence seq1 = client.atomicSequence("testSeq", 1, true); + seq1.get(); + + final IgniteAtomicSequence seq2 = client.atomicSequence("testSeq2", 10, true); + seq2.get(); + + injectTestSystemOut(); + + assertEquals(EXIT_CODE_OK, execute("--cache", "list", "testSeq.*", "seq")); + + assertTrue(testOut.toString().contains("testSeq")); + assertTrue(testOut.toString().contains("testSeq2")); + } + + /** + * + */ + public void testCacheGroups() throws Exception { + Ignite ignite = startGrid(); + + ignite.cluster().active(true); + + IgniteCache cache1 = ignite.createCache(new CacheConfiguration<>() + .setAffinity(new RendezvousAffinityFunction(false, 32)) + .setBackups(1) + .setGroupName("G100") + .setName("cacheG1")); + + IgniteCache cache2 = ignite.createCache(new CacheConfiguration<>() + .setAffinity(new RendezvousAffinityFunction(false, 32)) + .setBackups(1) + .setGroupName("G100") + .setName("cacheG2")); + + for (int i = 0; i < 100; i++) { + cache1.put(i, i); + + cache2.put(i, i); + } + + injectTestSystemOut(); + + assertEquals(EXIT_CODE_OK, execute("--cache", "list", ".*", "groups")); + + assertTrue(testOut.toString().contains("G100")); + } + + /** + * + */ + public void testCacheAffinity() throws Exception { + Ignite ignite = startGrid(); + + ignite.cluster().active(true); + + IgniteCache cache1 = ignite.createCache(new CacheConfiguration<>() + .setAffinity(new RendezvousAffinityFunction(false, 32)) + .setBackups(1) + .setName("cacheAf")); + + for (int i = 0; i < 100; i++) + cache1.put(i, i); + + injectTestSystemOut(); + + assertEquals(EXIT_CODE_OK, execute("--cache", "list", ".*")); + + assertTrue(testOut.toString().contains("cacheName=cacheAf")); + assertTrue(testOut.toString().contains("prim=32")); + assertTrue(testOut.toString().contains("mapped=32")); + assertTrue(testOut.toString().contains("affCls=RendezvousAffinityFunction")); + } + /** * @param h Handler. * @param validateClo Validate clo. * @param args Args. */ - private void validate( - CommandHandler h, IgniteInClosure> validateClo, String... args) - throws IgniteCheckedException { + private void validate(CommandHandler h, IgniteInClosure> validateClo, + String... args) { assertEquals(EXIT_CODE_OK, execute(h, args)); validateClo.apply(h.getLastOperationResult()); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java new file mode 100644 index 0000000000000..373bd15dad315 --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java @@ -0,0 +1,356 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ +package org.apache.ignite.internal.visor.verify; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteInterruptedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.CacheObjectContext; +import org.apache.ignite.internal.processors.cache.CacheObjectUtils; +import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +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.GridDhtPartitionState; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.verify.PartitionKey; +import org.apache.ignite.internal.processors.query.GridQueryProcessor; +import org.apache.ignite.internal.processors.query.QueryTypeDescriptorImpl; +import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; +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.GridIterator; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.resources.LoggerResource; +import org.h2.engine.Session; +import org.h2.index.Cursor; +import org.h2.index.Index; + +/** + * + */ +public class ValidateIndexesClosure implements IgniteCallable> { + /** */ + private static final long serialVersionUID = 0L; + + /** Ignite. */ + @IgniteInstanceResource + private transient IgniteEx ignite; + + /** Injected logger. */ + @LoggerResource + private IgniteLogger log; + + /** Cache names. */ + private Set cacheNames; + + /** Counter of processed partitions. */ + private final AtomicInteger completionCntr = new AtomicInteger(0); + + /** Calculation executor. */ + private volatile ExecutorService calcExecutor; + + /** + * @param cacheNames Cache names. + */ + public ValidateIndexesClosure(Set cacheNames) { + this.cacheNames = cacheNames; + } + + /** {@inheritDoc} */ + @Override public Map call() throws Exception { + calcExecutor = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors()); + + try { + return call0(); + } + finally { + calcExecutor.shutdown(); + } + } + + /** + * + */ + private Map call0() throws Exception { + Set grpIds = new HashSet<>(); + + Set missingCaches = new HashSet<>(); + + if (cacheNames != null) { + for (String cacheName : cacheNames) { + DynamicCacheDescriptor desc = ignite.context().cache().cacheDescriptor(cacheName); + + if (desc == null) { + missingCaches.add(cacheName); + + continue; + } + + grpIds.add(desc.groupId()); + } + + if (!missingCaches.isEmpty()) { + StringBuilder strBuilder = new StringBuilder("The following caches do not exist: "); + + for (String name : missingCaches) + strBuilder.append(name).append(", "); + + strBuilder.delete(strBuilder.length() - 2, strBuilder.length()); + + throw new IgniteException(strBuilder.toString()); + } + } + else { + Collection groups = ignite.context().cache().cacheGroups(); + + for (CacheGroupContext grp : groups) { + if (!grp.systemCache() && !grp.isLocal()) + grpIds.add(grp.groupId()); + } + } + + List>> procPartFutures = new ArrayList<>(); + + completionCntr.set(0); + + for (Integer grpId : grpIds) { + CacheGroupContext grpCtx = ignite.context().cache().cacheGroup(grpId); + + if (grpCtx == null) + continue; + + List parts = grpCtx.topology().localPartitions(); + + for (GridDhtLocalPartition part : parts) + procPartFutures.add(processPartitionAsync(grpCtx, part)); + } + + Map res = new HashMap<>(); + + long lastProgressLogTs = U.currentTimeMillis(); + + for (int i = 0; i < procPartFutures.size(); ) { + Future> fut = procPartFutures.get(i); + + try { + Map partRes = fut.get(1, TimeUnit.SECONDS); + + res.putAll(partRes); + + i++; + } + catch (InterruptedException | ExecutionException e) { + for (int j = i + 1; j < procPartFutures.size(); j++) + procPartFutures.get(j).cancel(false); + + if (e instanceof InterruptedException) + throw new IgniteInterruptedException((InterruptedException)e); + else if (e.getCause() instanceof IgniteException) + throw (IgniteException)e.getCause(); + else + throw new IgniteException(e.getCause()); + } + catch (TimeoutException ignored) { + if (U.currentTimeMillis() - lastProgressLogTs > 60 * 1000L) { + lastProgressLogTs = U.currentTimeMillis(); + + log.warning("ValidateIndexesClosure is still running, processed " + completionCntr.get() + " of " + + procPartFutures.size() + " local partitions"); + } + } + } + + return res; + } + + /** + * @param grpCtx Group context. + * @param part Local partition. + */ + private Future> processPartitionAsync( + final CacheGroupContext grpCtx, + final GridDhtLocalPartition part + ) { + return calcExecutor.submit(new Callable>() { + @Override public Map call() throws Exception { + return processPartition(grpCtx, part); + } + }); + } + + /** + * @param grpCtx Group context. + * @param part Local partition. + */ + private Map processPartition( + CacheGroupContext grpCtx, + GridDhtLocalPartition part + ) { + if (!part.reserve()) + return Collections.emptyMap(); + + ValidateIndexesPartitionResult partRes; + + try { + if (part.state() != GridDhtPartitionState.OWNING) + return Collections.emptyMap(); + + long updateCntrBefore = part.updateCounter(); + + long partSize = part.dataStore().fullSize(); + + GridIterator it = grpCtx.offheap().partitionIterator(part.id()); + + Object consId = ignite.context().discovery().localNode().consistentId(); + + boolean isPrimary = part.primary(grpCtx.topology().readyTopologyVersion()); + + partRes = new ValidateIndexesPartitionResult(updateCntrBefore, partSize, isPrimary, consId); + + boolean enoughIssues = false; + + long keysProcessed = 0; + long lastProgressLog = U.currentTimeMillis(); + + while (it.hasNextX()) { + if (enoughIssues) + break; + + CacheDataRow row = it.nextX(); + + int cacheId = row.cacheId() == 0 ? grpCtx.groupId() : row.cacheId(); + + GridCacheContext cacheCtx = row.cacheId() == 0 ? + grpCtx.singleCacheContext() : grpCtx.shared().cacheContext(row.cacheId()); + + if (cacheCtx == null) + throw new IgniteException("Unknown cacheId of CacheDataRow: " + cacheId); + + GridQueryProcessor qryProcessor = ignite.context().query(); + + try { + Method m = GridQueryProcessor.class.getDeclaredMethod("typeByValue", String.class, + CacheObjectContext.class, KeyCacheObject.class, CacheObject.class, boolean.class); + + m.setAccessible(true); + + QueryTypeDescriptorImpl res = (QueryTypeDescriptorImpl)m.invoke( + qryProcessor, cacheCtx.name(), cacheCtx.cacheObjectContext(), row.key(), row.value(), true); + + if (res == null) + continue; // Tolerate - (k, v) is just not indexed. + + IgniteH2Indexing indexing = (IgniteH2Indexing)qryProcessor.getIndexing(); + + GridH2Table gridH2Tbl = indexing.dataTable(cacheCtx.name(), res.tableName()); + + if (gridH2Tbl == null) + continue; // Tolerate - (k, v) is just not indexed. + + GridH2RowDescriptor gridH2RowDesc = gridH2Tbl.rowDescriptor(); + + GridH2Row h2Row = gridH2RowDesc.createRow(row); + + ArrayList indexes = gridH2Tbl.getIndexes(); + + for (Index idx : indexes) { + try { + Cursor cursor = idx.find((Session) null, h2Row, h2Row); + + if (cursor == null || !cursor.next()) + throw new IgniteCheckedException("Key not found."); + } + catch (Throwable t) { + Object o = CacheObjectUtils.unwrapBinaryIfNeeded( + grpCtx.cacheObjectContext(), row.key(), true, true); + + IndexValidationIssue is = new IndexValidationIssue( + o.toString(), cacheCtx.name(), idx.getName(), t); + + log.error("Failed to lookup key: " + is.toString()); + + enoughIssues |= partRes.reportIssue(is); + } + } + } + catch (IllegalAccessException | NoSuchMethodException e) { + log.error("Failed to invoke typeByValue", e); + + throw new IgniteException(e); + } + catch (InvocationTargetException e) { + Throwable target = e.getTargetException(); + + log.error("Failed to invoke typeByValue", target); + + throw new IgniteException(target); + } + finally { + keysProcessed++; + + if (U.currentTimeMillis() - lastProgressLog >= 60_000 && partSize > 0) { + log.warning("Processing partition " + part.id() + " (" + (keysProcessed * 100 / partSize) + + "% " + keysProcessed + "/" + partSize + ")"); + + lastProgressLog = U.currentTimeMillis(); + } + } + } + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to process partition [grpId=" + grpCtx.groupId() + + ", partId=" + part.id() + "]", e); + + return Collections.emptyMap(); + } + finally { + part.release(); + } + + PartitionKey partKey = new PartitionKey(grpCtx.groupId(), part.id(), grpCtx.cacheOrGroupName()); + + completionCntr.incrementAndGet(); + + return Collections.singletonMap(partKey, partRes); + } +} diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTask.java b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTask.java new file mode 100644 index 0000000000000..1a89c2cadb57e --- /dev/null +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTask.java @@ -0,0 +1,99 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.ignite.internal.visor.verify; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.IgniteException; +import org.apache.ignite.compute.ComputeJobResult; +import org.apache.ignite.internal.processors.cache.verify.PartitionKey; +import org.apache.ignite.internal.processors.task.GridInternal; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.visor.VisorJob; +import org.apache.ignite.internal.visor.VisorMultiNodeTask; +import org.jetbrains.annotations.Nullable; + +/** + * + */ +@GridInternal +public class VisorValidateIndexesTask extends VisorMultiNodeTask { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Nullable @Override protected VisorValidateIndexesTaskResult reduce0(List list) throws IgniteException { + Map exceptions = new HashMap<>(); + Map jobResults = new HashMap<>(); + + for (ComputeJobResult res : list) { + if (res.getException() != null) + exceptions.put(res.getNode().id(), res.getException()); + else + jobResults.put(res.getNode().id(), res.getData()); + } + + return new VisorValidateIndexesTaskResult(jobResults, exceptions); + } + + /** {@inheritDoc} */ + @Override protected VisorJob job(VisorValidateIndexesTaskArg arg) { + return new VisorValidateIndexesJob(arg, debug); + } + + /** + * + */ + private static class VisorValidateIndexesJob extends VisorJob { + /** */ + private static final long serialVersionUID = 0L; + + /** + * @param arg Argument. + * @param debug Debug. + */ + protected VisorValidateIndexesJob(@Nullable VisorValidateIndexesTaskArg arg, boolean debug) { + super(arg, debug); + } + + /** {@inheritDoc} */ + @Override protected VisorValidateIndexesJobResult run(@Nullable VisorValidateIndexesTaskArg arg) throws IgniteException { + try { + ValidateIndexesClosure clo = new ValidateIndexesClosure(arg.getCaches()); + + ignite.context().resource().injectGeneric(clo); + + Map res = clo.call(); + + return new VisorValidateIndexesJobResult(res); + + } + catch (Exception e) { + throw new IgniteException(e); + } + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorValidateIndexesJob.class, this); + } + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java b/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java new file mode 100644 index 0000000000000..9e9c7770a644c --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java @@ -0,0 +1,121 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.ignite.util; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.cache.QueryIndex; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.util.typedef.F; + +import static org.apache.ignite.internal.commandline.CommandHandler.EXIT_CODE_OK; + +/** + * + */ +public class GridCommandHandlerIndexingTest extends GridCommandHandlerTest { + /** + * + */ + public void testValidateIndexes() throws Exception { + Ignite ignite = startGrids(2); + + ignite.cluster().active(true); + + Ignite client = startGrid("client"); + + IgniteCache personCache = client.getOrCreateCache(new CacheConfiguration() + .setName("persons-cache-vi") + .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC) + .setAtomicityMode(CacheAtomicityMode.ATOMIC) + .setBackups(1) + .setQueryEntities(F.asList(personEntity(true, true))) + .setAffinity(new RendezvousAffinityFunction(false, 32))); + + ThreadLocalRandom rand = ThreadLocalRandom.current(); + + for (int i = 0; i < 1000; i++) + personCache.put(i, new Person(rand.nextInt(), String.valueOf(rand.nextLong()))); + + injectTestSystemOut(); + + assertEquals(EXIT_CODE_OK, execute("--cache", "validate_indexes", "persons-cache-vi")); + + assertTrue(testOut.toString().contains("validate_indexes has finished, no issues found")); + } + + /** + * @param idxName Index name. + * @param idxOrgId Index org id. + */ + private QueryEntity personEntity(boolean idxName, boolean idxOrgId) { + QueryEntity entity = new QueryEntity(); + + entity.setKeyType(Integer.class.getName()); + entity.setValueType(Person.class.getName()); + + entity.addQueryField("orgId", Integer.class.getName(), null); + entity.addQueryField("name", String.class.getName(), null); + + List idxs = new ArrayList<>(); + + if (idxName) { + QueryIndex idx = new QueryIndex("name"); + + idxs.add(idx); + } + + if (idxOrgId) { + QueryIndex idx = new QueryIndex("orgId"); + + idxs.add(idx); + } + + entity.setIndexes(idxs); + + return entity; + } + + /** + * + */ + private static class Person implements Serializable { + /** */ + int orgId; + + /** */ + String name; + + /** + * @param orgId Organization ID. + * @param name Name. + */ + public Person(int orgId, String name) { + this.orgId = orgId; + this.name = name; + } + } +} From d1ecf732d1d998afbb526dcf16c2314cebcb82a5 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Thu, 26 Apr 2018 19:38:03 +0300 Subject: [PATCH 0135/1463] IGNITE-8404 Fixed NPE in MappedFileMemoryProvider --- .../mem/file/MappedFileMemoryProvider.java | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/mem/file/MappedFileMemoryProvider.java b/modules/core/src/main/java/org/apache/ignite/internal/mem/file/MappedFileMemoryProvider.java index 3800214b90f7f..7186b27f08118 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/mem/file/MappedFileMemoryProvider.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/mem/file/MappedFileMemoryProvider.java @@ -94,14 +94,18 @@ public MappedFileMemoryProvider(IgniteLogger log, File allocationPath) { /** {@inheritDoc} */ @Override public void shutdown() { - for (MappedFile file : mappedFiles) { - try { - file.close(); - } - catch (IOException e) { - log.error("Failed to close memory-mapped file upon stop (will ignore) [file=" + - file.file() + ", err=" + e.getMessage() + ']'); + if (mappedFiles != null) { + for (MappedFile file : mappedFiles) { + try { + file.close(); + } + catch (IOException e) { + log.error("Failed to close memory-mapped file upon stop (will ignore) [file=" + + file.file() + ", err=" + e.getMessage() + ']'); + } } + + mappedFiles = null; } } From 6987800de568dac16ad0d6318413900733a95bde Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Fri, 27 Apr 2018 13:08:03 +0300 Subject: [PATCH 0136/1463] IGNITE-8408 Fixed IgniteUtils.invoke --- .../java/org/apache/ignite/internal/util/IgniteUtils.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index 42e96fb6067f0..2defefad4721e 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 @@ -8143,10 +8143,10 @@ public static T invoke(@Nullable Class cls, @Nullable Object obj, String assert mtdName != null; try { - for (Class c = cls != null ? cls : obj.getClass(); cls != Object.class; cls = cls.getSuperclass()) { + for (cls = cls != null ? cls : obj.getClass(); cls != Object.class; cls = cls.getSuperclass()) { Method mtd = null; - for (Method declaredMtd : c.getDeclaredMethods()) { + for (Method declaredMtd : cls.getDeclaredMethods()) { if (declaredMtd.getName().equals(mtdName)) { if (mtd == null) mtd = declaredMtd; @@ -8202,11 +8202,11 @@ public static T invoke(@Nullable Class cls, @Nullable Object obj, String assert mtdName != null; try { - for (Class c = cls != null ? cls : obj.getClass(); cls != Object.class; cls = cls.getSuperclass()) { + for (cls = cls != null ? cls : obj.getClass(); cls != Object.class; cls = cls.getSuperclass()) { Method mtd; try { - mtd = c.getDeclaredMethod(mtdName, paramTypes); + mtd = cls.getDeclaredMethod(mtdName, paramTypes); } catch (NoSuchMethodException ignored) { continue; From 5ec1796408d45296cbb26bb93353e23e18b039c8 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Fri, 27 Apr 2018 15:06:08 +0300 Subject: [PATCH 0137/1463] IGNITE-8393 Unexpected error during WAL compression fixed Signed-off-by: Andrey Gura --- .../GridCacheDatabaseSharedManager.java | 6 +- .../wal/FileWriteAheadLogManager.java | 53 +++++++--- .../FsyncModeFileWriteAheadLogManager.java | 59 +++++++---- .../persistence/db/wal/WalCompactionTest.java | 99 ++++++++++++++++++- 4 files changed, 180 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 42c7090a857f8..d847132611e5f 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 @@ -747,8 +747,10 @@ private void unRegistrateMetricsMBean() { WALPointer restore = restoreMemory(status); - if (restore == null && status.endPtr != CheckpointStatus.NULL_PTR) - throw new StorageException("Restore wal pointer = " + restore + ", while status.endPtr = " + status.endPtr + "."); + if (restore == null && status.endPtr != CheckpointStatus.NULL_PTR) { + throw new StorageException("Restore wal pointer = " + restore + ", while status.endPtr = " + + status.endPtr + ". Can't restore memory - critical part of WAL archive is missing."); + } // First, bring memory to the last consistent checkpoint state if needed. // This method should return a pointer to the last valid record in the WAL. 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 a28b73b9ab2a0..779534489a3bc 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 @@ -39,8 +39,10 @@ import java.util.Arrays; import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.TreeSet; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.PriorityBlockingQueue; @@ -1869,17 +1871,28 @@ 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() { - FileDescriptor[] descs = scan(walArchiveDir.listFiles(WAL_SEGMENT_FILE_FILTER)); + FileDescriptor[] descs = scan(walArchiveDir.listFiles(WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER)); + + Set indices = new HashSet<>(); + Set duplicateIndices = new HashSet<>(); + + for (FileDescriptor desc : descs) { + if (!indices.add(desc.idx)) + duplicateIndices.add(desc.idx); + } for (FileDescriptor desc : descs) { + if (desc.isCompressed()) + continue; + // Do not delete reserved or locked segment and any segment after it. if (segmentReservedOrLocked(desc.idx)) return; - if (desc.idx < lastCompressedIdx) { + if (desc.idx < lastCompressedIdx && duplicateIndices.contains(desc.idx)) { if (!desc.file.delete()) U.warn(log, "Failed to remove obsolete WAL segment (make sure the process has enough rights): " + desc.file.getAbsolutePath() + ", exists: " + desc.file.exists()); @@ -1892,22 +1905,24 @@ private void deleteObsoleteRawSegments() { init(); while (!Thread.currentThread().isInterrupted() && !stopped) { + long currReservedSegment = -1; + try { deleteObsoleteRawSegments(); - long nextSegment = tryReserveNextSegmentOrWait(); - if (nextSegment == -1) + currReservedSegment = tryReserveNextSegmentOrWait(); + if (currReservedSegment == -1) continue; - File tmpZip = new File(walArchiveDir, FileDescriptor.fileName(nextSegment) + ".zip" + ".tmp"); + File tmpZip = new File(walArchiveDir, FileDescriptor.fileName(currReservedSegment) + ".zip" + ".tmp"); - File zip = new File(walArchiveDir, FileDescriptor.fileName(nextSegment) + ".zip"); + File zip = new File(walArchiveDir, FileDescriptor.fileName(currReservedSegment) + ".zip"); - File raw = new File(walArchiveDir, FileDescriptor.fileName(nextSegment)); + File raw = new File(walArchiveDir, FileDescriptor.fileName(currReservedSegment)); if (!Files.exists(raw.toPath())) throw new IgniteCheckedException("WAL archive segment is missing: " + raw); - compressSegmentToFile(nextSegment, raw, tmpZip); + compressSegmentToFile(currReservedSegment, raw, tmpZip); Files.move(tmpZip.toPath(), zip.toPath()); @@ -1917,14 +1932,27 @@ private void deleteObsoleteRawSegments() { } } - lastCompressedIdx = nextSegment; + lastCompressedIdx = currReservedSegment; } catch (IgniteCheckedException | IOException e) { - U.error(log, "Unexpected error during WAL compression", e); + U.error(log, "Compression of WAL segment [idx=" + currReservedSegment + + "] was skipped due to unexpected error", e); + + lastCompressedIdx++; } catch (InterruptedException ignore) { Thread.currentThread().interrupt(); } + finally { + try { + if (currReservedSegment != -1) + release(new FileWALPointer(currReservedSegment, 0, 0)); + } + catch (IgniteCheckedException e) { + U.error(log, "Can't release raw WAL segment [idx=" + currReservedSegment + + "] after compression", e); + } + } } } @@ -1977,9 +2005,6 @@ private void compressSegmentToFile(long nextSegment, File raw, File zip) zos.write(heapBuf.array()); } - finally { - release(new FileWALPointer(nextSegment, 0, 0)); - } } /** 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 e354b439ef2b0..dfb1c41e7378a 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 @@ -34,9 +34,11 @@ import java.util.Arrays; import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.NavigableMap; +import java.util.Set; import java.util.TreeMap; import java.util.TreeSet; import java.util.concurrent.PriorityBlockingQueue; @@ -1703,19 +1705,30 @@ 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() { - FileDescriptor[] descs = scan(walArchiveDir.listFiles(WAL_SEGMENT_FILE_FILTER)); + FsyncModeFileWriteAheadLogManager.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) { + if (!indices.add(desc.idx)) + duplicateIndices.add(desc.idx); + } FileArchiver archiver0 = archiver; - for (FileDescriptor desc : descs) { + for (FsyncModeFileWriteAheadLogManager.FileDescriptor desc : descs) { + if (desc.isCompressed()) + continue; + // Do not delete reserved or locked segment and any segment after it. if (archiver0 != null && archiver0.reserved(desc.idx)) return; - if (desc.idx < lastCompressedIdx) { + if (desc.idx < lastCompressedIdx && duplicateIndices.contains(desc.idx)) { if (!desc.file.delete()) U.warn(log, "Failed to remove obsolete WAL segment (make sure the process has enough rights): " + desc.file.getAbsolutePath() + ", exists: " + desc.file.exists()); @@ -1728,39 +1741,54 @@ private void deleteObsoleteRawSegments() { init(); while (!Thread.currentThread().isInterrupted() && !stopped) { + long currReservedSegment = -1; + try { deleteObsoleteRawSegments(); - long nextSegment = tryReserveNextSegmentOrWait(); - if (nextSegment == -1) + currReservedSegment = tryReserveNextSegmentOrWait(); + if (currReservedSegment == -1) continue; - File tmpZip = new File(walArchiveDir, FileDescriptor.fileName(nextSegment) + ".zip" + ".tmp"); + File tmpZip = new File(walArchiveDir, FileWriteAheadLogManager.FileDescriptor.fileName(currReservedSegment) + ".zip" + ".tmp"); - File zip = new File(walArchiveDir, FileDescriptor.fileName(nextSegment) + ".zip"); + File zip = new File(walArchiveDir, FileWriteAheadLogManager.FileDescriptor.fileName(currReservedSegment) + ".zip"); - File raw = new File(walArchiveDir, FileDescriptor.fileName(nextSegment)); + File raw = new File(walArchiveDir, FileWriteAheadLogManager.FileDescriptor.fileName(currReservedSegment)); if (!Files.exists(raw.toPath())) throw new IgniteCheckedException("WAL archive segment is missing: " + raw); - compressSegmentToFile(nextSegment, raw, tmpZip); + compressSegmentToFile(currReservedSegment, raw, tmpZip); Files.move(tmpZip.toPath(), zip.toPath()); - if (mode == WALMode.FSYNC) { + if (mode != WALMode.NONE) { try (FileIO f0 = ioFactory.create(zip, CREATE, READ, WRITE)) { f0.force(); } } - lastCompressedIdx = nextSegment; + lastCompressedIdx = currReservedSegment; } catch (IgniteCheckedException | IOException e) { - U.error(log, "Unexpected error during WAL compression", e); + U.error(log, "Compression of WAL segment [idx=" + currReservedSegment + + "] was skipped due to unexpected error", e); + + lastCompressedIdx++; } - catch (InterruptedException e) { + catch (InterruptedException ignore) { Thread.currentThread().interrupt(); } + finally { + try { + if (currReservedSegment != -1) + release(new FileWALPointer(currReservedSegment, 0, 0)); + } + catch (IgniteCheckedException e) { + U.error(log, "Can't release raw WAL segment [idx=" + currReservedSegment + + "] after compression", e); + } + } } } @@ -1804,9 +1832,6 @@ private void compressSegmentToFile(long nextSegment, File raw, File zip) iter.nextX(); } } - finally { - release(new FileWALPointer(nextSegment, 0, 0)); - } } /** 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 6b79d906a4937..fe0169f70d908 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 @@ -18,6 +18,7 @@ import java.io.File; import java.io.FilenameFilter; +import java.io.RandomAccessFile; import java.util.Arrays; import java.util.Comparator; import org.apache.ignite.IgniteCache; @@ -57,6 +58,12 @@ public class WalCompactionTest extends GridCommonAbstractTest { /** Entries count. */ public static final int ENTRIES = 1000; + /** Compaction enabled flag. */ + private boolean compactionEnabled; + + /** Wal mode. */ + private WALMode walMode; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String name) throws Exception { IgniteConfiguration cfg = super.getConfiguration(name); @@ -67,10 +74,10 @@ public class WalCompactionTest extends GridCommonAbstractTest { .setDefaultDataRegionConfiguration(new DataRegionConfiguration() .setPersistenceEnabled(true) .setMaxSize(200 * 1024 * 1024)) - .setWalMode(WALMode.LOG_ONLY) + .setWalMode(walMode) .setWalSegmentSize(WAL_SEGMENT_SIZE) .setWalHistorySize(500) - .setWalCompactionEnabled(true)); + .setWalCompactionEnabled(compactionEnabled)); CacheConfiguration ccfg = new CacheConfiguration(); @@ -91,6 +98,10 @@ public class WalCompactionTest extends GridCommonAbstractTest { stopAllGrids(); cleanPersistenceDir(); + + compactionEnabled = true; + + walMode = WALMode.LOG_ONLY; } /** {@inheritDoc} */ @@ -125,7 +136,7 @@ public void testApplyingUpdatesFromCompactedWal() throws Exception { ig.context().cache().context().database().wakeupForCheckpoint("Forced checkpoint").get(); ig.context().cache().context().database().wakeupForCheckpoint("Forced checkpoint").get(); - Thread.sleep(15_000); // Allow compressor to archive WAL segments. + Thread.sleep(15_000); // Allow compressor to compress WAL segments. String nodeFolderName = ig.context().pdsFolderResolver().resolveFolders().folderName(); @@ -187,6 +198,86 @@ else if (arr[i] != 1) { assertFalse(fail); } + /** + * + */ + public void testCompressorToleratesEmptyWalSegmentsFsync() throws Exception { + testCompressorToleratesEmptyWalSegments(WALMode.FSYNC); + } + + /** + * + */ + public void testCompressorToleratesEmptyWalSegmentsLogOnly() throws Exception { + testCompressorToleratesEmptyWalSegments(WALMode.LOG_ONLY); + } + + /** + * Tests that WAL compaction won't be stopped by single broken WAL segment. + */ + private void testCompressorToleratesEmptyWalSegments(WALMode walMode) throws Exception { + this.walMode = walMode; + compactionEnabled = false; + + IgniteEx ig = startGrid(0); + ig.cluster().active(true); + + IgniteCache cache = ig.cache("cache"); + + for (int i = 0; i < 2500; i++) { // At least 50MB of raw data in total. + final byte[] val = new byte[20000]; + + val[i] = 1; + + cache.put(i, val); + } + + // WAL archive segment is allowed to be compressed when it's at least one checkpoint away from current WAL head. + ig.context().cache().context().database().wakeupForCheckpoint("Forced checkpoint").get(); + ig.context().cache().context().database().wakeupForCheckpoint("Forced checkpoint").get(); + + String nodeFolderName = ig.context().pdsFolderResolver().resolveFolders().folderName(); + + stopAllGrids(); + + int emptyIdx = 5; + + File dbDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false); + 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)); + + try (RandomAccessFile raf = new RandomAccessFile(walSegment, "rw")) { + raf.setLength(0); // Clear wal segment, but don't delete. + } + + compactionEnabled = true; + + ig = startGrid(0); + ig.cluster().active(true); + + Thread.sleep(15_000); // Allow compressor to compress WAL segments. + + File[] compressedSegments = nodeArchiveDir.listFiles(new FilenameFilter() { + @Override public boolean accept(File dir, String name) { + return name.endsWith(".wal.zip"); + } + }); + + long maxIdx = -1; + for (File f : compressedSegments) { + String idxPart = f.getName().substring(0, f.getName().length() - ".wal.zip".length()); + + maxIdx = Math.max(maxIdx, Long.parseLong(idxPart)); + } + + System.out.println("Max compressed index: " + maxIdx); + assertTrue(maxIdx > emptyIdx); + + assertTrue(walSegment.exists()); // Failed to compress WAL segment shoudn't be deleted. + } + /** * @throws Exception If failed. */ @@ -240,7 +331,7 @@ public void testSeekingStartInCompactedSegment() throws Exception { ig.context().cache().context().database().wakeupForCheckpoint("Forced checkpoint").get(); ig.context().cache().context().database().wakeupForCheckpoint("Forced checkpoint").get(); - Thread.sleep(15_000); // Allow compressor to archive WAL segments. + Thread.sleep(15_000); // Allow compressor to compress WAL segments. File walDir = new File(dbDir, "wal"); File archiveDir = new File(walDir, "archive"); From 2679cc5a18eb00599c9f00b0f46dbcaa6352c0cb Mon Sep 17 00:00:00 2001 From: Akmal Chaudhri Date: Fri, 27 Apr 2018 16:41:30 +0300 Subject: [PATCH 0138/1463] IGNITE-7909: Java examples for Spark Data Frames. - Fixes #3883. Signed-off-by: Nikolay Izhikov --- .../spark/JavaIgniteCatalogExample.java | 143 ++++++++++++++ .../spark/JavaIgniteDataFrameExample.java | 154 +++++++++++++++ .../JavaIgniteDataFrameWriteExample.java | 185 ++++++++++++++++++ .../examples/JavaIgniteDataFrameSelfTest.java | 54 +++++ .../IgniteExamplesSparkSelfTestSuite.java | 2 + 5 files changed, 538 insertions(+) create mode 100644 examples/src/main/spark/org/apache/ignite/examples/spark/JavaIgniteCatalogExample.java create mode 100644 examples/src/main/spark/org/apache/ignite/examples/spark/JavaIgniteDataFrameExample.java create mode 100644 examples/src/main/spark/org/apache/ignite/examples/spark/JavaIgniteDataFrameWriteExample.java create mode 100644 examples/src/test/spark/org/apache/ignite/spark/examples/JavaIgniteDataFrameSelfTest.java diff --git a/examples/src/main/spark/org/apache/ignite/examples/spark/JavaIgniteCatalogExample.java b/examples/src/main/spark/org/apache/ignite/examples/spark/JavaIgniteCatalogExample.java new file mode 100644 index 0000000000000..c9313f6792b31 --- /dev/null +++ b/examples/src/main/spark/org/apache/ignite/examples/spark/JavaIgniteCatalogExample.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.examples.spark; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.log4j.Level; +import org.apache.log4j.Logger; +import org.apache.spark.sql.AnalysisException; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.ignite.IgniteSparkSession; + +import static org.apache.ignite.internal.util.typedef.X.println; + +/** + * + */ +public class JavaIgniteCatalogExample { + /** + * Ignite config file. + */ + private static final String CONFIG = "examples/config/example-ignite.xml"; + + /** + * Test cache name. + */ + private static final String CACHE_NAME = "testCache"; + + /** */ + public static void main(String args[]) throws AnalysisException { + + setupServerAndData(); + + //Creating Ignite-specific implementation of Spark session. + IgniteSparkSession igniteSession = IgniteSparkSession.builder() + .appName("Spark Ignite catalog example") + .master("local") + .config("spark.executor.instances", "2") + .igniteConfig(CONFIG) + .getOrCreate(); + + //Adjust the logger to exclude the logs of no interest. + Logger.getRootLogger().setLevel(Level.ERROR); + Logger.getLogger("org.apache.ignite").setLevel(Level.INFO); + + System.out.println("List of available tables:"); + + //Showing existing tables. + igniteSession.catalog().listTables().show(); + + System.out.println("PERSON table description:"); + + //Showing `person` schema. + igniteSession.catalog().listColumns("person").show(); + + System.out.println("CITY table description:"); + + //Showing `city` schema. + igniteSession.catalog().listColumns("city").show(); + + println("Querying all persons from city with ID=2."); + + //Selecting data through Spark SQL engine. + Dataset df = igniteSession.sql("SELECT * FROM person WHERE CITY_ID = 2"); + + System.out.println("Result schema:"); + + df.printSchema(); + + System.out.println("Result content:"); + + df.show(); + + System.out.println("Querying all persons living in Denver."); + + //Selecting data through Spark SQL engine. + Dataset df2 = igniteSession.sql("SELECT * FROM person p JOIN city c ON c.ID = p.CITY_ID WHERE c.NAME = 'Denver'"); + + System.out.println("Result schema:"); + + df2.printSchema(); + + System.out.println("Result content:"); + + df2.show(); + + Ignition.stop(false); + } + + /** */ + private static void setupServerAndData() { + //Starting Ignite. + Ignite ignite = Ignition.start(CONFIG); + + //Creating cache. + CacheConfiguration ccfg = new CacheConfiguration<>(CACHE_NAME).setSqlSchema("PUBLIC"); + + IgniteCache cache = ignite.getOrCreateCache(ccfg); + + //Create tables. + cache.query(new SqlFieldsQuery( + "CREATE TABLE city (id LONG PRIMARY KEY, name VARCHAR) WITH \"template=replicated\"")).getAll(); + + cache.query(new SqlFieldsQuery( + "CREATE TABLE person (id LONG, name VARCHAR, city_id LONG, PRIMARY KEY (id, city_id)) " + + "WITH \"backups=1, affinityKey=city_id\"")).getAll(); + + cache.query(new SqlFieldsQuery("CREATE INDEX on Person (city_id)")).getAll(); + + //Inserting some data into table. + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO city (id, name) VALUES (?, ?)"); + + cache.query(qry.setArgs(1L, "Forest Hill")).getAll(); + cache.query(qry.setArgs(2L, "Denver")).getAll(); + cache.query(qry.setArgs(3L, "St. Petersburg")).getAll(); + + qry = new SqlFieldsQuery("INSERT INTO person (id, name, city_id) values (?, ?, ?)"); + + cache.query(qry.setArgs(1L, "John Doe", 3L)).getAll(); + cache.query(qry.setArgs(2L, "Jane Roe", 2L)).getAll(); + cache.query(qry.setArgs(3L, "Mary Major", 1L)).getAll(); + cache.query(qry.setArgs(4L, "Richard Miles", 2L)).getAll(); + } +} diff --git a/examples/src/main/spark/org/apache/ignite/examples/spark/JavaIgniteDataFrameExample.java b/examples/src/main/spark/org/apache/ignite/examples/spark/JavaIgniteDataFrameExample.java new file mode 100644 index 0000000000000..20bcf83e2d4d4 --- /dev/null +++ b/examples/src/main/spark/org/apache/ignite/examples/spark/JavaIgniteDataFrameExample.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.spark; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.spark.IgniteDataFrameSettings; +import org.apache.log4j.Level; +import org.apache.log4j.Logger; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; + +import static org.apache.spark.sql.functions.col; + +/** + * + */ +public class JavaIgniteDataFrameExample { + /** + * Ignite config file. + */ + private static final String CONFIG = "examples/config/example-ignite.xml"; + + /** + * Test cache name. + */ + private static final String CACHE_NAME = "testCache"; + + /** */ + public static void main(String args[]) { + + setupServerAndData(); + + //Creating spark session. + SparkSession spark = SparkSession + .builder() + .appName("JavaIgniteDataFrameExample") + .master("local") + .config("spark.executor.instances", "2") + .getOrCreate(); + + // Adjust the logger to exclude the logs of no interest. + Logger.getRootLogger().setLevel(Level.ERROR); + Logger.getLogger("org.apache.ignite").setLevel(Level.INFO); + + // Executing examples. + + sparkDSLExample(spark); + + nativeSparkSqlExample(spark); + + Ignition.stop(false); + } + + /** */ + private static void sparkDSLExample(SparkSession spark) { + System.out.println("Querying using Spark DSL."); + + Dataset igniteDF = spark.read() + .format(IgniteDataFrameSettings.FORMAT_IGNITE()) //Data source type. + .option(IgniteDataFrameSettings.OPTION_TABLE(), "person") //Table to read. + .option(IgniteDataFrameSettings.OPTION_CONFIG_FILE(), CONFIG) //Ignite config. + .load() + .filter(col("id").geq(2)) //Filter clause. + .filter(col("name").like("%M%")); //Another filter clause. + + System.out.println("Data frame schema:"); + + igniteDF.printSchema(); //Printing query schema to console. + + System.out.println("Data frame content:"); + + igniteDF.show(); //Printing query results to console. + } + + /** */ + private static void nativeSparkSqlExample(SparkSession spark) { + System.out.println("Querying using Spark SQL."); + + Dataset df = spark.read() + .format(IgniteDataFrameSettings.FORMAT_IGNITE()) //Data source type. + .option(IgniteDataFrameSettings.OPTION_TABLE(), "person") //Table to read. + .option(IgniteDataFrameSettings.OPTION_CONFIG_FILE(), CONFIG) //Ignite config. + .load(); + + //Registering DataFrame as Spark view. + df.createOrReplaceTempView("person"); + + //Selecting data from Ignite through Spark SQL Engine. + Dataset igniteDF = spark.sql("SELECT * FROM person WHERE id >= 2 AND name = 'Mary Major'"); + + System.out.println("Result schema:"); + + igniteDF.printSchema(); //Printing query schema to console. + + System.out.println("Result content:"); + + igniteDF.show(); //Printing query results to console. + } + + /** */ + private static void setupServerAndData() { + //Starting Ignite. + Ignite ignite = Ignition.start(CONFIG); + + //Creating first test cache. + CacheConfiguration ccfg = new CacheConfiguration<>(CACHE_NAME).setSqlSchema("PUBLIC"); + + IgniteCache cache = ignite.getOrCreateCache(ccfg); + + //Creating SQL tables. + cache.query(new SqlFieldsQuery( + "CREATE TABLE city (id LONG PRIMARY KEY, name VARCHAR) WITH \"template=replicated\"")).getAll(); + + cache.query(new SqlFieldsQuery( + "CREATE TABLE person (id LONG, name VARCHAR, city_id LONG, PRIMARY KEY (id, city_id)) " + + "WITH \"backups=1, affinity_key=city_id\"")).getAll(); + + cache.query(new SqlFieldsQuery("CREATE INDEX on Person (city_id)")).getAll(); + + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO city (id, name) VALUES (?, ?)"); + + //Inserting some data to tables. + cache.query(qry.setArgs(1L, "Forest Hill")).getAll(); + cache.query(qry.setArgs(2L, "Denver")).getAll(); + cache.query(qry.setArgs(3L, "St. Petersburg")).getAll(); + + qry = new SqlFieldsQuery("INSERT INTO person (id, name, city_id) values (?, ?, ?)"); + + cache.query(qry.setArgs(1L, "John Doe", 3L)).getAll(); + cache.query(qry.setArgs(2L, "Jane Roe", 2L)).getAll(); + cache.query(qry.setArgs(3L, "Mary Major", 1L)).getAll(); + cache.query(qry.setArgs(4L, "Richard Miles", 2L)).getAll(); + } +} diff --git a/examples/src/main/spark/org/apache/ignite/examples/spark/JavaIgniteDataFrameWriteExample.java b/examples/src/main/spark/org/apache/ignite/examples/spark/JavaIgniteDataFrameWriteExample.java new file mode 100644 index 0000000000000..6fc1393dc0812 --- /dev/null +++ b/examples/src/main/spark/org/apache/ignite/examples/spark/JavaIgniteDataFrameWriteExample.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.examples.spark; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.spark.IgniteDataFrameSettings; +import org.apache.log4j.Level; +import org.apache.log4j.Logger; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; + +import java.util.List; + +import static org.apache.ignite.internal.util.IgniteUtils.resolveIgnitePath; +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.reverse; + +/** + * + */ +public class JavaIgniteDataFrameWriteExample { + /** + * Ignite config file. + */ + private static final String CONFIG = "examples/config/example-ignite.xml"; + + /** + * Test cache name. + */ + private static final String CACHE_NAME = "testCache"; + + /** */ + public static void main(String args[]) { + //Starting Ignite. + Ignite ignite = Ignition.start(CONFIG); + + //Starting Ignite server node. + setupServerAndData(ignite); + + //Creating spark session. + SparkSession spark = SparkSession + .builder() + .appName("Spark Ignite data sources write example") + .master("local") + .config("spark.executor.instances", "2") + .getOrCreate(); + + // Adjust the logger to exclude the logs of no interest. + Logger.getRootLogger().setLevel(Level.ERROR); + Logger.getLogger("org.apache.ignite").setLevel(Level.INFO); + + // Executing examples. + System.out.println("Example of writing json file to Ignite:"); + + writeJSonToIgnite(ignite, spark); + + System.out.println("Example of modifying existing Ignite table data through Data Fram API:"); + + editDataAndSaveToNewTable(ignite, spark); + + Ignition.stop(false); + } + + /** */ + private static void writeJSonToIgnite(Ignite ignite, SparkSession spark) { + //Load content of json file to data frame. + Dataset personsDataFrame = spark.read().json( + resolveIgnitePath("examples/src/main/resources/person.json").getAbsolutePath()); + + System.out.println("Json file content:"); + + //Printing content of json file to console. + personsDataFrame.show(); + + System.out.println("Writing Data Frame to Ignite:"); + + //Writing content of data frame to Ignite. + personsDataFrame.write() + .format(IgniteDataFrameSettings.FORMAT_IGNITE()) + .option(IgniteDataFrameSettings.OPTION_CONFIG_FILE(), CONFIG) + .option(IgniteDataFrameSettings.OPTION_TABLE(), "json_person") + .option(IgniteDataFrameSettings.OPTION_CREATE_TABLE_PRIMARY_KEY_FIELDS(), "id") + .option(IgniteDataFrameSettings.OPTION_CREATE_TABLE_PARAMETERS(), "template=replicated") + .save(); + + System.out.println("Done!"); + + System.out.println("Reading data from Ignite table:"); + + CacheConfiguration ccfg = new CacheConfiguration<>(CACHE_NAME); + + IgniteCache cache = ignite.getOrCreateCache(ccfg); + + //Reading saved data from Ignite. + List> data = cache.query(new SqlFieldsQuery("SELECT id, name, department FROM json_person")).getAll(); + + System.out.println(data); + } + + /** */ + private static void editDataAndSaveToNewTable(Ignite ignite, SparkSession spark) { + //Load content of Ignite table to data frame. + Dataset personDataFrame = spark.read() + .format(IgniteDataFrameSettings.FORMAT_IGNITE()) + .option(IgniteDataFrameSettings.OPTION_CONFIG_FILE(), CONFIG) + .option(IgniteDataFrameSettings.OPTION_TABLE(), "person") + .load(); + + System.out.println("Data frame content:"); + + //Printing content of data frame to console. + personDataFrame.show(); + + System.out.println("Modifying Data Frame and write it to Ignite:"); + + personDataFrame + .withColumn("id", col("id").plus(42)) //Edit id column + .withColumn("name", reverse(col("name"))) //Edit name column + .write().format(IgniteDataFrameSettings.FORMAT_IGNITE()) + .option(IgniteDataFrameSettings.OPTION_CONFIG_FILE(), CONFIG) + .option(IgniteDataFrameSettings.OPTION_TABLE(), "new_persons") + .option(IgniteDataFrameSettings.OPTION_CREATE_TABLE_PRIMARY_KEY_FIELDS(), "id, city_id") + .option(IgniteDataFrameSettings.OPTION_CREATE_TABLE_PARAMETERS(), "backups=1") + .mode(SaveMode.Overwrite) //Overwriting entire table. + .save(); + + System.out.println("Done!"); + + System.out.println("Reading data from Ignite table:"); + + CacheConfiguration ccfg = new CacheConfiguration<>(CACHE_NAME); + + IgniteCache cache = ignite.getOrCreateCache(ccfg); + + //Reading saved data from Ignite. + List> data = cache.query(new SqlFieldsQuery("SELECT id, name, city_id FROM new_persons")).getAll(); + + System.out.println(data); + } + + /** */ + private static void setupServerAndData(Ignite ignite) { + //Creating first test cache. + CacheConfiguration ccfg = new CacheConfiguration<>(CACHE_NAME).setSqlSchema("PUBLIC"); + + IgniteCache cache = ignite.getOrCreateCache(ccfg); + + //Creating SQL table. + cache.query(new SqlFieldsQuery( + "CREATE TABLE person (id LONG, name VARCHAR, city_id LONG, PRIMARY KEY (id)) " + + "WITH \"backups=1\"")).getAll(); + + cache.query(new SqlFieldsQuery("CREATE INDEX on Person (city_id)")).getAll(); + + //Inserting some data to tables. + SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO person (id, name, city_id) values (?, ?, ?)"); + + cache.query(qry.setArgs(1L, "John Doe", 3L)).getAll(); + cache.query(qry.setArgs(2L, "Jane Roe", 2L)).getAll(); + cache.query(qry.setArgs(3L, "Mary Major", 1L)).getAll(); + cache.query(qry.setArgs(4L, "Richard Miles", 2L)).getAll(); + } +} diff --git a/examples/src/test/spark/org/apache/ignite/spark/examples/JavaIgniteDataFrameSelfTest.java b/examples/src/test/spark/org/apache/ignite/spark/examples/JavaIgniteDataFrameSelfTest.java new file mode 100644 index 0000000000000..295814d96383d --- /dev/null +++ b/examples/src/test/spark/org/apache/ignite/spark/examples/JavaIgniteDataFrameSelfTest.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.spark.examples; + +import org.apache.ignite.examples.spark.JavaIgniteCatalogExample; +import org.apache.ignite.examples.spark.JavaIgniteDataFrameExample; +import org.apache.ignite.examples.spark.JavaIgniteDataFrameWriteExample; +import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest; +import org.junit.Test; + +/** + */ +public class JavaIgniteDataFrameSelfTest extends GridAbstractExamplesTest { + static final String[] EMPTY_ARGS = new String[0]; + + /** + * @throws Exception If failed. + */ + @Test + public void testCatalogExample() throws Exception { + JavaIgniteCatalogExample.main(EMPTY_ARGS); + } + + /** + * @throws Exception If failed. + */ + @Test + public void testDataFrameExample() throws Exception { + JavaIgniteDataFrameExample.main(EMPTY_ARGS); + } + + /** + * @throws Exception If failed. + */ + @Test + public void testDataFrameWriteExample() throws Exception { + JavaIgniteDataFrameWriteExample.main(EMPTY_ARGS); + } +} diff --git a/examples/src/test/spark/org/apache/ignite/spark/testsuites/IgniteExamplesSparkSelfTestSuite.java b/examples/src/test/spark/org/apache/ignite/spark/testsuites/IgniteExamplesSparkSelfTestSuite.java index df1a243e3f067..6328ee241a190 100644 --- a/examples/src/test/spark/org/apache/ignite/spark/testsuites/IgniteExamplesSparkSelfTestSuite.java +++ b/examples/src/test/spark/org/apache/ignite/spark/testsuites/IgniteExamplesSparkSelfTestSuite.java @@ -19,6 +19,7 @@ import junit.framework.TestSuite; import org.apache.ignite.spark.examples.IgniteDataFrameSelfTest; +import org.apache.ignite.spark.examples.JavaIgniteDataFrameSelfTest; import org.apache.ignite.spark.examples.SharedRDDExampleSelfTest; import org.apache.ignite.testframework.GridTestUtils; @@ -42,6 +43,7 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(SharedRDDExampleSelfTest.class)); suite.addTest(new TestSuite(IgniteDataFrameSelfTest.class)); + suite.addTest(new TestSuite(JavaIgniteDataFrameSelfTest.class)); return suite; } From ef7d4255b36c8ed60d4bd3372bd84e61a1f76cf6 Mon Sep 17 00:00:00 2001 From: Alexey Kukushkin Date: Thu, 26 Apr 2018 19:31:43 +0300 Subject: [PATCH 0139/1463] IGNITE-8237 Ignite blocks on SecurityException in exchange-worker due to unauthorised on-heap cache configuration. - Fixes #3818. Signed-off-by: dpavlov (cherry picked from commit 54cb262) --- .../processors/cache/GridCacheProcessor.java | 52 ++++++++++++------- 1 file changed, 33 insertions(+), 19 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 39c7e715c06f6..898380cb5f7e6 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 @@ -123,6 +123,7 @@ import org.apache.ignite.internal.processors.query.schema.SchemaNodeLeaveExchangeWorkerTask; import org.apache.ignite.internal.processors.query.schema.message.SchemaAbstractDiscoveryMessage; import org.apache.ignite.internal.processors.query.schema.message.SchemaProposeDiscoveryMessage; +import org.apache.ignite.internal.processors.security.SecurityContext; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions; import org.apache.ignite.internal.util.F0; @@ -1176,9 +1177,6 @@ private void startCache(GridCacheAdapter cache, QuerySchema schema) throws CacheConfiguration cfg = cacheCtx.config(); - if (cacheCtx.userCache()) - authorizeCacheCreate(cacheCtx.name(), cfg); - // Intentionally compare Boolean references using '!=' below to check if the flag has been explicitly set. if (cfg.isStoreKeepBinary() && cfg.isStoreKeepBinary() != CacheConfiguration.DFLT_STORE_KEEP_BINARY && !(ctx.config().getMarshaller() instanceof BinaryMarshaller)) @@ -2513,6 +2511,23 @@ private GridCacheSharedContext createSharedContext(GridKernalContext kernalCtx, StringBuilder errorMessage = new StringBuilder(); for (CacheJoinNodeDiscoveryData.CacheInfo cacheInfo : nodeData.caches().values()) { + try { + byte[] secCtxBytes = node.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2); + + if (secCtxBytes != null) { + SecurityContext secCtx = U.unmarshal(marsh, secCtxBytes, U.resolveClassLoader(ctx.config())); + + if (secCtx != null && cacheInfo.cacheType() == CacheType.USER) + authorizeCacheCreate(cacheInfo.cacheData().config(), secCtx); + } + } + catch (SecurityException | IgniteCheckedException ex) { + if (errorMessage.length() > 0) + errorMessage.append("\n"); + + errorMessage.append(ex.getMessage()); + } + DynamicCacheDescriptor localDesc = cacheDescriptor(cacheInfo.cacheData().config().getName()); if (localDesc == null) @@ -3363,30 +3378,29 @@ private Collection initiateCacheChanges( } /** - * Authorize dynamic cache management. + * Authorize creating cache. + */ + private void authorizeCacheCreate(CacheConfiguration cfg, SecurityContext secCtx) { + ctx.security().authorize(null, SecurityPermission.CACHE_CREATE, secCtx); + + if (cfg != null && cfg.isOnheapCacheEnabled() && + IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_DISABLE_ONHEAP_CACHE)) + throw new SecurityException("Authorization failed for enabling on-heap cache."); + } + + /** + * Authorize dynamic cache management for this node. */ private void authorizeCacheChange(DynamicCacheChangeRequest req) { + // Null security context means authorize this node. if (req.cacheType() == null || req.cacheType() == CacheType.USER) { if (req.stop()) - ctx.security().authorize(req.cacheName(), SecurityPermission.CACHE_DESTROY, null); + ctx.security().authorize(null, SecurityPermission.CACHE_DESTROY, null); else - authorizeCacheCreate(req.cacheName(), req.startCacheConfiguration()); + authorizeCacheCreate(req.startCacheConfiguration(), null); } } - /** - * Authorize start/create cache operation. - */ - private void authorizeCacheCreate(String cacheName, CacheConfiguration cacheCfg) { - ctx.security().authorize(cacheName, SecurityPermission.CACHE_CREATE, null); - - if (cacheCfg != null && cacheCfg.isOnheapCacheEnabled() && - System.getProperty(IgniteSystemProperties.IGNITE_DISABLE_ONHEAP_CACHE, "false") - .toUpperCase().equals("TRUE") - ) - throw new SecurityException("Authorization failed for enabling on-heap cache."); - } - /** * @return Non null exception if node is stopping or disconnected. */ From 504a13a1cda5a7c6b8cbd612a5d3da76167a238b Mon Sep 17 00:00:00 2001 From: dpavlov Date: Thu, 26 Apr 2018 19:38:05 +0300 Subject: [PATCH 0140/1463] IGNITE-8237 Javadoc for method parameters added. (cherry picked from commit ebe55e3) --- .../ignite/internal/processors/cache/GridCacheProcessor.java | 3 +++ 1 file changed, 3 insertions(+) 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 898380cb5f7e6..ccccdec536cff 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 @@ -3379,6 +3379,8 @@ private Collection initiateCacheChanges( /** * Authorize creating cache. + * @param cfg Cache configuration. + * @param secCtx Optional security context. */ private void authorizeCacheCreate(CacheConfiguration cfg, SecurityContext secCtx) { ctx.security().authorize(null, SecurityPermission.CACHE_CREATE, secCtx); @@ -3390,6 +3392,7 @@ private void authorizeCacheCreate(CacheConfiguration cfg, SecurityContext secCtx /** * Authorize dynamic cache management for this node. + * @param req start/stop cache request. */ private void authorizeCacheChange(DynamicCacheChangeRequest req) { // Null security context means authorize this node. From d3140981c42d636904f8ef088e06fda219e9fc15 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Fri, 27 Apr 2018 22:15:15 +0300 Subject: [PATCH 0141/1463] IGNITE-8416 CommandHandlerParsingTest stably fails with parsing error Signed-off-by: Andrey Gura --- .../internal/commandline/CommandHandlerParsingTest.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java index fda7e6054ffbc..98b8e01c99ea3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java @@ -40,6 +40,9 @@ public void testParseAndValidateUserAndPassword() { CommandHandler hnd = new CommandHandler(); for (Command cmd : Command.values()) { + if (cmd == Command.CACHE) + continue; // --cache subcommand requires its own specific arguments. + try { hnd.parseAndValidate(asList("--user")); @@ -91,6 +94,9 @@ public void testConnectionSettings() { CommandHandler hnd = new CommandHandler(); for (Command cmd : Command.values()) { + if (cmd == Command.CACHE) + continue; // --cache subcommand requires its own specific arguments. + Arguments args = hnd.parseAndValidate(asList(cmd.text())); assertEquals(cmd, args.command()); @@ -98,7 +104,7 @@ public void testConnectionSettings() { assertEquals(DFLT_PORT, args.port()); args = hnd.parseAndValidate(asList("--port", "12345", "--host", "test-host", "--ping-interval", "5000", - "--ping-timeout", "40000", cmd.text())); + "--ping-timeout", "40000", cmd.text())); assertEquals(cmd, args.command()); assertEquals("test-host", args.host()); From 0061e7c79f5c7137f3be0a5100b50189c21aa6cd Mon Sep 17 00:00:00 2001 From: EdShangGG Date: Sat, 28 Apr 2018 19:27:27 +0300 Subject: [PATCH 0142/1463] IGNITE-7628 SqlQuery hangs indefinitely with additional not registered in baseline node. Signed-off-by: Andrey Gura --- .../discovery/GridDiscoveryManager.java | 52 +++-- .../near/IgniteSqlQueryWithBaselineTest.java | 184 ++++++++++++++++++ .../IgniteCacheQuerySelfTestSuite2.java | 2 + 3 files changed, 221 insertions(+), 17 deletions(-) create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteSqlQueryWithBaselineTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java index 261e73db97786..15badf27a071b 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 @@ -367,7 +367,7 @@ public void cleanCachesAndGroups() { */ public void addCacheGroup(CacheGroupDescriptor grpDesc, IgnitePredicate filter, CacheMode cacheMode) { CacheGroupAffinity old = registeredCacheGrps.put(grpDesc.groupId(), - new CacheGroupAffinity(grpDesc.cacheOrGroupName(), filter, cacheMode)); + new CacheGroupAffinity(grpDesc.cacheOrGroupName(), filter, cacheMode, grpDesc.persistenceEnabled())); assert old == null : old; } @@ -2387,12 +2387,6 @@ else if (node.version().compareTo(minVer) < 0) assert !rmtNodes.contains(loc) : "Remote nodes collection shouldn't contain local node" + " [rmtNodes=" + rmtNodes + ", loc=" + loc + ']'; - Map> allCacheNodes = U.newHashMap(allNodes.size()); - Map> cacheGrpAffNodes = U.newHashMap(allNodes.size()); - Set rmtNodesWithCaches = new TreeSet<>(NodeOrderComparator.getInstance()); - - fillAffinityNodeCaches(allNodes, allCacheNodes, cacheGrpAffNodes, rmtNodesWithCaches); - BaselineTopology blt = state.baselineTopology(); if (blt != null) { @@ -2435,6 +2429,13 @@ else if (node.version().compareTo(minVer) < 0) baselineNodes = null; } + Map> allCacheNodes = U.newHashMap(allNodes.size()); + Map> cacheGrpAffNodes = U.newHashMap(allNodes.size()); + Set rmtNodesWithCaches = new TreeSet<>(NodeOrderComparator.getInstance()); + + fillAffinityNodeCaches(allNodes, allCacheNodes, cacheGrpAffNodes, rmtNodesWithCaches, + nodeIdToConsIdx == null ? null : nodeIdToConsIdx.keySet()); + return new DiscoCache( topVer, state, @@ -3127,23 +3128,29 @@ private static class CacheGroupAffinity { /** Cache mode. */ private final CacheMode cacheMode; + /** Persistent cache group or not. */ + private final boolean persistentCacheGrp; + /** * @param name Name. * @param cacheFilter Node filter. * @param cacheMode Cache mode. + * @param persistentCacheGrp Persistence is configured for cache or not. */ CacheGroupAffinity( - String name, - IgnitePredicate cacheFilter, - CacheMode cacheMode) { + String name, + IgnitePredicate cacheFilter, + CacheMode cacheMode, + boolean persistentCacheGrp) { this.name = name; this.cacheFilter = cacheFilter; this.cacheMode = cacheMode; + this.persistentCacheGrp = persistentCacheGrp; } /** {@inheritDoc} */ @Override public String toString() { - return "CacheGroupAffinity [name=" + name + ']'; + return S.toString(CacheGroupAffinity.class, this); } } @@ -3268,14 +3275,19 @@ private Boolean cacheClientNode(ClusterNode node) { /** * Fills affinity node caches. - * * @param allNodes All nodes. * @param allCacheNodes All cache nodes. * @param cacheGrpAffNodes Cache group aff nodes. * @param rmtNodesWithCaches Rmt nodes with caches. - */ - private void fillAffinityNodeCaches(List allNodes, Map> allCacheNodes, - Map> cacheGrpAffNodes, Set rmtNodesWithCaches) { + * @param bltNodes Baseline node ids. + */ + private void fillAffinityNodeCaches( + List allNodes, + Map> allCacheNodes, + Map> cacheGrpAffNodes, + Set rmtNodesWithCaches, + Set bltNodes + ) { for (ClusterNode node : allNodes) { assert node.order() != 0 : "Invalid node order [locNode=" + localNode() + ", node=" + node + ']'; assert !node.isDaemon(); @@ -3285,6 +3297,9 @@ private void fillAffinityNodeCaches(List allNodes, Map nodes = cacheGrpAffNodes.get(grpId); if (nodes == null) @@ -3324,7 +3339,10 @@ public DiscoCache createDiscoCacheOnCacheChange( Map> cacheGrpAffNodes = U.newHashMap(allNodes.size()); Set rmtNodesWithCaches = new TreeSet<>(NodeOrderComparator.getInstance()); - fillAffinityNodeCaches(allNodes, allCacheNodes, cacheGrpAffNodes, rmtNodesWithCaches); + Map nodeIdToConsIdx = discoCache.nodeIdToConsIdx; + + fillAffinityNodeCaches(allNodes, allCacheNodes, cacheGrpAffNodes, rmtNodesWithCaches, + nodeIdToConsIdx == null ? null : nodeIdToConsIdx.keySet()); return new DiscoCache( topVer, @@ -3340,7 +3358,7 @@ public DiscoCache createDiscoCacheOnCacheChange( cacheGrpAffNodes, discoCache.nodeMap, discoCache.alives, - discoCache.nodeIdToConsIdx, + nodeIdToConsIdx, discoCache.consIdxToNodeId, discoCache.minimumNodeVersion(), discoCache.minimumServerNodeVersion()); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteSqlQueryWithBaselineTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteSqlQueryWithBaselineTest.java new file mode 100644 index 0000000000000..203a319a3c6b7 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteSqlQueryWithBaselineTest.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.io.Serializable; +import java.util.Collection; + +import org.apache.ignite.IgniteCache; +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.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import javax.cache.Cache; + +/** + * + */ +public class IgniteSqlQueryWithBaselineTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setDataStorageConfiguration( + new DataStorageConfiguration() + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration() + .setMaxSize(200 * 1024 * 1024) + .setPersistenceEnabled(true) + ) + ); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + + disco.setIpFinder(IP_FINDER); + + cfg.setDiscoverySpi(disco); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** */ + public static class C1 implements Serializable { + /** */ + private static final long serialVersionUID = 1L; + + /** */ + public C1(int id) { + this.id = id; + } + + /** */ + @QuerySqlField(index = true) + protected Integer id; + } + + /** */ + public static class C2 implements Serializable { + /** */ + private static final long serialVersionUID = 1L; + + /** */ + C2(int id) { + this.id = id; + } + + /** */ + @QuerySqlField(index = true) + protected Integer id; + } + + /** + * @throws Exception If failed. + */ + public void testQueryWithNodeNotInBLT() throws Exception { + startGrids(2); + + grid(0).cluster().active(true); + + startGrid(2); //Start extra node. + + doQuery(); + } + + /** + * @throws Exception If failed. + */ + public void testQueryWithoutBLTNode() throws Exception { + startGrids(2); + + grid(0).cluster().active(true); + + startGrid(2); //Start extra node. + stopGrid(1); + + doQuery(); + } + + /** + * @throws Exception If failed. + */ + public void testQueryFromNotBLTNode() throws Exception { + startGrid(1); + + grid(1).cluster().active(true); + + startGrid(0); //Start extra node. + + doQuery(); + } + + /** + * + */ + private void doQuery() { + CacheConfiguration c1Conf = new CacheConfiguration<>("C1"); + c1Conf.setIndexedTypes(Integer.class, C1.class).setBackups(2); + + CacheConfiguration c2Conf = new CacheConfiguration<>("C2"); + c2Conf.setIndexedTypes(Integer.class, C2.class).setBackups(2); + + final IgniteCache cache = grid(0).getOrCreateCache(c1Conf); + + final IgniteCache cache1 = grid(0).getOrCreateCache(c2Conf); + + for (int i = 0; i < 100; i++) { + cache.put(i, new C1(i)); + + cache1.put(i, new C2(i)); + } + + String sql = "SELECT C1.*" + + " from C1 inner join \"C2\".C2 as D on C1.id = D.id" + + " order by C1.id asc"; + + SqlQuery qry = new SqlQuery<>(C1.class, sql); + + qry.setDistributedJoins(true); + + log.info("before query run..."); + + Collection> res = cache.query(qry).getAll(); + + log.info("result size: " + res.size()); + } +} \ No newline at end of file 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 11d98e2c451e8..5b888ce7c65f8 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 @@ -33,6 +33,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest2; import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartTxSelfTest; import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.near.IgniteSqlQueryWithBaselineTest; import org.apache.ignite.internal.processors.cache.index.DynamicColumnsConcurrentAtomicPartitionedSelfTest; import org.apache.ignite.internal.processors.cache.index.DynamicColumnsConcurrentAtomicReplicatedSelfTest; import org.apache.ignite.internal.processors.cache.index.DynamicColumnsConcurrentTransactionalPartitionedSelfTest; @@ -87,6 +88,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteCacheClientQueryReplicatedNodeRestartSelfTest.class); suite.addTestSuite(IgniteCacheQueryNodeFailTest.class); suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class); + suite.addTestSuite(IgniteSqlQueryWithBaselineTest.class); suite.addTestSuite(IgniteChangingBaselineCacheQueryNodeRestartSelfTest.class); suite.addTestSuite(IgniteStableBaselineCacheQueryNodeRestartsSelfTest.class); suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest2.class); From 8f86a1d7a63552ed6b620643162f5a70da7b414f Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Thu, 3 May 2018 17:01:20 +0300 Subject: [PATCH 0143/1463] IGNITE-8226 Improved logging - Fixes #3796. Signed-off-by: Alexey Goncharuk --- .../dht/GridClientPartitionTopology.java | 6 ++---- .../dht/GridDhtPartitionTopologyImpl.java | 20 +++++++++++++------ .../dht/preloader/GridDhtPartitionMap.java | 5 +++++ 3 files changed, 21 insertions(+), 10 deletions(-) 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 cf8fc34ac4a05..477d316b6914d 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 @@ -853,16 +853,14 @@ private boolean shouldOverridePartitionMap(GridDhtPartitionMap currentMap, GridD /** * Method checks is new partition map more stale than current partition map - * New partition map is stale if topology version or update sequence are less than of current map + * New partition map is stale if topology version or update sequence are less or equal than of current map * * @param currentMap Current partition map * @param newMap New partition map * @return True if new partition map is more stale than current partition map, false in other case */ private boolean isStaleUpdate(GridDhtPartitionMap currentMap, GridDhtPartitionMap newMap) { - return currentMap != null && - (newMap.topologyVersion().compareTo(currentMap.topologyVersion()) < 0 || - newMap.topologyVersion().compareTo(currentMap.topologyVersion()) == 0 && newMap.updateSequence() <= currentMap.updateSequence()); + return currentMap != null && newMap.compareTo(currentMap) <= 0; } /** {@inheritDoc} */ 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 fba40b7203aa4..0a08fe1dd44b2 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 @@ -1632,16 +1632,14 @@ else if (part.updateCounter() > 0) { /** * Method checks is new partition map more stale than current partition map - * New partition map is stale if topology version or update sequence are less than of current map + * New partition map is stale if topology version or update sequence are less or equal than of current map * * @param currentMap Current partition map * @param newMap New partition map * @return True if new partition map is more stale than current partition map, false in other case */ private boolean isStaleUpdate(GridDhtPartitionMap currentMap, GridDhtPartitionMap newMap) { - return currentMap != null && - (newMap.topologyVersion().compareTo(currentMap.topologyVersion()) < 0 || - newMap.topologyVersion().compareTo(currentMap.topologyVersion()) == 0 && newMap.updateSequence() <= currentMap.updateSequence()); + return currentMap != null && newMap.compareTo(currentMap) <= 0; } /** {@inheritDoc} */ @@ -1697,11 +1695,21 @@ private boolean isStaleUpdate(GridDhtPartitionMap currentMap, GridDhtPartitionMa parts.updateSequence(cur.updateSequence(), cur.topologyVersion()); } else if (isStaleUpdate(cur, parts)) { - U.warn(log, "Stale update for single partition map update (will ignore) [" + + assert cur != null; + + String msg = "Stale update for single partition map update (will ignore) [" + "grp=" + grp.cacheOrGroupName() + ", exchId=" + exchId + ", curMap=" + cur + - ", newMap=" + parts + ']'); + ", newMap=" + parts + ']'; + + // This is usual situation when partition maps are equal, just print debug message. + if (cur.compareTo(parts) == 0) { + if (log.isDebugEnabled()) + log.debug(msg); + } + else + U.warn(log, msg); return false; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java index cb697149639fa..28c8c8453faad 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java @@ -243,6 +243,11 @@ public AffinityTopologyVersion topologyVersion() { @Override public int compareTo(GridDhtPartitionMap o) { assert nodeId.equals(o.nodeId); + int topVerCompare = top.compareTo(o.top); + + if (topVerCompare != 0) + return topVerCompare; + return Long.compare(updateSeq, o.updateSeq); } From b40cd3dd3da68de268271f7a56d64a61ef4c2480 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Thu, 3 May 2018 17:05:40 +0300 Subject: [PATCH 0144/1463] IGNITE-8405 Update partition owners during exchange in 1 operation. - Fixes #3929. Signed-off-by: Alexey Goncharuk --- .../dht/GridClientPartitionTopology.java | 67 ++++++++++----- .../dht/GridDhtPartitionTopology.java | 15 ++-- .../dht/GridDhtPartitionTopologyImpl.java | 83 ++++++++++++------- .../GridDhtPartitionsExchangeFuture.java | 21 ++--- .../util/tostring/GridToStringBuilder.java | 65 +++++++++++++-- ...aselineCacheQueryNodeRestartsSelfTest.java | 2 +- 6 files changed, 174 insertions(+), 79 deletions(-) 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 477d316b6914d..b3652331b210b 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 @@ -28,6 +28,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.stream.Collectors; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cluster.ClusterNode; @@ -50,15 +51,14 @@ import org.apache.ignite.internal.util.GridPartitionStateMap; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.EVICTED; import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.MOVING; import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.OWNING; -import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.RENTING; /** * Partition topology for node which does not have any local partitions. @@ -1138,39 +1138,62 @@ private void removeNode(UUID nodeId) { } /** {@inheritDoc} */ - @Override public Set setOwners(int p, Set owners, boolean haveHistory, boolean updateSeq) { - Set result = haveHistory ? Collections.emptySet() : new HashSet(); + @Override public Map> resetOwners(Map> ownersByUpdCounters, Set haveHistory) { + Map> result = new HashMap<>(); lock.writeLock().lock(); try { - for (Map.Entry e : node2part.entrySet()) { - GridDhtPartitionMap partMap = e.getValue(); - UUID remoteNodeId = e.getKey(); + // Process remote partitions. + for (Map.Entry> entry : ownersByUpdCounters.entrySet()) { + int part = entry.getKey(); + Set newOwners = entry.getValue(); - if (!partMap.containsKey(p)) - continue; + for (Map.Entry remotes : node2part.entrySet()) { + UUID remoteNodeId = remotes.getKey(); + GridDhtPartitionMap partMap = remotes.getValue(); - if (partMap.get(p) == OWNING && !owners.contains(remoteNodeId)) { - partMap.put(p, MOVING); + GridDhtPartitionState state = partMap.get(part); + + if (state == null || state != OWNING) + continue; - if (!haveHistory) - result.add(remoteNodeId); + if (!newOwners.contains(remoteNodeId)) { + partMap.put(part, MOVING); - partMap.updateSequence(partMap.updateSequence() + 1, partMap.topologyVersion()); + partMap.updateSequence(partMap.updateSequence() + 1, partMap.topologyVersion()); - U.warn(log, "Partition has been scheduled for rebalancing due to outdated update counter " + - "[nodeId=" + remoteNodeId + ", groupId=" + grpId + - ", partId=" + p + ", haveHistory=" + haveHistory + "]"); + result.computeIfAbsent(remoteNodeId, n -> new HashSet<>()); + result.get(remoteNodeId).add(part); + } } } - part2node.put(p, owners); + for (Map.Entry> entry : result.entrySet()) { + UUID nodeId = entry.getKey(); + Set partsToRebalance = entry.getValue(); - if (updateSeq) - this.updateSeq.incrementAndGet(); - } - finally { + if (!partsToRebalance.isEmpty()) { + Set historical = partsToRebalance.stream() + .filter(haveHistory::contains) + .collect(Collectors.toSet()); + + // Filter out partitions having WAL history. + partsToRebalance.removeAll(historical); + + U.warn(log, "Partitions have been scheduled for rebalancing due to outdated update counter " + + "[grpId=" + grpId + + ", nodeId=" + nodeId + + ", partsFull=" + S.compact(partsToRebalance) + + ", partsHistorical=" + S.compact(historical) + "]"); + } + } + + for (Map.Entry> entry : ownersByUpdCounters.entrySet()) + part2node.put(entry.getKey(), entry.getValue()); + + updateSeq.incrementAndGet(); + } finally { lock.writeLock().unlock(); } 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 2df2e8960afc9..d6c54506e1208 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 @@ -389,16 +389,15 @@ public boolean update(@Nullable GridDhtPartitionExchangeId exchId, public boolean rebalanceFinished(AffinityTopologyVersion topVer); /** - * Make nodes from provided set owners for a given partition. - * State of all current owners that aren't contained in the set will be reset to MOVING. + * Calculates nodes and partitions which have non-actual state and must be rebalanced. + * State of all current owners that aren't contained in the given {@code ownersByUpdCounters} will be reset to MOVING. * - * @param p Partition ID. - * @param owners Set of new owners. - * @param haveHistory {@code True} if there is WAL history to rebalance given partition. - * @param updateSeq If should increment sequence when updated. - * @return Set of node IDs that should reload partitions. + * @param ownersByUpdCounters Map (partition, set of node IDs that have most actual state about partition + * (update counter is maximal) and should hold OWNING state for such partition). + * @param haveHistory Set of partitions which have WAL history to rebalance. + * @return Map (nodeId, set of partitions that should be rebalanced fully by this node). */ - public Set setOwners(int p, Set owners, boolean haveHistory, boolean updateSeq); + public Map> resetOwners(Map> ownersByUpdCounters, Set haveHistory); /** * Callback on exchange done. 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 0a08fe1dd44b2..aa32902a4ccdd 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 @@ -30,6 +30,7 @@ import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReferenceArray; +import java.util.stream.Collectors; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; @@ -60,6 +61,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.LT; +import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -2076,8 +2078,8 @@ else if (plc != PartitionLossPolicy.IGNORE) { } /** {@inheritDoc} */ - @Override public Set setOwners(int p, Set ownersByUpdCounters, boolean haveHistory, boolean updateSeq) { - Set result = haveHistory ? Collections.emptySet() : new HashSet(); + @Override public Map> resetOwners(Map> ownersByUpdCounters, Set haveHistory) { + Map> result = new HashMap<>(); ctx.database().checkpointReadLock(); @@ -2085,50 +2087,73 @@ else if (plc != PartitionLossPolicy.IGNORE) { lock.writeLock().lock(); try { - GridDhtLocalPartition locPart = locParts.get(p); + // First process local partitions. + for (Map.Entry> entry : ownersByUpdCounters.entrySet()) { + int part = entry.getKey(); + Set newOwners = entry.getValue(); + + GridDhtLocalPartition locPart = localPartition(part); - if (locPart != null) { - if (locPart.state() == OWNING && !ownersByUpdCounters.contains(ctx.localNodeId())) { - rebalancePartition(p, haveHistory); + if (locPart == null || locPart.state() != OWNING) + continue; - if (!haveHistory) - result.add(ctx.localNodeId()); + if (!newOwners.contains(ctx.localNodeId())) { + rebalancePartition(part, haveHistory.contains(part)); - U.warn(log, "Partition has been scheduled for rebalancing due to outdated update counter " + - "[nodeId=" + ctx.localNodeId() + ", grp=" + grp.cacheOrGroupName() + - ", partId=" + p + ", haveHistory=" + haveHistory + "]"); + result.computeIfAbsent(ctx.localNodeId(), n -> new HashSet<>()); + result.get(ctx.localNodeId()).add(part); } } - for (Map.Entry e : node2part.entrySet()) { - UUID remoteNodeId = e.getKey(); - GridDhtPartitionMap partMap = e.getValue(); + // Then process remote partitions. + for (Map.Entry> entry : ownersByUpdCounters.entrySet()) { + int part = entry.getKey(); + Set newOwners = entry.getValue(); - if (!partMap.containsKey(p)) - continue; + for (Map.Entry remotes : node2part.entrySet()) { + UUID remoteNodeId = remotes.getKey(); + GridDhtPartitionMap partMap = remotes.getValue(); + + GridDhtPartitionState state = partMap.get(part); - if (partMap.get(p) == OWNING && !ownersByUpdCounters.contains(remoteNodeId)) { - partMap.put(p, MOVING); + if (state == null || state != OWNING) + continue; - if (!haveHistory) - result.add(remoteNodeId); + if (!newOwners.contains(remoteNodeId)) { + partMap.put(part, MOVING); - partMap.updateSequence(partMap.updateSequence() + 1, partMap.topologyVersion()); + partMap.updateSequence(partMap.updateSequence() + 1, partMap.topologyVersion()); - if (partMap.nodeId().equals(ctx.localNodeId())) - this.updateSeq.setIfGreater(partMap.updateSequence()); + if (partMap.nodeId().equals(ctx.localNodeId())) + updateSeq.setIfGreater(partMap.updateSequence()); - U.warn(log, "Partition has been scheduled for rebalancing due to outdated update counter " + - "[nodeId=" + remoteNodeId + ", grp=" + grp.cacheOrGroupName() + - ", partId=" + p + ", haveHistory=" + haveHistory + "]"); + result.computeIfAbsent(remoteNodeId, n -> new HashSet<>()); + result.get(remoteNodeId).add(part); + } } } - if (updateSeq) { - long updSeq = this.updateSeq.incrementAndGet(); + for (Map.Entry> entry : result.entrySet()) { + UUID nodeId = entry.getKey(); + Set rebalancedParts = entry.getValue(); + + if (!rebalancedParts.isEmpty()) { + Set historical = rebalancedParts.stream() + .filter(haveHistory::contains) + .collect(Collectors.toSet()); - node2part = new GridDhtPartitionFullMap(node2part, updSeq); + // Filter out partitions having WAL history. + rebalancedParts.removeAll(historical); + + U.warn(log, "Partitions have been scheduled for rebalancing due to outdated update counter " + + "[grp=" + grp.cacheOrGroupName() + + ", nodeId=" + nodeId + + ", partsFull=" + S.compact(rebalancedParts) + + ", partsHistorical=" + S.compact(historical) + "]"); + } } + + node2part = new GridDhtPartitionFullMap(node2part, updateSeq.incrementAndGet()); } finally { lock.writeLock().unlock(); } 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 33bd98950973c..39f4ed11cfcf2 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 @@ -59,7 +59,6 @@ import org.apache.ignite.internal.managers.discovery.DiscoCache; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.pagemem.wal.record.ExchangeRecord; -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.CacheAffinityChangeMessage; @@ -73,7 +72,6 @@ import org.apache.ignite.internal.processors.cache.ExchangeDiscoveryEvents; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; -import org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.LocalJoinCachesContext; import org.apache.ignite.internal.processors.cache.StateChangeRequest; @@ -2375,19 +2373,18 @@ else if (cntr == maxCntr.cnt) } } - for (Map.Entry e : maxCntrs.entrySet()) { - int p = e.getKey(); - long maxCntr = e.getValue().cnt; - - entryLeft--; + Map> ownersByUpdCounters = new HashMap<>(maxCntrs.size()); + for (Map.Entry e : maxCntrs.entrySet()) + ownersByUpdCounters.put(e.getKey(), e.getValue().nodes); - if (entryLeft != 0 && maxCntr == 0) - continue; + Map> partitionsToRebalance = top.resetOwners(ownersByUpdCounters, haveHistory); - Set nodesToReload = top.setOwners(p, e.getValue().nodes, haveHistory.contains(p), entryLeft == 0); + for (Map.Entry> e : partitionsToRebalance.entrySet()) { + UUID nodeId = e.getKey(); + Set parts = e.getValue(); - for (UUID nodeId : nodesToReload) - partsToReload.put(nodeId, top.groupId(), p); + for (int part : parts) + partsToReload.put(nodeId, top.groupId(), part); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java index 56eef1df14813..d36111263dd01 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java @@ -17,32 +17,36 @@ package org.apache.ignite.internal.util.tostring; -import org.apache.ignite.IgniteException; -import org.apache.ignite.IgniteSystemProperties; -import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.internal.U; -import org.jetbrains.annotations.Nullable; - import java.io.Externalizable; import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; import java.lang.reflect.Field; import java.lang.reflect.Modifier; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.EventListener; import java.util.HashMap; import java.util.LinkedList; +import java.util.List; import java.util.Map; import java.util.Queue; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteSystemProperties; +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.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; -import static org.apache.ignite.IgniteSystemProperties.IGNITE_TO_STRING_INCLUDE_SENSITIVE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_TO_STRING_COLLECTION_LIMIT; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_TO_STRING_INCLUDE_SENSITIVE; /** * Provides auto-generation framework for {@code toString()} output. @@ -1818,4 +1822,51 @@ else if (!f.isAnnotationPresent(GridToStringExclude.class) && return cd; } + + /** + * Returns sorted and compacted string representation of given {@code col}. + * Two nearby numbers with difference at most 1 are compacted to one continuous segment. + * E.g. collection of [1, 2, 3, 5, 6, 7, 10] will be compacted to [1-3, 5-7, 10]. + * + * @param col Collection of integers. + * @return Compacted string representation of given collections. + */ + public static String compact(@NotNull Collection col) { + if (col.isEmpty()) + return "[]"; + + SB sb = new SB(); + sb.a('['); + + List l = new ArrayList<>(col); + Collections.sort(l); + + int left = l.get(0), right = left; + for (int i = 1; i < l.size(); i++) { + int val = l.get(i); + + if (right == val || right + 1 == val) { + right = val; + continue; + } + + if (left == right) + sb.a(left); + else + sb.a(left).a('-').a(right); + + sb.a(',').a(' '); + + left = right = val; + } + + if (left == right) + sb.a(left); + else + sb.a(left).a('-').a(right); + + sb.a(']'); + + return sb.toString(); + } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteStableBaselineCacheQueryNodeRestartsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteStableBaselineCacheQueryNodeRestartsSelfTest.java index 81d803884e374..3248906739bf4 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteStableBaselineCacheQueryNodeRestartsSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/baseline/IgniteStableBaselineCacheQueryNodeRestartsSelfTest.java @@ -50,7 +50,7 @@ public class IgniteStableBaselineCacheQueryNodeRestartsSelfTest extends IgniteCa initStoreStrategy(); - grid(0).active(true); + grid(0).cluster().active(true); stopGrid(gridCount()); From cad48a6266c6c119932f351bb4a4a64e2295494b Mon Sep 17 00:00:00 2001 From: Denis Mekhanikov Date: Fri, 20 Apr 2018 17:11:36 +0300 Subject: [PATCH 0145/1463] ignite-8205 Clear list of local services in GridServiceProcessor#onKernalStop Signed-off-by: Andrey Gura (cherry picked from commit fbe24f8e3b0d9016a69670ca2bc50766865adf38) --- .../service/GridServiceProcessor.java | 8 +- .../ServiceDeploymentOnActivationTest.java | 244 ++++++++++++++++++ .../testsuites/IgniteKernalSelfTestSuite.java | 2 + 3 files changed, 252 insertions(+), 2 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/service/ServiceDeploymentOnActivationTest.java 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 63f50273c88ae..9cf27d28c98ee 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 @@ -29,6 +29,7 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; @@ -107,7 +108,6 @@ import org.apache.ignite.thread.OomExceptionHandler; import org.apache.ignite.transactions.Transaction; import org.jetbrains.annotations.Nullable; -import java.util.concurrent.ConcurrentHashMap; import static org.apache.ignite.IgniteSystemProperties.IGNITE_SERVICES_COMPATIBILITY_MODE; import static org.apache.ignite.IgniteSystemProperties.getString; @@ -245,11 +245,12 @@ private void onKernalStart0() throws IgniteCheckedException { ctx.cache().context().deploy().ignoreOwnership(true); if (!ctx.clientNode() && serviceCache.context().affinityNode()) { + // Register query listener and run it for local entries. It is also invoked on rebalancing. serviceCache.context().continuousQueries().executeInternalQuery( new ServiceEntriesListener(), null, true, true, false ); } - else { + else { // Listener for client nodes is registered in onContinuousProcessorStarted method. assert !ctx.isDaemon(); ctx.closure().runLocalSafe(new Runnable() { @@ -324,6 +325,8 @@ private IgniteInternalCache serviceCache() { synchronized (locSvcs) { for (Collection ctxs0 : locSvcs.values()) ctxs.addAll(ctxs0); + + locSvcs.clear(); } for (ServiceContextImpl ctx : ctxs) { @@ -1627,6 +1630,7 @@ private void processDeployment(CacheEntryEvent CLIENT_FILTER = new IgnitePredicate() { + @Override public boolean apply(ClusterNode node) { + return node.isClient(); + } + }; + + /** */ + private boolean client; + + /** */ + private boolean persistence; + + /** */ + private ServiceConfiguration srvcCfg; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + TcpDiscoverySpi discoverySpi = new TcpDiscoverySpi(); + discoverySpi.setIpFinder(IP_FINDER); + cfg.setDiscoverySpi(discoverySpi); + + cfg.setClientMode(client); + + if (srvcCfg != null) + cfg.setServiceConfiguration(srvcCfg); + + if (persistence) { + cfg.setDataStorageConfiguration( + new DataStorageConfiguration() + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration() + .setPersistenceEnabled(true) + .setMaxSize(10 * 1024 * 1024) + ).setWalMode(WALMode.LOG_ONLY) + ); + } + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + client = false; + persistence = false; + srvcCfg = null; + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testServersWithPersistence() throws Exception { + persistence = true; + + checkRedeployment(2, 0, F.alwaysTrue(), 2, false, true); + } + + /** + * @throws Exception if failed. + */ + public void testClientsWithPersistence() throws Exception { + persistence = true; + + checkRedeployment(2, 2, CLIENT_FILTER, 2, false, true); + } + + /** + * @throws Exception if failed. + */ + public void testServersWithoutPersistence() throws Exception { + persistence = false; + + checkRedeployment(2, 0, F.alwaysTrue(), 2, false, false); + } + + /** + * @throws Exception if failed. + */ + public void testClientsWithoutPersistence() throws Exception { + persistence = false; + + checkRedeployment(2, 2, CLIENT_FILTER, 2, false, false); + } + + /** + * @throws Exception If failed. + */ + public void testServersStaticConfigWithPersistence() throws Exception { + persistence = true; + + checkRedeployment(2, 0, F.alwaysTrue(), 2, true, true); + } + + /** + * @throws Exception If failed. + */ + public void testClientsStaticConfigWithPersistence() throws Exception { + persistence = true; + + checkRedeployment(2, 2, CLIENT_FILTER, 2, true, true); + } + + /** + * @throws Exception If failed. + */ + public void testServersStaticConfigWithoutPersistence() throws Exception { + persistence = false; + + checkRedeployment(2, 0, F.alwaysTrue(), 2, true, true); + } + + /** + * @throws Exception If failed. + */ + public void _testClientsStaticConfigWithoutPersistence() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8279"); + + persistence = false; + + checkRedeployment(2, 2, CLIENT_FILTER, 2, true, true); + } + + /** + * @param srvsNum Number of server nodes to start. + * @param clientsNum Number of client nodes to start. + * @param nodeFilter Node filter. + * @param deps Expected number of deployed services. + * @param isStatic Static or dynamic service deployment is used. + * @param expRedep {@code true} if services should be redeployed on activation. {@code false} otherwise. + * @throws Exception If failed. + */ + private void checkRedeployment(int srvsNum, int clientsNum, IgnitePredicate nodeFilter, int deps, + boolean isStatic, boolean expRedep) throws Exception { + + if (isStatic) + srvcCfg = getServiceConfiguration(nodeFilter); + + CountDownLatch exeLatch = new CountDownLatch(deps); + CountDownLatch cancelLatch = new CountDownLatch(deps); + + DummyService.exeLatch(SERVICE_NAME, exeLatch); + DummyService.cancelLatch(SERVICE_NAME, cancelLatch); + + for (int i = 0; i < srvsNum; i++) + startGrid(i); + + client = true; + + for (int i = 0; i < clientsNum; i++) + startGrid(srvsNum + i); + + Ignite ignite = grid(0); + + ignite.cluster().active(true); + + if (!isStatic) { + ServiceConfiguration srvcCfg = getServiceConfiguration(nodeFilter); + + ignite.services().deploy(srvcCfg); + } + + assertTrue(exeLatch.await(10, TimeUnit.SECONDS)); + + ignite.cluster().active(false); + + assertTrue(cancelLatch.await(10, TimeUnit.SECONDS)); + + exeLatch = new CountDownLatch(expRedep ? deps : 1); + + DummyService.exeLatch(SERVICE_NAME, exeLatch); + + ignite.cluster().active(true); + + if (expRedep) + assertTrue(exeLatch.await(10, TimeUnit.SECONDS)); + else + assertFalse(exeLatch.await(1, TimeUnit.SECONDS)); + } + + /** + * @param nodeFilter Node filter. + * @return Service configuration. + */ + private ServiceConfiguration getServiceConfiguration(IgnitePredicate nodeFilter) { + ServiceConfiguration srvcCfg = new ServiceConfiguration(); + srvcCfg.setName(SERVICE_NAME); + srvcCfg.setMaxPerNodeCount(1); + srvcCfg.setNodeFilter(nodeFilter); + srvcCfg.setService(new DummyService()); + return srvcCfg; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java index f26205bf79538..12e4802af31cc 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java @@ -68,6 +68,7 @@ import org.apache.ignite.internal.processors.service.IgniteServiceDynamicCachesSelfTest; import org.apache.ignite.internal.processors.service.IgniteServiceProxyTimeoutInitializedTest; import org.apache.ignite.internal.processors.service.IgniteServiceReassignmentTest; +import org.apache.ignite.internal.processors.service.ServiceDeploymentOnActivationTest; import org.apache.ignite.internal.processors.service.ServicePredicateAccessCacheTest; import org.apache.ignite.internal.util.GridStartupWithUndefinedIgniteHomeSelfTest; import org.apache.ignite.services.ServiceThreadPoolSelfTest; @@ -146,6 +147,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(ServiceThreadPoolSelfTest.class); suite.addTestSuite(GridServiceProcessorBatchDeploySelfTest.class); suite.addTestSuite(GridServiceDeploymentCompoundFutureSelfTest.class); + suite.addTestSuite(ServiceDeploymentOnActivationTest.class); suite.addTestSuite(IgniteServiceDeploymentClassLoadingDefaultMarshallerTest.class); suite.addTestSuite(IgniteServiceDeploymentClassLoadingJdkMarshallerTest.class); From 430b7b14f8da30ee36196919b99408e63620a635 Mon Sep 17 00:00:00 2001 From: Denis Mekhanikov Date: Fri, 20 Apr 2018 18:41:06 +0300 Subject: [PATCH 0146/1463] IGNITE-8134 Subscribe to system cache events on nodes outside BLT Signed-off-by: Andrey Gura (cherry picked from commit c82277eb4e48f95dfec8cb0206c019820a765432) --- .../cluster/DiscoveryDataClusterState.java | 7 + .../service/GridServiceProcessor.java | 14 +- .../ServiceDeploymentOutsideBaselineTest.java | 280 ++++++++++++++++++ .../testsuites/IgniteKernalSelfTestSuite.java | 2 + 4 files changed, 300 insertions(+), 3 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/service/ServiceDeploymentOutsideBaselineTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/DiscoveryDataClusterState.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/DiscoveryDataClusterState.java index b022754c6670a..840541097ec42 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/DiscoveryDataClusterState.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/DiscoveryDataClusterState.java @@ -200,6 +200,13 @@ public boolean active() { return baselineTopology; } + /** + * @return {@code True} if baseline topology is set in the cluster. {@code False} otherwise. + */ + public boolean hasBaselineTopology() { + return baselineTopology != null; + } + /** * @return Nodes participating in state change exchange. */ 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 9cf27d28c98ee..b8022a3184c27 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 @@ -72,6 +72,7 @@ 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; +import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport; import org.apache.ignite.internal.processors.task.GridInternal; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; @@ -244,10 +245,17 @@ private void onKernalStart0() throws IgniteCheckedException { if (ctx.deploy().enabled()) ctx.cache().context().deploy().ignoreOwnership(true); - if (!ctx.clientNode() && serviceCache.context().affinityNode()) { - // Register query listener and run it for local entries. It is also invoked on rebalancing. + if (!ctx.clientNode()) { + DiscoveryDataClusterState clusterState = ctx.state().clusterState(); + + boolean isLocLsnr = !clusterState.hasBaselineTopology() || + CU.baselineNode(ctx.cluster().get().localNode(), clusterState); + + // Register query listener and run it for local entries, if data is available locally. + // It is also invoked on rebalancing. + // Otherwise remote listener is registered. serviceCache.context().continuousQueries().executeInternalQuery( - new ServiceEntriesListener(), null, true, true, false + new ServiceEntriesListener(), null, isLocLsnr, true, false ); } else { // Listener for client nodes is registered in onContinuousProcessorStarted method. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ServiceDeploymentOutsideBaselineTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ServiceDeploymentOutsideBaselineTest.java new file mode 100644 index 0000000000000..1a5b630900599 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ServiceDeploymentOutsideBaselineTest.java @@ -0,0 +1,280 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.service; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCluster; +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.lang.IgniteFuture; +import org.apache.ignite.services.ServiceConfiguration; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** */ +public class ServiceDeploymentOutsideBaselineTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final String SERVICE_NAME = "test-service"; + + /** */ + private boolean persistence; + + /** */ + private ServiceConfiguration srvcCfg; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + TcpDiscoverySpi discoverySpi = new TcpDiscoverySpi(); + discoverySpi.setIpFinder(IP_FINDER); + cfg.setDiscoverySpi(discoverySpi); + + if (persistence) { + cfg.setDataStorageConfiguration( + new DataStorageConfiguration() + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration() + .setPersistenceEnabled(true) + .setMaxSize(10 * 1024 * 1024) + ).setWalMode(WALMode.LOG_ONLY) + ); + } + + if (srvcCfg != null) + cfg.setServiceConfiguration(srvcCfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + persistence = false; + srvcCfg = null; + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testDeployOutsideBaseline() throws Exception { + checkDeploymentFromOutsideNode(true, false); + } + + /** + * @throws Exception If failed. + */ + public void testDeployOutsideBaselineNoPersistence() throws Exception { + checkDeploymentFromOutsideNode(false, false); + } + + /** + * @throws Exception If failed. + */ + public void testDeployOutsideBaselineStatic() throws Exception { + checkDeploymentFromOutsideNode(true, true); + } + + /** + * @throws Exception If failed. + */ + public void testDeployOutsideBaselineStaticNoPersistence() throws Exception { + checkDeploymentFromOutsideNode(false, true); + } + + /** + * @throws Exception If failed. + */ + public void testDeployFromNodeAddedToBlt() throws Exception { + checkDeployWithNodeAddedToBlt(true); + } + + /** + * @throws Exception If failed. + */ + public void testDeployToNodeAddedToBlt() throws Exception { + checkDeployWithNodeAddedToBlt(false); + } + + /** + * @throws Exception If failed. + */ + public void testDeployFromNodeRemovedFromBlt() throws Exception { + checkDeployFromNodeRemovedFromBlt(true, false); + } + + /** + * @throws Exception If failed. + */ + public void testDeployFromNodeRemovedFromBltStatic() throws Exception { + checkDeployFromNodeRemovedFromBlt(true, true); + } + + /** + * @throws Exception If failed. + */ + public void testDeployToNodeRemovedFromBlt() throws Exception { + checkDeployFromNodeRemovedFromBlt(false, false); + } + + /** + * @param persistence If {@code true}, then persistence will be enabled. + * @param staticDeploy If {@code true}, then static deployment will be used instead of a dynamic one. + * @throws Exception If failed. + */ + private void checkDeploymentFromOutsideNode(boolean persistence, boolean staticDeploy) throws Exception { + this.persistence = persistence; + + Ignite insideNode = startGrid(0); + + if (persistence) + insideNode.cluster().active(true); + else { + IgniteCluster cluster = insideNode.cluster(); + + cluster.setBaselineTopology(cluster.topologyVersion()); + } + + CountDownLatch exeLatch = new CountDownLatch(1); + + DummyService.exeLatch(SERVICE_NAME, exeLatch); + + deployServiceFromNewNode(staticDeploy); + + assertTrue(exeLatch.await(10, TimeUnit.SECONDS)); + } + + /** + * @param from If {@code true}, then added node will be an initiator of deployment. + * Otherwise deployment to this node will be tested. + * @throws Exception If failed. + */ + private void checkDeployWithNodeAddedToBlt(boolean from) throws Exception { + persistence = true; + + Ignite insideNode = startGrid(0); + + IgniteCluster cluster = insideNode.cluster(); + + cluster.active(true); + + Ignite outsideNode = startGrid(1); + + cluster.setBaselineTopology(cluster.topologyVersion()); + + CountDownLatch exeLatch = new CountDownLatch(from ? 1 : 2); + + DummyService.exeLatch(SERVICE_NAME, exeLatch); + + if (from) { + IgniteFuture depFut = outsideNode.services().deployClusterSingletonAsync(SERVICE_NAME, new DummyService()); + + depFut.get(10, TimeUnit.SECONDS); + } + else { + IgniteFuture depFut = outsideNode.services().deployNodeSingletonAsync(SERVICE_NAME, new DummyService()); + + depFut.get(10, TimeUnit.SECONDS); + } + + assertTrue(exeLatch.await(10, TimeUnit.SECONDS)); + } + + /** + * @param from If {@code true}, then added node will be an initiator of deployment. + * Otherwise deployment to this node will be tested. + * @param staticDeploy If {@code true}, then static deployment will be used instead of a dynamic one. + * @throws Exception If failed. + */ + private void checkDeployFromNodeRemovedFromBlt(boolean from, boolean staticDeploy) throws Exception { + persistence = true; + + Ignite insideNode = startGrid(0); + startGrid(1); + + IgniteCluster cluster = insideNode.cluster(); + + cluster.active(true); + + stopGrid(1); + + cluster.setBaselineTopology(cluster.topologyVersion()); + + CountDownLatch exeLatch = new CountDownLatch(from ? 1 : 2); + + DummyService.exeLatch(SERVICE_NAME, exeLatch); + + if (from) + deployServiceFromNewNode(staticDeploy); + else { + startGrid(1); + + IgniteFuture depFut = insideNode.services().deployNodeSingletonAsync(SERVICE_NAME, new DummyService()); + + depFut.get(10, TimeUnit.SECONDS); + } + + assertTrue(exeLatch.await(10, TimeUnit.SECONDS)); + } + + /** + * @param staticDeploy If {@code true}, then static deployment will be used instead of a dynamic one. + * @throws Exception If node failed to start. + */ + private void deployServiceFromNewNode(boolean staticDeploy) throws Exception { + if (staticDeploy) { + srvcCfg = getClusterSingletonServiceConfiguration(); + + startGrid(1); + } + else { + Ignite node = startGrid(1); + + IgniteFuture depFut = node.services().deployClusterSingletonAsync(SERVICE_NAME, new DummyService()); + + depFut.get(10, TimeUnit.SECONDS); + } + } + + /** + * @return Test service configuration. + */ + private ServiceConfiguration getClusterSingletonServiceConfiguration() { + ServiceConfiguration srvcCfg = new ServiceConfiguration(); + srvcCfg.setName(SERVICE_NAME); + srvcCfg.setService(new DummyService()); + srvcCfg.setTotalCount(1); + + return srvcCfg; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java index 12e4802af31cc..8e3411a8cc045 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java @@ -69,6 +69,7 @@ import org.apache.ignite.internal.processors.service.IgniteServiceProxyTimeoutInitializedTest; import org.apache.ignite.internal.processors.service.IgniteServiceReassignmentTest; import org.apache.ignite.internal.processors.service.ServiceDeploymentOnActivationTest; +import org.apache.ignite.internal.processors.service.ServiceDeploymentOutsideBaselineTest; import org.apache.ignite.internal.processors.service.ServicePredicateAccessCacheTest; import org.apache.ignite.internal.util.GridStartupWithUndefinedIgniteHomeSelfTest; import org.apache.ignite.services.ServiceThreadPoolSelfTest; @@ -148,6 +149,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(GridServiceProcessorBatchDeploySelfTest.class); suite.addTestSuite(GridServiceDeploymentCompoundFutureSelfTest.class); suite.addTestSuite(ServiceDeploymentOnActivationTest.class); + suite.addTestSuite(ServiceDeploymentOutsideBaselineTest.class); suite.addTestSuite(IgniteServiceDeploymentClassLoadingDefaultMarshallerTest.class); suite.addTestSuite(IgniteServiceDeploymentClassLoadingJdkMarshallerTest.class); From 3a01b88419e08df540a177df277aedc88cf059b3 Mon Sep 17 00:00:00 2001 From: Andrei Aleksandrov Date: Sat, 28 Apr 2018 19:07:55 +0300 Subject: [PATCH 0147/1463] backport IGNITE-7918 fix to 2.5 master --- .../affinity/GridAffinityAssignmentCache.java | 58 +++-- .../affinity/GridAffinityProcessor.java | 81 ++++++- .../affinity/AffinityHistoryCleanupTest.java | 86 ++++---- .../GridAffinityProcessorMemoryLeakTest.java | 202 ++++++++++++++++++ .../testsuites/IgniteBasicTestSuite.java | 2 + 5 files changed, 346 insertions(+), 83 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorMemoryLeakTest.java 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 427d60328b7fb..2153e59f634bc 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 @@ -114,9 +114,6 @@ public class GridAffinityAssignmentCache { /** Node stop flag. */ private volatile IgniteCheckedException stopErr; - /** History size ignoring client events changes. */ - private final AtomicInteger histSize = new AtomicInteger(); - /** Full history size. */ private final AtomicInteger fullHistSize = new AtomicInteger(); @@ -198,11 +195,13 @@ public int groupId() { */ public void initialize(AffinityTopologyVersion topVer, List> affAssignment) { assert topVer.compareTo(lastVersion()) >= 0 : "[topVer = " + topVer + ", last=" + lastVersion() + ']'; + assert idealAssignment != null; GridAffinityAssignment assignment = new GridAffinityAssignment(topVer, affAssignment, idealAssignment); - affCache.put(topVer, new HistoryAffinityAssignment(assignment)); + HistoryAffinityAssignment hAff = affCache.put(topVer, new HistoryAffinityAssignment(assignment)); + head.set(assignment); for (Map.Entry entry : readyFuts.entrySet()) { @@ -215,7 +214,9 @@ public void initialize(AffinityTopologyVersion topVer, List> a } } - onHistoryAdded(assignment); + // In case if value was replaced there is no sense to clean the history. + if (hAff == null) + onHistoryAdded(); } /** @@ -259,6 +260,8 @@ public void onReconnected() { affCache.clear(); + fullHistSize.set(0); + head.set(new GridAffinityAssignment(AffinityTopologyVersion.NONE)); stopErr = null; @@ -429,12 +432,14 @@ public void clientEventTopologyChange(DiscoveryEvent evt, AffinityTopologyVersio GridAffinityAssignment aff = head.get(); - assert evt.type() == EVT_DISCOVERY_CUSTOM_EVT || aff.primaryPartitions(evt.eventNode().id()).isEmpty() : evt; - assert evt.type() == EVT_DISCOVERY_CUSTOM_EVT || aff.backupPartitions(evt.eventNode().id()).isEmpty() : evt; + assert evt.type() == EVT_DISCOVERY_CUSTOM_EVT || aff.primaryPartitions(evt.eventNode().id()).isEmpty() : evt; + + assert evt.type() == EVT_DISCOVERY_CUSTOM_EVT || aff.backupPartitions(evt.eventNode().id()).isEmpty() : evt; GridAffinityAssignment assignmentCpy = new GridAffinityAssignment(topVer, aff); - affCache.put(topVer, new HistoryAffinityAssignment(assignmentCpy)); + HistoryAffinityAssignment hAff = affCache.put(topVer, new HistoryAffinityAssignment(assignmentCpy)); + head.set(assignmentCpy); for (Map.Entry entry : readyFuts.entrySet()) { @@ -447,7 +452,9 @@ public void clientEventTopologyChange(DiscoveryEvent evt, AffinityTopologyVersio } } - onHistoryAdded(assignmentCpy); + // In case if value was replaced there is no sense to clean the history. + if (hAff == null) + onHistoryAdded(); } /** @@ -724,27 +731,15 @@ private void awaitTopologyVersion(AffinityTopologyVersion topVer) { } /** - * @param aff Added affinity assignment. + * Cleaning the affinity history. */ - private void onHistoryAdded(GridAffinityAssignment aff) { - int fullSize = fullHistSize.incrementAndGet(); - - int size; - - if (aff.clientEventChange()) - size = histSize.get(); - else - size = histSize.incrementAndGet(); - - int rmvCnt = size - MAX_HIST_SIZE; + private void onHistoryAdded() { + if (fullHistSize.incrementAndGet() > MAX_HIST_SIZE) { + Iterator it = affCache.values().iterator(); - if (rmvCnt <= 0) { - if (fullSize > MAX_HIST_SIZE * 2) - rmvCnt = MAX_HIST_SIZE; - } + int rmvCnt = MAX_HIST_SIZE / 2; - if (rmvCnt > 0) { - Iterator it = affCache.values().iterator(); + AffinityTopologyVersion topVerRmv = null; while (it.hasNext() && rmvCnt > 0) { AffinityAssignment aff0 = it.next(); @@ -753,11 +748,14 @@ private void onHistoryAdded(GridAffinityAssignment aff) { rmvCnt--; - if (!aff0.clientEventChange()) - histSize.decrementAndGet(); - fullHistSize.decrementAndGet(); + + topVerRmv = aff0.topologyVersion(); } + + topVerRmv = it.hasNext() ? it.next().topologyVersion() : topVerRmv; + + ctx.affinity().removeCachedAffinity(topVerRmv); } } 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 128eaf0699ef6..e26c0ce830cc4 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 @@ -26,9 +26,10 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ConcurrentSkipListMap; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.affinity.Affinity; @@ -63,7 +64,6 @@ import org.apache.ignite.lang.IgniteUuid; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.concurrent.ConcurrentHashMap; import static org.apache.ignite.cache.CacheMode.LOCAL; import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; @@ -86,8 +86,11 @@ public class GridAffinityProcessor extends GridProcessorAdapter { /** Time to wait between errors (in milliseconds). */ private static final long ERROR_WAIT = 500; + /** Log. */ + private final IgniteLogger log; + /** Affinity map. */ - private final ConcurrentMap> affMap = new ConcurrentHashMap<>(); + private final ConcurrentSkipListMap> affMap = new ConcurrentSkipListMap<>(); /** Listener. */ private final GridLocalEventListener lsnr = new GridLocalEventListener() { @@ -131,6 +134,8 @@ public class GridAffinityProcessor extends GridProcessorAdapter { */ public GridAffinityProcessor(GridKernalContext ctx) { super(ctx); + + log = ctx.log(GridAffinityProcessor.class); } /** {@inheritDoc} */ @@ -212,6 +217,34 @@ public int partition0(String cacheName, Object key, @Nullable AffinityInfo aff) return affInfo != null ? F.first(affInfo.assignment().get(partId)) : null; } + /** + * Removes cached affinity instances with affinity topology versions less than {@code topVer}. + * + * @param topVer topology version. + */ + public void removeCachedAffinity(AffinityTopologyVersion topVer) { + assert topVer != null; + + int oldSize = affMap.size(); + + Iterator>> it = + affMap.headMap(new AffinityAssignmentKey(topVer)).entrySet().iterator(); + + while (it.hasNext()) { + Map.Entry> entry = it.next(); + + assert entry.getValue() != null; + + if (!entry.getValue().isDone()) + continue; + + it.remove(); + } + + if (log.isDebugEnabled()) + log.debug("Affinity cached values were cleared: " + (oldSize - affMap.size())); + } + /** * Maps keys to nodes for given cache. @@ -358,6 +391,9 @@ private Map> keysToNodes(@Nullable final String c @SuppressWarnings("ErrorNotRethrown") @Nullable private AffinityInfo affinityCache(final String cacheName, AffinityTopologyVersion topVer) throws IgniteCheckedException { + + assert cacheName != null; + AffinityAssignmentKey key = new AffinityAssignmentKey(cacheName, topVer); IgniteInternalFuture fut = affMap.get(key); @@ -658,7 +694,7 @@ private GridAffinityAssignment assignment() { /** * */ - private static class AffinityAssignmentKey { + private static class AffinityAssignmentKey implements Comparable { /** */ private String cacheName; @@ -669,11 +705,20 @@ private static class AffinityAssignmentKey { * @param cacheName Cache name. * @param topVer Topology version. */ - private AffinityAssignmentKey(String cacheName, @NotNull AffinityTopologyVersion topVer) { + private AffinityAssignmentKey(@NotNull String cacheName, @NotNull AffinityTopologyVersion topVer) { this.cacheName = cacheName; this.topVer = topVer; } + /** + * Current constructor should be used only in removeCachedAffinity for creating of the special keys for removing. + * + * @param topVer Topology version. + */ + private AffinityAssignmentKey(@NotNull AffinityTopologyVersion topVer) { + this.topVer = topVer; + } + /** {@inheritDoc} */ @Override public boolean equals(Object o) { if (this == o) @@ -700,6 +745,32 @@ private AffinityAssignmentKey(String cacheName, @NotNull AffinityTopologyVersion @Override public String toString() { return S.toString(AffinityAssignmentKey.class, this); } + + /** {@inheritDoc} */ + @Override public int compareTo(AffinityAssignmentKey o) { + assert o != null; + + if (this == o) + return 0; + + int res = this.topVer.compareTo(o.topVer); + + // Key with null cache name must be less than any key with not null cache name for the same topVer. + if (res == 0) { + if (cacheName == null && o.cacheName != null) + return -1; + + if (cacheName != null && o.cacheName == null) + return 1; + + if (cacheName == null && o.cacheName == null) + return 0; + + return cacheName.compareTo(o.cacheName); + } + + return res; + } } /** diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java index 605cc5f57c95b..f89d9ee7381ad 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java @@ -89,34 +89,34 @@ public void testAffinityHistoryCleanup() throws Exception { Ignite ignite = startGrid(0); - checkHistory(ignite, F.asList(topVer(1, 0)), 1); + checkHistory(ignite, F.asList(topVer(1, 0)), 1); //fullHistSize = 1 startGrid(1); checkHistory(ignite, F.asList( - topVer(1, 0), - topVer(2, 0), - topVer(2, 1)), + topVer(1, 0), // FullHistSize = 1. + topVer(2, 0), // FullHistSize = 2. + topVer(2, 1)), // FullHistSize = 3. 3); startGrid(2); checkHistory(ignite, F.asList( - topVer(1, 0), - topVer(2, 0), - topVer(2, 1), - topVer(3, 0), - topVer(3, 1)), + topVer(1, 0), // FullHistSize = 1. + topVer(2, 0), // FullHistSize = 2. + topVer(2, 1), // FullHistSize = 3. + topVer(3, 0), // FullHistSize = 4. + topVer(3, 1)), // FullHistSize = 5. 5); startGrid(3); checkHistory(ignite, F.asList( - topVer(2, 1), - topVer(3, 0), - topVer(3, 1), - topVer(4, 0), - topVer(4, 1)), + topVer(2, 1), // FullHistSize = 3. + topVer(3, 0), // FullHistSize = 4. + topVer(3, 1), // FullHistSize = 5. + topVer(4, 0), // FullHistSize = (6 - IGNITE_AFFINITY_HISTORY_SIZE(5)/2) = 4. + topVer(4, 1)), // FullHistSize = 5. 5); client = true; @@ -126,13 +126,11 @@ public void testAffinityHistoryCleanup() throws Exception { stopGrid(4); checkHistory(ignite, F.asList( - topVer(2, 1), - topVer(3, 0), - topVer(3, 1), - topVer(4, 0), - topVer(4, 1), - topVer(5, 0), - topVer(6, 0)), + topVer(3, 1), // FullHistSize = 5. + topVer(4, 0), // FullHistSize = (6 - IGNITE_AFFINITY_HISTORY_SIZE(5)/2) = 4. + topVer(4, 1), // FullHistSize = 5. + topVer(5, 0), // FullHistSize = (6 - IGNITE_AFFINITY_HISTORY_SIZE(5)/2) = 4. + topVer(6, 0)), // FullHistSize = 5. 5); startGrid(4); @@ -140,15 +138,11 @@ public void testAffinityHistoryCleanup() throws Exception { stopGrid(4); checkHistory(ignite, F.asList( - topVer(2, 1), - topVer(3, 0), - topVer(3, 1), - topVer(4, 0), - topVer(4, 1), - topVer(5, 0), - topVer(6, 0), - topVer(7, 0), - topVer(8, 0)), + topVer(4, 1), // FullHistSize = 5. + topVer(5, 0), // FullHistSize = (6 - IGNITE_AFFINITY_HISTORY_SIZE(5)/2) = 4. + topVer(6, 0), // FullHistSize = 5. + topVer(7, 0), // FullHistSize = (6 - IGNITE_AFFINITY_HISTORY_SIZE(5)/2) = 4. + topVer(8, 0)), // FullHistSize = 5. 5); startGrid(4); @@ -156,28 +150,24 @@ public void testAffinityHistoryCleanup() throws Exception { stopGrid(4); checkHistory(ignite, F.asList( - topVer(5, 0), - topVer(6, 0), - topVer(7, 0), - topVer(8, 0), - topVer(9, 0), - topVer(10, 0)), - 0); + topVer(6, 0), // FullHistSize = 5. + topVer(7, 0), // FullHistSize = (6 - IGNITE_AFFINITY_HISTORY_SIZE(5)/2) = 4. + topVer(8, 0), // FullHistSize = 5. + topVer(9, 0), // FullHistSize = (6 - IGNITE_AFFINITY_HISTORY_SIZE(5)/2) = 4. + topVer(10, 0)), // FullHistSize = 5. + 5); client = false; startGrid(4); checkHistory(ignite, F.asList( - topVer(5, 0), - topVer(6, 0), - topVer(7, 0), - topVer(8, 0), - topVer(9, 0), - topVer(10, 0), - topVer(11, 0), - topVer(11, 1)), - 2); + topVer(8, 0), // FullHistSize = 5. + topVer(9, 0), // FullHistSize = (6 - IGNITE_AFFINITY_HISTORY_SIZE(5)/2) = 4. + topVer(10, 0), // FullHistSize = 5. + topVer(11, 0), // FullHistSize = (6 - IGNITE_AFFINITY_HISTORY_SIZE(5)/2) = 4. + topVer(11, 1)), // FullHistSize = 5. + 5); } finally { if (histProp != null) @@ -203,9 +193,9 @@ private void checkHistory(Ignite ignite, List expHist, for (GridCacheContext cctx : proc.context().cacheContexts()) { GridAffinityAssignmentCache aff = GridTestUtils.getFieldValue(cctx.affinity(), "aff"); - AtomicInteger histSize = GridTestUtils.getFieldValue(aff, "histSize"); + AtomicInteger fullHistSize = GridTestUtils.getFieldValue(aff, "fullHistSize"); - assertEquals(expSize, histSize.get()); + assertEquals(expSize, fullHistSize.get()); Map cache = GridTestUtils.getFieldValue(aff, "affCache"); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorMemoryLeakTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorMemoryLeakTest.java new file mode 100644 index 0000000000000..3b6857d5ad2ca --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorMemoryLeakTest.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.affinity; + +import java.util.concurrent.ConcurrentSkipListMap; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteKernal; +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.testframework.junits.common.GridCommonTest; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE; +import static org.apache.ignite.IgniteSystemProperties.getInteger; + +/** + * Tests for {@link GridAffinityProcessor}. + */ +@GridCommonTest(group = "Affinity Processor") +public class GridAffinityProcessorMemoryLeakTest extends GridCommonAbstractTest { + /** Max value for affinity history size name. Should be the same as in GridAffinityAssignmentCache.MAX_HIST_SIZE */ + private final int MAX_HIST_SIZE = getInteger(IGNITE_AFFINITY_HISTORY_SIZE, 100); + + /** Cache name. */ + private static final String CACHE_NAME = "cache"; + + /** IP finder. */ + private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); + + discoSpi.setForceServerMode(true); + + discoSpi.setIpFinder(ipFinder); + + cfg.setDiscoverySpi(discoSpi); + + CacheConfiguration cacheCfg = defaultCacheConfiguration(); + + cacheCfg.setName(CACHE_NAME); + + cacheCfg.setStoreKeepBinary(true); + + cacheCfg.setCacheMode(CacheMode.LOCAL); + + cfg.setCacheConfiguration(cacheCfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + /** + * Test affinity functions caching and clean up. + * + * @throws Exception In case of any exception. + */ + public void testAffinityProcessor() throws Exception { + Ignite ignite = startGrid(0); + + IgniteKernal grid = (IgniteKernal)grid(0); + + IgniteCache cache; + + IgniteCache globalCache = getOrCreateGlobalCache(ignite); + + IgniteDataStreamer globalStreamer; + + int count = MAX_HIST_SIZE * 4; + + int size; + + do { + try { + cache = createLocalCache(ignite, count); + + cache.put("Key" + count, "Value" + count); + + cache.destroy(); + + globalStreamer = createGlobalStreamer(ignite, globalCache); + + globalStreamer.addData("GlobalKey" + count, "GlobalValue" + count); + + globalStreamer.flush(); + + globalStreamer.close(); + + size = ((ConcurrentSkipListMap)GridTestUtils.getFieldValue(grid.context().affinity(), "affMap")).size(); + + assertTrue("Cache has size that bigger then expected [size=" + size + "" + + ", expLimit=" + MAX_HIST_SIZE * 3 + "]", size < MAX_HIST_SIZE * 3); + } + catch (Exception e) { + fail("Error was handled [" + e.getMessage() + "]"); + } + } + while (count-- > 0); + } + + /** + * Creates global cache. + * + * @param ignite instance of {@code Ignite}. + * @param id unique id for local cache. + * @return local cache instance. + */ + private static IgniteCache createLocalCache(Ignite ignite, long id) { + final String cacheName = "localCache" + id; + + final CacheConfiguration cCfg = new CacheConfiguration<>(); + + cCfg.setName(cacheName); + + cCfg.setCacheMode(CacheMode.LOCAL); + + cCfg.setGroupName("some group"); + + ignite.destroyCache(cacheName); // Local cache is not really local - reference can be kept by other nodes if restart during the load happens. + + return ignite.createCache(cCfg).withKeepBinary(); + } + + /** + * Gets or creates global cache. + * + * @param ignite instance of {@code Ignite}. + * @return global cache instance. + */ + private static IgniteCache getOrCreateGlobalCache(Ignite ignite) { + final String cacheName = "GlobalCache"; + + final CacheConfiguration cCfg = new CacheConfiguration<>(); + + cCfg.setName(cacheName); + + cCfg.setStoreKeepBinary(true); + + cCfg.setCacheMode(CacheMode.PARTITIONED); + + cCfg.setOnheapCacheEnabled(false); + + cCfg.setCopyOnRead(false); + + cCfg.setBackups(0); + + cCfg.setWriteBehindEnabled(false); + + cCfg.setReadThrough(false); + + return ignite.getOrCreateCache(cCfg).withKeepBinary(); + } + + /** + * Creates streamer for global cache. + * + * @param ignite instance of {@code Ignite}. + * @param cache instance of global cache. + * @return instance of {@code IgniteDataStreamer}. + */ + private static IgniteDataStreamer createGlobalStreamer(Ignite ignite, + IgniteCache cache) { + IgniteDataStreamer streamer = ignite.dataStreamer(cache.getName()); + + streamer.allowOverwrite(true); + + streamer.skipStore(true); + + streamer.keepBinary(false); + + return streamer; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java index e71a56981f09c..cc93c5bfbb62e 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java @@ -44,6 +44,7 @@ import org.apache.ignite.internal.MarshallerContextLockingSelfTest; import org.apache.ignite.internal.TransactionsMXBeanImplTest; import org.apache.ignite.internal.managers.IgniteDiagnosticMessagesTest; +import org.apache.ignite.internal.processors.affinity.GridAffinityProcessorMemoryLeakTest; import org.apache.ignite.internal.processors.affinity.GridAffinityProcessorRendezvousSelfTest; import org.apache.ignite.internal.processors.cache.GridLocalIgniteSerializationTest; import org.apache.ignite.internal.processors.cache.GridProjectionForCachesOnDaemonNodeSelfTest; @@ -133,6 +134,7 @@ public static TestSuite suite(@Nullable final Set ignoredTests) throws Ex GridTestUtils.addTestIfNeeded(suite, GridReleaseTypeSelfTest.class, ignoredTests); suite.addTestSuite(GridProductVersionSelfTest.class); suite.addTestSuite(GridAffinityProcessorRendezvousSelfTest.class); + suite.addTestSuite(GridAffinityProcessorMemoryLeakTest.class); suite.addTestSuite(GridClosureProcessorSelfTest.class); suite.addTestSuite(GridClosureProcessorRemoteTest.class); suite.addTestSuite(GridClosureSerializationTest.class); From 104b771de3f39e1b36026bac8d1bec088053a944 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Tue, 8 May 2018 15:03:00 +0300 Subject: [PATCH 0148/1463] IGNITE-8429 Unexpected error during incorrect WAL segment decompression, causes node termination Signed-off-by: Andrey Gura (cherry picked from commit f3a61e4) --- .../preloader/GridDhtPartitionSupplier.java | 4 +-- .../wal/FileWriteAheadLogManager.java | 32 +++++++++++-------- .../FsyncModeFileWriteAheadLogManager.java | 32 +++++++++++-------- 3 files changed, 40 insertions(+), 28 deletions(-) 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 84e6828df4dc3..4946d7e3f0eed 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 @@ -32,10 +32,10 @@ import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; import org.apache.ignite.internal.processors.cache.IgniteRebalanceIterator; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState; -import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; 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.persistence.CacheDataRow; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.T3; import org.apache.ignite.internal.util.typedef.internal.S; 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 779534489a3bc..6ac102fff5e75 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 @@ -33,6 +33,7 @@ import java.nio.ByteOrder; import java.nio.MappedByteBuffer; import java.nio.channels.ClosedByInterruptException; +import java.nio.file.FileAlreadyExistsException; import java.nio.file.Files; import java.sql.Time; import java.util.ArrayList; @@ -2065,11 +2066,11 @@ private class FileDecompressor extends Thread { /** {@inheritDoc} */ @Override public void run() { - Throwable err = null; - while (!Thread.currentThread().isInterrupted() && !stopped) { + long segmentToDecompress = -1L; + try { - long segmentToDecompress = segmentsQueue.take(); + segmentToDecompress = segmentsQueue.take(); if (stopped) break; @@ -2087,7 +2088,16 @@ private class FileDecompressor extends Thread { io.write(arr, 0, bytesRead); } - Files.move(unzipTmp.toPath(), unzip.toPath()); + try { + Files.move(unzipTmp.toPath(), unzip.toPath()); + } + catch (FileAlreadyExistsException e) { + U.error(log, "Can't rename temporary unzipped segment: raw segment is already present " + + "[tmp=" + unzipTmp + ", raw=" + unzip + "]", e); + + if (!unzipTmp.delete()) + U.error(log, "Can't delete temporary unzipped segment [tmp=" + unzipTmp + "]"); + } synchronized (this) { decompressionFutures.remove(segmentToDecompress).onDone(); @@ -2097,16 +2107,12 @@ private class FileDecompressor extends Thread { Thread.currentThread().interrupt(); } catch (Throwable t) { - err = t; - } - finally { - if (err == null && !stopped) - err = new IllegalStateException("Thread " + getName() + " is terminated unexpectedly"); + if (!stopped && segmentToDecompress != -1L) { + IgniteCheckedException e = new IgniteCheckedException("Error during WAL segment " + + "decompression [segmentIdx=" + segmentToDecompress + "]", t); - if (err instanceof OutOfMemoryError) - cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, err)); - else if (err != null) - cctx.kernalContext().failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, err)); + decompressionFutures.remove(segmentToDecompress).onDone(e); + } } } } 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 dfb1c41e7378a..cf643fde42378 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 @@ -28,6 +28,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.nio.file.FileAlreadyExistsException; import java.nio.file.Files; import java.sql.Time; import java.util.ArrayList; @@ -1873,11 +1874,11 @@ private class FileDecompressor extends Thread { /** {@inheritDoc} */ @Override public void run() { - Throwable err = null; - while (!Thread.currentThread().isInterrupted() && !stopped) { + long segmentToDecompress = -1L; + try { - long segmentToDecompress = segmentsQueue.take(); + segmentToDecompress = segmentsQueue.take(); if (stopped) break; @@ -1895,7 +1896,16 @@ private class FileDecompressor extends Thread { io.write(arr, 0, bytesRead); } - Files.move(unzipTmp.toPath(), unzip.toPath()); + try { + Files.move(unzipTmp.toPath(), unzip.toPath()); + } + catch (FileAlreadyExistsException e) { + U.error(log, "Can't rename temporary unzipped segment: raw segment is already present " + + "[tmp=" + unzipTmp + ", raw=" + unzip + ']', e); + + if (!unzipTmp.delete()) + U.error(log, "Can't delete temporary unzipped segment [tmp=" + unzipTmp + ']'); + } synchronized (this) { decompressionFutures.remove(segmentToDecompress).onDone(); @@ -1905,16 +1915,12 @@ private class FileDecompressor extends Thread { Thread.currentThread().interrupt(); } catch (Throwable t) { - err = t; - } - finally { - if (err == null && !stopped) - err = new IllegalStateException("Thread " + getName() + " is terminated unexpectedly"); + if (!stopped && segmentToDecompress != -1L) { + IgniteCheckedException e = new IgniteCheckedException("Error during WAL segment " + + "decompression [segmentIdx=" + segmentToDecompress + ']', t); - if (err instanceof OutOfMemoryError) - cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, err)); - else if (err != null) - cctx.kernalContext().failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, err)); + decompressionFutures.remove(segmentToDecompress).onDone(e); + } } } } From 99a07ed2296b75ad8d5dfff38ecd3e98eafc50e8 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Tue, 8 May 2018 15:49:59 +0300 Subject: [PATCH 0149/1463] IGNITE-8429 Unexpected error during incorrect WAL segment decompression, causes node termination + synchronized block --- .../cache/persistence/wal/FileWriteAheadLogManager.java | 4 +++- .../persistence/wal/FsyncModeFileWriteAheadLogManager.java | 4 +++- 2 files changed, 6 insertions(+), 2 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 6ac102fff5e75..8755e1bf31f4b 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 @@ -2111,7 +2111,9 @@ private class FileDecompressor extends Thread { IgniteCheckedException e = new IgniteCheckedException("Error during WAL segment " + "decompression [segmentIdx=" + segmentToDecompress + "]", t); - decompressionFutures.remove(segmentToDecompress).onDone(e); + synchronized (this) { + decompressionFutures.remove(segmentToDecompress).onDone(e); + } } } } 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 cf643fde42378..c446f7f228649 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 @@ -1919,7 +1919,9 @@ private class FileDecompressor extends Thread { IgniteCheckedException e = new IgniteCheckedException("Error during WAL segment " + "decompression [segmentIdx=" + segmentToDecompress + ']', t); - decompressionFutures.remove(segmentToDecompress).onDone(e); + synchronized (this) { + decompressionFutures.remove(segmentToDecompress).onDone(e); + } } } } From 63719a6c400449411c7bab6e0951a886468e7490 Mon Sep 17 00:00:00 2001 From: Anton Kalashnikov Date: Tue, 8 May 2018 16:35:05 +0300 Subject: [PATCH 0150/1463] IGNITE-8435 StorageException is handled like NodeStoppingException during failing transaction commit Signed-off-by: Andrey Gura (cherry picked from commit e8b846b) --- .../internal/InvalidEnvironmentException.java | 25 ++ .../internal/NodeStoppingException.java | 2 +- .../pagemem/wal/StorageException.java | 3 +- .../GridDistributedTxRemoteAdapter.java | 8 +- .../dht/GridDhtTxFinishFuture.java | 8 +- .../transactions/IgniteTxLocalAdapter.java | 8 +- .../ignite/internal/util/typedef/X.java | 4 +- .../AccountTransferTransactionTest.java | 331 ++++++++++++++++++ .../testsuites/IgniteBasicTestSuite.java | 2 + 9 files changed, 375 insertions(+), 16 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/InvalidEnvironmentException.java create mode 100644 modules/core/src/test/java/org/apache/ignite/failure/AccountTransferTransactionTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/InvalidEnvironmentException.java b/modules/core/src/main/java/org/apache/ignite/internal/InvalidEnvironmentException.java new file mode 100644 index 0000000000000..d45a443e8afd8 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/InvalidEnvironmentException.java @@ -0,0 +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; + +/** + * Marker interface of invalid environment exception. + */ +public interface InvalidEnvironmentException { + +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/NodeStoppingException.java b/modules/core/src/main/java/org/apache/ignite/internal/NodeStoppingException.java index 75447a14c53fa..cc39b14cd92d6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/NodeStoppingException.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/NodeStoppingException.java @@ -22,7 +22,7 @@ /** * */ -public class NodeStoppingException extends IgniteCheckedException { +public class NodeStoppingException extends IgniteCheckedException implements InvalidEnvironmentException { /** */ private static final long serialVersionUID = 0L; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/StorageException.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/StorageException.java index 2da08b9442965..debc391271a97 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/StorageException.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/StorageException.java @@ -19,12 +19,13 @@ import java.io.IOException; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.InvalidEnvironmentException; import org.jetbrains.annotations.NotNull; /** * Exception is needed to distinguish WAL manager & page store critical I/O errors. */ -public class StorageException extends IgniteCheckedException { +public class StorageException extends IgniteCheckedException implements InvalidEnvironmentException { /** */ private static final long serialVersionUID = 0L; 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 a692b2e8dc15d..5e3111c1250cf 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 @@ -28,8 +28,8 @@ import java.util.UUID; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.InvalidEnvironmentException; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.pagemem.wal.StorageException; import org.apache.ignite.internal.pagemem.wal.WALPointer; import org.apache.ignite.internal.pagemem.wal.record.DataEntry; @@ -734,21 +734,21 @@ else if (op == READ) { } } catch (Throwable ex) { - boolean nodeStopping = X.hasCause(ex, NodeStoppingException.class); + boolean hasIOIssue = X.hasCause(ex, InvalidEnvironmentException.class); // In case of error, we still make the best effort to commit, // as there is no way to rollback at this point. err = new IgniteTxHeuristicCheckedException("Commit produced a runtime exception " + "(all transaction entries will be invalidated): " + CU.txString(this), ex); - if (nodeStopping) { + if (hasIOIssue) { U.warn(log, "Failed to commit transaction, node is stopping [tx=" + this + ", err=" + ex + ']'); } else U.error(log, "Commit failed.", err); - uncommit(nodeStopping); + uncommit(hasIOIssue); state(UNKNOWN); 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 63807107fcb84..0ed8419b32297 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,10 +26,10 @@ 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.NodeStoppingException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.processors.cache.GridCacheCompoundIdentityFuture; import org.apache.ignite.internal.processors.cache.GridCacheFuture; @@ -168,7 +168,7 @@ public void rollbackOnError(Throwable e) { if (ERR_UPD.compareAndSet(this, null, e)) { tx.setRollbackOnly(); - if (X.hasCause(e, NodeStoppingException.class)) + if (X.hasCause(e, InvalidEnvironmentException.class)) onComplete(); else finish(false); @@ -225,9 +225,9 @@ public void onResult(UUID nodeId, GridDhtTxFinishResponse res) { if (this.tx.onePhaseCommit() && (this.tx.state() == COMMITTING)) { try { - boolean nodeStop = err != null && X.hasCause(err, NodeStoppingException.class); + boolean hasInvalidEnvironmentIssue = X.hasCause(err, InvalidEnvironmentException.class); - this.tx.tmFinish(err == null, nodeStop, false); + this.tx.tmFinish(err == null, hasInvalidEnvironmentIssue, false); } catch (IgniteCheckedException finishErr) { U.error(log, "Failed to finish tx: " + tx, e); 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 61650ccefa393..6f11a57500ec2 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 @@ -30,8 +30,8 @@ import javax.cache.processor.EntryProcessor; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.internal.InvalidEnvironmentException; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.pagemem.wal.StorageException; import org.apache.ignite.internal.pagemem.wal.WALPointer; import org.apache.ignite.internal.pagemem.wal.record.DataEntry; @@ -846,13 +846,13 @@ assert ownsLock(txEntry.cached()): throw ex; } else { - boolean nodeStopping = X.hasCause(ex, NodeStoppingException.class); + boolean hasInvalidEnvironmentIssue = X.hasCause(ex, InvalidEnvironmentException.class); IgniteCheckedException err = 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); - if (nodeStopping) { + if (hasInvalidEnvironmentIssue) { U.warn(log, "Failed to commit transaction, node is stopping " + "[tx=" + this + ", err=" + ex + ']'); } @@ -865,7 +865,7 @@ assert ownsLock(txEntry.cached()): try { // Courtesy to minimize damage. - uncommit(nodeStopping); + uncommit(hasInvalidEnvironmentIssue); } catch (Throwable ex1) { U.error(log, "Failed to uncommit transaction: " + this, ex1); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java b/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java index 1a43daa09c918..49732b6817630 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java @@ -433,14 +433,14 @@ private static void cloneField(Map identityIdxs, List * {@code false} otherwise. */ @SafeVarargs - public static boolean hasCause(@Nullable Throwable t, @Nullable Class... cls) { + public static boolean hasCause(@Nullable Throwable t, @Nullable Class... cls) { if (t == null || F.isEmpty(cls)) return false; assert cls != null; for (Throwable th = t; th != null; th = th.getCause()) { - for (Class c : cls) { + for (Class c : cls) { if (c.isAssignableFrom(th.getClass())) return true; } diff --git a/modules/core/src/test/java/org/apache/ignite/failure/AccountTransferTransactionTest.java b/modules/core/src/test/java/org/apache/ignite/failure/AccountTransferTransactionTest.java new file mode 100644 index 0000000000000..8d7cf15608ed2 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/failure/AccountTransferTransactionTest.java @@ -0,0 +1,331 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.failure; + +import javax.management.MBeanServer; +import javax.management.MBeanServerInvocationHandler; +import javax.management.ObjectName; +import java.lang.management.ManagementFactory; +import java.util.ArrayList; +import java.util.Random; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +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.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy; +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.TestRecordingCommunicationSpi; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.worker.WorkersControlMXBeanImpl; +import org.apache.ignite.mxbean.WorkersControlMXBean; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.transactions.Transaction; +import org.jetbrains.annotations.NotNull; + +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; + +/** + * Test transfer amount between accounts with enabled {@link StopNodeFailureHandler}. + */ +public class AccountTransferTransactionTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + /** Count of accounts in one thread. */ + private static final int ACCOUNTS_CNT = 20; + /** Count of threads and caches. */ + private static final int THREADS_CNT = 20; + /** Count of nodes to start. */ + private static final int NODES_CNT = 3; + /** Count of transaction on cache. */ + private static final int TRANSACTION_CNT = 10; + + /** {@inheritDoc} */ + @Override protected FailureHandler getFailureHandler(String igniteInstanceName) { + return new StopNodeFailureHandler(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String name) throws Exception { + final IgniteConfiguration cfg = super.getConfiguration(name); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER); + cfg.setCommunicationSpi(new TestRecordingCommunicationSpi()); + cfg.setLocalHost("127.0.0.1"); + cfg.setDataStorageConfiguration(new DataStorageConfiguration() + .setDefaultDataRegionConfiguration(new DataRegionConfiguration() + .setMaxSize(50 * 1024 * 1024) + .setPersistenceEnabled(true)) + ); + + CacheConfiguration[] cacheConfigurations = new CacheConfiguration[THREADS_CNT]; + for (int i = 0; i < THREADS_CNT; i++) { + cacheConfigurations[i] = new CacheConfiguration() + .setName(cacheName(i)) + .setAffinity(new RendezvousAffinityFunction(false, 32)) + .setBackups(1) + .setAtomicityMode(TRANSACTIONAL) + .setCacheMode(CacheMode.PARTITIONED) + .setWriteSynchronizationMode(FULL_SYNC) + .setEvictionPolicy(new FifoEvictionPolicy(1000)) + .setOnheapCacheEnabled(true); + } + + cfg.setCacheConfiguration(cacheConfigurations); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** + * Test transfer amount. + */ + public void testTransferAmount() throws Exception { + //given: started some nodes with client. + startGrids(NODES_CNT); + + IgniteEx igniteClient = startGrid(getClientConfiguration(NODES_CNT)); + + igniteClient.cluster().active(true); + + Random random = new Random(); + + long[] initAmount = new long[THREADS_CNT]; + + //and: fill all accounts on all caches and calculate total amount for every cache. + for (int cachePrefixIdx = 0; cachePrefixIdx < THREADS_CNT; cachePrefixIdx++) { + IgniteCache cache = igniteClient.getOrCreateCache(cacheName(cachePrefixIdx)); + + try (Transaction tx = igniteClient.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + for (int accountId = 0; accountId < ACCOUNTS_CNT; accountId++) { + Long amount = (long)random.nextInt(1000); + + cache.put(accountId, amount); + + initAmount[cachePrefixIdx] += amount; + } + + tx.commit(); + } + } + + //when: start transfer amount from account to account in different threads. + CountDownLatch firstTransactionDone = new CountDownLatch(THREADS_CNT); + + ArrayList transferThreads = new ArrayList<>(); + + for (int i = 0; i < THREADS_CNT; i++) { + transferThreads.add(new TransferAmountTxThread(firstTransactionDone, igniteClient, cacheName(i))); + + transferThreads.get(i).start(); + } + + firstTransactionDone.await(10, TimeUnit.SECONDS); + + //and: terminate disco-event-worker thread on one node. + WorkersControlMXBean bean = workersMXBean(1); + + bean.terminateWorker( + bean.getWorkerNames().stream() + .filter(name -> name.startsWith("disco-event-worker")) + .findFirst() + .orElse(null) + ); + + for (Thread thread : transferThreads) { + thread.join(); + } + + long[] resultAmount = new long[THREADS_CNT]; + + //then: calculate total amount for every thread. + for (int j = 0; j < THREADS_CNT; j++) { + String cacheName = cacheName(j); + + IgniteCache cache = igniteClient.getOrCreateCache(cacheName); + + try (Transaction tx = igniteClient.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + + for (int i = 0; i < ACCOUNTS_CNT; i++) + resultAmount[j] += getNotNullValue(cache, i); + tx.commit(); + } + + long diffAmount = initAmount[j] - resultAmount[j]; + + //and: check that result amount equal to init amount. + assertTrue( + String.format("Total amount before and after transfer is not same: diff=%s, cache=%s", + diffAmount, cacheName), + diffAmount == 0 + ); + } + } + + /** + * Make test cache name by prefix. + */ + @NotNull private String cacheName(int cachePrefixIdx) { + return "cache" + cachePrefixIdx; + } + + /** + * Ignite configuration for client. + */ + @NotNull private IgniteConfiguration getClientConfiguration(int nodesPrefix) throws Exception { + IgniteConfiguration clientConf = getConfiguration(getTestIgniteInstanceName(nodesPrefix)); + + clientConf.setClientMode(true); + + return clientConf; + } + + /** + * Extract not null value from cache. + */ + private long getNotNullValue(IgniteCache cache, int i) { + Object value = cache.get(i); + + return value == null ? 0 : ((Long)value); + } + + /** + * Configure workers mx bean. + */ + private WorkersControlMXBean workersMXBean(int igniteInt) throws Exception { + ObjectName mbeanName = U.makeMBeanName( + getTestIgniteInstanceName(igniteInt), + "Kernal", + WorkersControlMXBeanImpl.class.getSimpleName() + ); + + MBeanServer mbeanSrv = ManagementFactory.getPlatformMBeanServer(); + + if (!mbeanSrv.isRegistered(mbeanName)) + fail("MBean is not registered: " + mbeanName.getCanonicalName()); + + return MBeanServerInvocationHandler.newProxyInstance(mbeanSrv, mbeanName, WorkersControlMXBean.class, true); + } + + /** + * + */ + private static class TransferAmountTxThread extends Thread { + /** */ + private CountDownLatch firstTransactionLatch; + /** */ + private Ignite ignite; + /** */ + private String cacheName; + /** */ + private Random random = new Random(); + + /** + * @param ignite Ignite. + */ + private TransferAmountTxThread(CountDownLatch firstTransactionLatch, final Ignite ignite, String cacheName) { + this.firstTransactionLatch = firstTransactionLatch; + this.ignite = ignite; + this.cacheName = cacheName; + } + + /** {@inheritDoc} */ + @Override public void run() { + for (int i = 0; i < TRANSACTION_CNT; i++) { + try { + updateInTransaction(ignite.cache(cacheName)); + } + finally { + if (i == 0) + firstTransactionLatch.countDown(); + } + } + } + + /** + * @throws IgniteException if fails + */ + @SuppressWarnings("unchecked") + private void updateInTransaction(IgniteCache cache) throws IgniteException { + int accIdFrom = random.nextInt(ACCOUNTS_CNT); + int accIdTo = random.nextInt(ACCOUNTS_CNT); + + if (accIdFrom == accIdTo) + accIdTo = (int)getNextAccountId(accIdFrom); + + Long acctFrom; + Long acctTo; + + try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + acctFrom = (Long)cache.get(accIdFrom); + acctTo = (Long)cache.get(accIdTo); + + long transactionAmount = (long)(random.nextDouble() * acctFrom); + + cache.put(accIdFrom, acctFrom - transactionAmount); + cache.put(accIdTo, acctTo + transactionAmount); + + tx.commit(); + } + } + + /** + * @param curr current + * @return random value + */ + private long getNextAccountId(long curr) { + long randomVal; + + do { + randomVal = random.nextInt(ACCOUNTS_CNT); + } + while (curr == randomVal); + + return randomVal; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java index cc93c5bfbb62e..a1a45dcda9295 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java @@ -20,6 +20,7 @@ import java.util.Set; import junit.framework.TestSuite; import org.apache.ignite.GridSuppressedExceptionSelfTest; +import org.apache.ignite.failure.AccountTransferTransactionTest; import org.apache.ignite.failure.FailureHandlerTriggeredTest; import org.apache.ignite.failure.IoomFailureHandlerTest; import org.apache.ignite.failure.OomFailureHandlerTest; @@ -207,6 +208,7 @@ public static TestSuite suite(@Nullable final Set ignoredTests) throws Ex suite.addTestSuite(StopNodeOrHaltFailureHandlerTest.class); suite.addTestSuite(IoomFailureHandlerTest.class); suite.addTestSuite(OomFailureHandlerTest.class); + suite.addTestSuite(AccountTransferTransactionTest.class); return suite; } From ced29ddd300a4f0bfbe471e1cc7bef311bbf84a3 Mon Sep 17 00:00:00 2001 From: Anton Kalashnikov Date: Tue, 8 May 2018 17:10:53 +0300 Subject: [PATCH 0151/1463] IGNITE-8430 implemented getCurrentCoordinatorFormatted method in IgniteMXBean - Fixes #3957. Signed-off-by: dpavlov (cherry picked from commit 666f220) --- .../apache/ignite/internal/IgniteKernal.java | 22 +++++++++++++++++++ .../apache/ignite/mxbean/IgniteMXBean.java | 6 +++++ .../ignite/util/mbeans/GridMBeanSelfTest.java | 16 +++++++++++--- 3 files changed, 41 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 8305d4ee239ed..046c4b8f84339 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 @@ -105,6 +105,7 @@ import org.apache.ignite.internal.binary.BinaryUtils; import org.apache.ignite.internal.cluster.ClusterGroupAdapter; import org.apache.ignite.internal.cluster.IgniteClusterEx; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.failure.FailureProcessor; import org.apache.ignite.internal.managers.GridManager; import org.apache.ignite.internal.managers.checkpoint.GridCheckpointManager; @@ -297,6 +298,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { /** Force complete reconnect future. */ private static final Object STOP_RECONNECT = new Object(); + /** Separator for formatted coordinator properties. */ + public static final String COORDINATOR_PROPERTIES_SEPARATOR = ","; + static { LongJVMPauseDetector.start(); } @@ -502,6 +506,24 @@ public IgniteKernal(@Nullable GridSpringResourceContext rsrcCtx) { return Arrays.toString(cfg.getCheckpointSpi()); } + /** {@inheritDoc} */ + @Override public String getCurrentCoordinatorFormatted() { + ClusterNode node = ctx.discovery().oldestAliveServerNode(AffinityTopologyVersion.NONE); + + if (node == null) + return ""; + + return new StringBuilder() + .append(node.addresses()) + .append(COORDINATOR_PROPERTIES_SEPARATOR) + .append(node.id()) + .append(COORDINATOR_PROPERTIES_SEPARATOR) + .append(node.order()) + .append(COORDINATOR_PROPERTIES_SEPARATOR) + .append(node.hostNames()) + .toString(); + } + /** {@inheritDoc} */ @Override public String getCommunicationSpiFormatted() { assert cfg != null; diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java index 77ea035309f79..cc4bc5b65f350 100644 --- a/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java +++ b/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java @@ -432,6 +432,12 @@ public interface IgniteMXBean { @MXBeanDescription("Dumps debug information for the current node.") public void dumpDebugInfo(); + /** + * Gets a formatted properties of current coordinator. + */ + @MXBeanDescription("Formatted properties of current coordinator.") + public String getCurrentCoordinatorFormatted(); + /** * Runs IO latency test against all remote server nodes in cluster. * diff --git a/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanSelfTest.java b/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanSelfTest.java index 4f5ed19cfacae..4e329f9fc9437 100644 --- a/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanSelfTest.java @@ -22,6 +22,7 @@ import javax.management.MBeanOperationInfo; import javax.management.MBeanParameterInfo; import javax.management.StandardMBean; +import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.mxbean.IgniteStandardMXBean; import org.apache.ignite.mxbean.IgniteMXBean; import org.apache.ignite.mxbean.MXBeanDescription; @@ -161,10 +162,19 @@ public void testEmptyNameMBeanInfo() throws Exception { */ public void testIgniteKernalReturnsValidMBeanInfo() throws Exception { try { - IgniteMXBean ignite = (IgniteMXBean)startGrid(); + IgniteEx igniteCrd = startGrid(0); - assertNotNull(ignite.getUserAttributesFormatted()); - assertNotNull(ignite.getLifecycleBeansFormatted()); + IgniteMXBean igniteMXBean = (IgniteMXBean)startGrid(1); + + assertNotNull(igniteMXBean.getUserAttributesFormatted()); + assertNotNull(igniteMXBean.getLifecycleBeansFormatted()); + + String coordinatorFormatted = igniteMXBean.getCurrentCoordinatorFormatted(); + + assertTrue(coordinatorFormatted.contains(igniteCrd.localNode().addresses().toString())); + assertTrue(coordinatorFormatted.contains(igniteCrd.localNode().hostNames().toString())); + assertTrue(coordinatorFormatted.contains(Long.toString(igniteCrd.localNode().order()))); + assertTrue(coordinatorFormatted.contains(igniteCrd.localNode().id().toString())); } finally { stopAllGrids(); From 10635667003bfb59f014b0b651787db972a67ab9 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Thu, 10 May 2018 10:37:36 +0300 Subject: [PATCH 0152/1463] IGNITE-7343: JDBC thin driver: mutex to avoid unpredictable behavior when connection is accessed from multiple threads. This closes #3337. --- .../jdbc/thin/JdbcThinConnectionSelfTest.java | 53 ++++++- .../jdbc/thin/JdbcThinConnection.java | 21 ++- .../internal/jdbc/thin/JdbcThinTcpIo.java | 137 ++++++++++++------ 3 files changed, 155 insertions(+), 56 deletions(-) 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 ed0b32403db3a..b462e4679fe03 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 @@ -33,8 +33,10 @@ import java.util.concurrent.Callable; import java.util.concurrent.Executor; import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; 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.jdbc.thin.JdbcThinConnection; import org.apache.ignite.internal.jdbc.thin.JdbcThinTcpIo; @@ -543,7 +545,7 @@ public void testClose() throws Exception { assert conn.isClosed(); - assert !conn.isValid(2): "Connection must be closed"; + assert !conn.isValid(2) : "Connection must be closed"; GridTestUtils.assertThrows(log, new Callable() { @Override public Object call() throws Exception { @@ -1212,7 +1214,6 @@ public void testClearGetWarnings() throws Exception { } }); - // Exception when called on closed connection checkConnectionClosed(new RunnableX() { @Override public void run() throws Exception { @@ -1929,6 +1930,54 @@ public void testAuthenticateDisableOnServerClientTryAuthenticate() { }, SQLException.class, "Can not perform the operation because the authentication is not enabled for the cluster"); } + /** + * @throws Exception If failed. + */ + public void testMultithreadingException() throws Exception { + int threadCnt = 10; + + final boolean end[] = new boolean[] {false}; + + final SQLException exs [] = new SQLException[threadCnt]; + + final AtomicInteger exCnt = new AtomicInteger(0); + + try (final Connection conn = DriverManager.getConnection(URL)) { + final IgniteInternalFuture f = GridTestUtils.runMultiThreadedAsync(new Runnable() { + @Override public void run() { + try { + conn.createStatement(); + + while (!end[0]) + conn.createStatement().execute("SELECT 1"); + + conn.createStatement().execute("SELECT 1"); + } + catch (SQLException e) { + end[0] = true; + exs[exCnt.getAndIncrement()] = e; + } + catch (Exception e) { + e.printStackTrace(System.err); + + fail("Unexpected exception (see details above): " + e.getMessage()); + } + } + }, threadCnt, "run-query"); + + f.get(); + + boolean exceptionFound = false; + + for (SQLException e : exs) { + if (e != null && e.getMessage().contains("Concurrent access to JDBC connection is not allowed")) + exceptionFound = true; + } + + assertTrue("Concurrent access to JDBC connection is not allowed", exceptionFound); + } + } + /** * @return Savepoint. */ 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 6bf768ba917fb..3478124ac1811 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 @@ -72,6 +72,9 @@ public class JdbcThinConnection implements Connection { /** Logger. */ private static final Logger LOG = Logger.getLogger(JdbcThinConnection.class.getName()); + /** Statements modification mutex. */ + final private Object stmtsMux = new Object(); + /** Schema name. */ private String schema; @@ -118,7 +121,7 @@ public class JdbcThinConnection implements Connection { private boolean connected; /** Tracked statements to close on disconnect. */ - private ArrayList stmts = new ArrayList<>(); + private final ArrayList stmts = new ArrayList<>(); /** * Creates new connection. @@ -268,7 +271,9 @@ private void executeBatch(boolean lastBatch) throws SQLException { if (timeout > 0) stmt.timeout(timeout); - stmts.add(stmt); + synchronized (stmtsMux) { + stmts.add(stmt); + } return stmt; } @@ -299,7 +304,9 @@ private void executeBatch(boolean lastBatch) throws SQLException { if (timeout > 0) stmt.timeout(timeout); - stmts.add(stmt); + synchronized (stmtsMux) { + stmts.add(stmt); + } return stmt; } @@ -812,10 +819,12 @@ private void onDisconnect() { lastStreamQry = null; - for (JdbcThinStatement s : stmts) - s.closeOnDisconnect(); + synchronized (stmtsMux) { + for (JdbcThinStatement s : stmts) + s.closeOnDisconnect(); - stmts.clear(); + stmts.clear(); + } } /** 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 bfd553d3b93fa..4631e5da713cb 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 @@ -114,6 +114,12 @@ public class JdbcThinTcpIo { /** Address index. */ private int srvIdx; + /** Ignite server version. */ + private Thread ownThread; + + /** Mutex. */ + private final Object mux = new Object(); + /** * Constructor. * @@ -140,66 +146,83 @@ public void start() throws SQLException, IOException { * @throws IOException On IO error in handshake. */ public void start(int timeout) throws SQLException, IOException { - List inaccessibleAddrs = null; + synchronized (mux) { + if (ownThread != null) { + throw new SQLException("Concurrent access to JDBC connection is not allowed" + + " [ownThread=" + ownThread.getName() + + ", curThread=" + Thread.currentThread().getName(), SqlStateCode.CLIENT_CONNECTION_FAILED); + } - List exceptions = null; + ownThread = Thread.currentThread(); + } - HostAndPortRange[] srvs = connProps.getAddresses(); + try { + List inaccessibleAddrs = null; - boolean connected = false; + List exceptions = null; - for (int i = 0; i < srvs.length; i++, srvIdx = (srvIdx + 1) % srvs.length) { - HostAndPortRange srv = srvs[srvIdx]; + HostAndPortRange[] srvs = connProps.getAddresses(); - InetAddress[] addrs = getAllAddressesByHost(srv.host()); + boolean connected = false; - for (InetAddress addr : addrs) { - for (int port = srv.portFrom(); port <= srv.portTo(); ++port) { - try { - connect(new InetSocketAddress(addr, port), timeout); + for (int i = 0; i < srvs.length; i++, srvIdx = (srvIdx + 1) % srvs.length) { + HostAndPortRange srv = srvs[srvIdx]; - connected = true; + InetAddress[] addrs = getAllAddressesByHost(srv.host()); - break; - } - catch (IOException | SQLException exception) { - if (inaccessibleAddrs == null) - inaccessibleAddrs = new ArrayList<>(); + for (InetAddress addr : addrs) { + for (int port = srv.portFrom(); port <= srv.portTo(); ++port) { + try { + connect(new InetSocketAddress(addr, port), timeout); - inaccessibleAddrs.add(addr.getHostName()); + connected = true; - if (exceptions == null) - exceptions = new ArrayList<>(); + break; + } + catch (IOException | SQLException exception) { + if (inaccessibleAddrs == null) + inaccessibleAddrs = new ArrayList<>(); - exceptions.add(exception); + inaccessibleAddrs.add(addr.getHostName()); + + if (exceptions == null) + exceptions = new ArrayList<>(); + + exceptions.add(exception); + } } } + + if (connected) + break; } - if (connected) - break; - } + if (!connected && inaccessibleAddrs != null && exceptions != null) { + if (exceptions.size() == 1) { + Exception ex = exceptions.get(0); - if (!connected && inaccessibleAddrs != null && exceptions != null) { - if (exceptions.size() == 1) { - Exception ex = exceptions.get(0); + if (ex instanceof SQLException) + throw (SQLException)ex; + else if (ex instanceof IOException) + throw (IOException)ex; + } - if (ex instanceof SQLException) - throw (SQLException)ex; - else if (ex instanceof IOException) - throw (IOException)ex; - } + SQLException e = new SQLException("Failed to connect to server [url=" + connProps.getUrl() + ']', + SqlStateCode.CLIENT_CONNECTION_FAILED); - SQLException e = new SQLException("Failed to connect to server [url=" + connProps.getUrl() + ']', - SqlStateCode.CLIENT_CONNECTION_FAILED); + for (Exception ex : exceptions) + e.addSuppressed(ex); - for (Exception ex : exceptions) - e.addSuppressed(ex); + throw e; + } - throw e; + handshake(CURRENT_VER); + } + finally { + synchronized (mux) { + ownThread = null; + } } - - handshake(CURRENT_VER); } /** @@ -398,24 +421,42 @@ private void handshake_2_1_0() throws IOException, SQLException { * @param req Request. * @return Server response. * @throws IOException In case of IO error. + * @throws SQLException On concurrent access to JDBC connection. */ @SuppressWarnings("unchecked") - JdbcResponse sendRequest(JdbcRequest req) throws IOException { - int cap = guessCapacity(req); + JdbcResponse sendRequest(JdbcRequest req) throws SQLException, IOException { + synchronized (mux) { + if (ownThread != null) { + throw new SQLException("Concurrent access to JDBC connection is not allowed" + + " [ownThread=" + ownThread.getName() + + ", curThread=" + Thread.currentThread().getName(), SqlStateCode.CONNECTION_FAILURE); + } - BinaryWriterExImpl writer = new BinaryWriterExImpl(null, new BinaryHeapOutputStream(cap), null, null); + ownThread = Thread.currentThread(); + } - req.writeBinary(writer); + try { + int cap = guessCapacity(req); - send(writer.array()); + BinaryWriterExImpl writer = new BinaryWriterExImpl(null, new BinaryHeapOutputStream(cap), null, null); + + req.writeBinary(writer); + + send(writer.array()); - BinaryReaderExImpl reader = new BinaryReaderExImpl(null, new BinaryHeapInputStream(read()), null, null, false); + BinaryReaderExImpl reader = new BinaryReaderExImpl(null, new BinaryHeapInputStream(read()), null, null, false); - JdbcResponse res = new JdbcResponse(); + JdbcResponse res = new JdbcResponse(); - res.readBinary(reader); + res.readBinary(reader); - return res; + return res; + } + finally { + synchronized (mux) { + ownThread = null; + } + } } /** From d894b234e918601fe3084eba947d2b640d5b7449 Mon Sep 17 00:00:00 2001 From: ascherbakoff Date: Thu, 10 May 2018 12:08:17 +0300 Subject: [PATCH 0153/1463] IGNITE-8437: Control utility fails to connect to cluster if zookeeper discovery used. This closes #3950. (cherry picked from commit 3cdee53) --- modules/zookeeper/pom.xml | 49 +++++++++++++++++++ .../resources/META-INF/classnames.properties | 18 +++++++ .../zk/ZookeeperDiscoverySpiTestSuite2.java | 4 +- 3 files changed, 70 insertions(+), 1 deletion(-) create mode 100644 modules/zookeeper/src/main/resources/META-INF/classnames.properties diff --git a/modules/zookeeper/pom.xml b/modules/zookeeper/pom.xml index 2d47ece408719..0777f1e36edbd 100644 --- a/modules/zookeeper/pom.xml +++ b/modules/zookeeper/pom.xml @@ -172,6 +172,55 @@ + + org.codehaus.mojo + exec-maven-plugin + 1.3.2 + + + org.apache.ignite + ignite-tools + ${project.version} + + + + + process-classes + + java + + + true + org.apache.ignite.tools.classgen.ClassesGenerator + + ${project.basedir}/target/classes + + + + org.apache.ignite + + + + + + org.apache.felix diff --git a/modules/zookeeper/src/main/resources/META-INF/classnames.properties b/modules/zookeeper/src/main/resources/META-INF/classnames.properties new file mode 100644 index 0000000000000..34e842cb4e56d --- /dev/null +++ b/modules/zookeeper/src/main/resources/META-INF/classnames.properties @@ -0,0 +1,18 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +org.apache.ignite.spi.discovery.zk.internal.ZookeeperClusterNode \ No newline at end of file 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 3775aa1df9b3d..ddb003bec5366 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 @@ -21,7 +21,6 @@ import org.apache.curator.test.TestingCluster; import org.apache.ignite.internal.ClusterNodeMetricsUpdateTest; import org.apache.ignite.internal.IgniteClientReconnectCacheTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheEntryListenerAtomicTest; import org.apache.ignite.internal.processors.cache.datastructures.IgniteClientDataStructuresTest; import org.apache.ignite.internal.processors.cache.datastructures.partitioned.GridCachePartitionedNodeRestartTxSelfTest; import org.apache.ignite.internal.processors.cache.datastructures.partitioned.GridCachePartitionedSequenceApiSelfTest; @@ -38,6 +37,7 @@ import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicMultiJvmFullApiSelfTest; import org.apache.ignite.internal.processors.cache.multijvm.GridCachePartitionedMultiJvmFullApiSelfTest; import org.apache.ignite.internal.processors.continuous.GridEventConsumeSelfTest; +import org.apache.ignite.util.GridCommandHandlerTest; /** * Regular Ignite tests executed with {@link org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi}. @@ -89,6 +89,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridCacheAtomicMultiJvmFullApiSelfTest.class); suite.addTestSuite(GridCachePartitionedMultiJvmFullApiSelfTest.class); + suite.addTestSuite(GridCommandHandlerTest.class); + return suite; } } From 634a5a96937fbc8b0615617241d18145af59bf94 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Thu, 10 May 2018 12:05:37 +0300 Subject: [PATCH 0154/1463] IGNITE-8422: Zookeeper discovery split brain detection shouldn't consider client nodes. This closes #3951. (cherry picked from commit 534fcec) --- .../DefaultCommunicationFailureResolver.java | 367 ++++++++--------- .../cluster/graph/BitSetIterator.java | 66 +++ .../internal/cluster/graph/ClusterGraph.java | 207 ++++++++++ .../FullyConnectedComponentSearcher.java | 341 ++++++++++++++++ .../tcp/TcpCommunicationSpi.java | 82 ++-- .../FullyConnectedComponentSearcherTest.java | 323 +++++++++++++++ .../internal/ZookeeperDiscoverySpiTest.java | 379 ++++++++++++++++++ 7 files changed, 1539 insertions(+), 226 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/cluster/graph/BitSetIterator.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/cluster/graph/ClusterGraph.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/cluster/graph/FullyConnectedComponentSearcher.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/cluster/FullyConnectedComponentSearcherTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/DefaultCommunicationFailureResolver.java b/modules/core/src/main/java/org/apache/ignite/configuration/DefaultCommunicationFailureResolver.java index a4c6da9e9986b..9ccadf39b2933 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/DefaultCommunicationFailureResolver.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/DefaultCommunicationFailureResolver.java @@ -18,13 +18,21 @@ package org.apache.ignite.configuration; import java.util.BitSet; +import java.util.HashSet; +import java.util.Iterator; import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.cluster.graph.BitSetIterator; +import org.apache.ignite.internal.cluster.graph.ClusterGraph; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.resources.LoggerResource; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; /** * Default Communication Failure Resolver. @@ -36,266 +44,239 @@ public class DefaultCommunicationFailureResolver implements CommunicationFailure /** {@inheritDoc} */ @Override public void resolve(CommunicationFailureContext ctx) { - ClusterGraph graph = new ClusterGraph(log, ctx); + ClusterPart largestCluster = findLargestConnectedCluster(ctx); - ClusterSearch cluster = graph.findLargestIndependentCluster(); + if (largestCluster == null) + return; - List nodes = ctx.topologySnapshot(); + log.info("Communication problem resolver found fully connected independent cluster [" + + "serverNodesCnt=" + largestCluster.srvNodesCnt + ", " + + "clientNodesCnt=" + largestCluster.connectedClients.size() + ", " + + "totalAliveNodes=" + ctx.topologySnapshot().size() + ", " + + "serverNodesIds=" + clusterNodeIds(largestCluster.srvNodesSet, ctx.topologySnapshot(), 1000) + "]"); - assert nodes.size() > 0; - assert cluster != null; - - if (graph.checkFullyConnected(cluster.nodesBitSet)) { - assert cluster.nodeCnt <= nodes.size(); - - if (cluster.nodeCnt < nodes.size()) { - if (log.isInfoEnabled()) { - log.info("Communication problem resolver found fully connected independent cluster [" + - "clusterSrvCnt=" + cluster.srvCnt + - ", clusterTotalNodes=" + cluster.nodeCnt + - ", totalAliveNodes=" + nodes.size() + "]"); - } - - for (int i = 0; i < nodes.size(); i++) { - if (!cluster.nodesBitSet.get(i)) - ctx.killNode(nodes.get(i)); - } - } - else - U.warn(log, "All alive nodes are fully connected, this should be resolved automatically."); - } - else { - if (log.isInfoEnabled()) { - log.info("Communication problem resolver failed to find fully connected independent cluster."); - } - } + keepCluster(ctx, largestCluster); } /** - * @param cluster Cluster nodes mask. - * @param nodes Nodes. - * @param limit IDs limit. - * @return Cluster node IDs string. + * Finds largest part of the cluster where each node is able to connect to each other. + * + * @param ctx Communication failure context. + * @return Largest part of the cluster nodes to keep. */ - private static String clusterNodeIds(BitSet cluster, List nodes, int limit) { - int startIdx = 0; + @Nullable private ClusterPart findLargestConnectedCluster(CommunicationFailureContext ctx) { + List srvNodes = ctx.topologySnapshot() + .stream() + .filter(node -> !CU.clientNode(node)) + .collect(Collectors.toList()); - StringBuilder builder = new StringBuilder(); + // Exclude client nodes from analysis. + ClusterGraph graph = new ClusterGraph(ctx, CU::clientNode); - int cnt = 0; + List components = graph.findConnectedComponents(); - for (;;) { - int idx = cluster.nextSetBit(startIdx); + if (components.isEmpty()) { + U.warn(log, "Unable to find at least one alive server node in the cluster " + ctx); - if (idx == -1) - break; + return null; + } - startIdx = idx + 1; + if (components.size() == 1) { + BitSet nodesSet = components.get(0); + int nodeCnt = nodesSet.cardinality(); - if (builder.length() == 0) { - builder.append('['); + boolean fullyConnected = graph.checkFullyConnected(nodesSet); + + if (fullyConnected && nodeCnt == srvNodes.size()) { + U.warn(log, "All alive nodes are fully connected, this should be resolved automatically."); + + return null; } - else - builder.append(", "); - builder.append(nodes.get(idx).id()); + if (log.isInfoEnabled()) + log.info("Communication problem resolver detected partial lost for some connections inside cluster. " + + "Will keep largest set of healthy fully-connected nodes. Other nodes will be killed forcibly."); - if (cnt++ > limit) - builder.append(", ..."); + BitSet fullyConnectedPart = graph.findLargestFullyConnectedComponent(nodesSet); + Set connectedClients = findConnectedClients(ctx, fullyConnectedPart); + + return new ClusterPart(fullyConnectedPart, connectedClients); } - builder.append(']'); + // If cluster has splitted on several parts and there are at least 2 parts which aren't single node + // It means that split brain has happened. + boolean isSplitBrain = components.size() > 1 && + components.stream().filter(cmp -> cmp.size() > 1).count() > 1; - return builder.toString(); - } + if (isSplitBrain) + U.warn(log, "Communication problem resolver detected split brain. " + + "Cluster has splitted on " + components.size() + " independent parts. " + + "Will keep only one largest fully-connected part. " + + "Other nodes will be killed forcibly."); + else + U.warn(log, "Communication problem resolver detected full lost for some connections inside cluster. " + + "Problem nodes will be found and killed forcibly."); - /** - * - */ - private static class ClusterSearch { - /** */ - int srvCnt; + // For each part of splitted cluster extract largest fully-connected component. + ClusterPart largestCluster = null; + for (int i = 0; i < components.size(); i++) { + BitSet clusterPart = components.get(i); - /** */ - int nodeCnt; + BitSet fullyConnectedPart = graph.findLargestFullyConnectedComponent(clusterPart); + Set connectedClients = findConnectedClients(ctx, fullyConnectedPart); - /** */ - final BitSet nodesBitSet; + ClusterPart curr = new ClusterPart(fullyConnectedPart, connectedClients); - /** - * @param nodes Total nodes. - */ - ClusterSearch(int nodes) { - nodesBitSet = new BitSet(nodes); + if (largestCluster == null || curr.compareTo(largestCluster) > 0) + largestCluster = curr; } + + assert largestCluster != null + : "Unable to find at least one alive independent cluster."; + + return largestCluster; } /** + * Keeps server cluster nodes presented in given {@code srvNodesSet}. + * Client nodes which have connections to presented {@code srvNodesSet} will be also keeped. + * Other nodes will be killed forcibly. * + * @param ctx Communication failure context. + * @param clusterPart Set of nodes need to keep in the cluster. */ - private static class ClusterGraph { - /** */ - private final static int WORD_IDX_SHIFT = 6; - - /** */ - private final IgniteLogger log; - - /** */ - private final int nodeCnt; - - /** */ - private final long[] visitBitSet; - - /** */ - private final CommunicationFailureContext ctx; - - /** */ - private final List nodes; + private void keepCluster(CommunicationFailureContext ctx, ClusterPart clusterPart) { + List allNodes = ctx.topologySnapshot(); - /** - * @param log Logger. - * @param ctx Context. - */ - ClusterGraph(IgniteLogger log, CommunicationFailureContext ctx) { - this.log = log; - this.ctx = ctx; + // Kill server nodes. + for (int idx = 0; idx < allNodes.size(); idx++) { + ClusterNode node = allNodes.get(idx); - nodes = ctx.topologySnapshot(); + // Client nodes will be processed separately. + if (CU.clientNode(node)) + continue; - nodeCnt = nodes.size(); + if (!clusterPart.srvNodesSet.get(idx)) + ctx.killNode(node); + } - assert nodeCnt > 0; + // Kill client nodes unable to connect to the presented part of cluster. + for (int idx = 0; idx < allNodes.size(); idx++) { + ClusterNode node = allNodes.get(idx); - visitBitSet = initBitSet(nodeCnt); + if (CU.clientNode(node) && !clusterPart.connectedClients.contains(node)) + ctx.killNode(node); } + } - /** - * @param bitIndex Bit index. - * @return Word index containing bit with given index. - */ - private static int wordIndex(int bitIndex) { - return bitIndex >> WORD_IDX_SHIFT; - } + /** + * Finds set of the client nodes which are able to connect to given set of server nodes {@code srvNodesSet}. + * + * @param ctx Communication failure context. + * @param srvNodesSet Server nodes set. + * @return Set of client nodes. + */ + private Set findConnectedClients(CommunicationFailureContext ctx, BitSet srvNodesSet) { + Set connectedClients = new HashSet<>(); - /** - * @param bitCnt Number of bits. - * @return Bit set words. - */ - static long[] initBitSet(int bitCnt) { - return new long[wordIndex(bitCnt - 1) + 1]; - } + List allNodes = ctx.topologySnapshot(); - /** - * @return Cluster nodes bit set. - */ - ClusterSearch findLargestIndependentCluster() { - ClusterSearch maxCluster = null; + for (ClusterNode node : allNodes) { + if (!CU.clientNode(node)) + continue; - for (int i = 0; i < nodeCnt; i++) { - if (getBit(visitBitSet, i)) - continue; + boolean hasConnections = true; - ClusterSearch cluster = new ClusterSearch(nodeCnt); + Iterator it = new BitSetIterator(srvNodesSet); + while (it.hasNext()) { + int srvNodeIdx = it.next(); + ClusterNode srvNode = allNodes.get(srvNodeIdx); - search(cluster, i); + if (!ctx.connectionAvailable(node, srvNode) || !ctx.connectionAvailable(srvNode, node)) { + hasConnections = false; - if (log.isInfoEnabled()) { - log.info("Communication problem resolver found cluster [srvCnt=" + cluster.srvCnt + - ", totalNodeCnt=" + cluster.nodeCnt + - ", nodeIds=" + clusterNodeIds(cluster.nodesBitSet, nodes, 1000) + "]"); + break; } - - if (maxCluster == null || cluster.srvCnt > maxCluster.srvCnt) - maxCluster = cluster; } - return maxCluster; + if (hasConnections) + connectedClients.add(node); } - /** - * @param cluster Cluster nodes bit set. - * @return {@code True} if all cluster nodes are able to connect to each other. - */ - boolean checkFullyConnected(BitSet cluster) { - int startIdx = 0; - - int clusterNodes = cluster.cardinality(); - - for (;;) { - int idx = cluster.nextSetBit(startIdx); + return connectedClients; + } - if (idx == -1) - break; + /** + * Class representing part of cluster. + */ + private static class ClusterPart implements Comparable { + /** Server nodes count. */ + int srvNodesCnt; - ClusterNode node1 = nodes.get(idx); + /** Server nodes set. */ + BitSet srvNodesSet; - for (int i = 0; i < clusterNodes; i++) { - if (!cluster.get(i) || i == idx) - continue; + /** Set of client nodes are able to connect to presented part of server nodes. */ + Set connectedClients; - ClusterNode node2 = nodes.get(i); + /** + * Constructor. + * + * @param srvNodesSet Server nodes set. + * @param connectedClients Set of client nodes. + */ + public ClusterPart(BitSet srvNodesSet, Set connectedClients) { + this.srvNodesSet = srvNodesSet; + this.srvNodesCnt = srvNodesSet.cardinality(); + this.connectedClients = connectedClients; + } - if (cluster.get(i) && !ctx.connectionAvailable(node1, node2)) - return false; - } + /** {@inheritDoc} */ + @Override public int compareTo(@NotNull ClusterPart o) { + int srvNodesCmp = Integer.compare(srvNodesCnt, o.srvNodesCnt); - startIdx = idx + 1; - } + if (srvNodesCmp != 0) + return srvNodesCmp; - return true; + return Integer.compare(connectedClients.size(), o.connectedClients.size()); } + } - /** - * @param cluster Current cluster bit set. - * @param idx Node index. - */ - void search(ClusterSearch cluster, int idx) { - assert !getBit(visitBitSet, idx); - - setBit(visitBitSet, idx); - - cluster.nodesBitSet.set(idx); - cluster.nodeCnt++; + /** + * @param cluster Cluster nodes mask. + * @param nodes Nodes. + * @param limit IDs limit. + * @return Cluster node IDs string. + */ + private static String clusterNodeIds(BitSet cluster, List nodes, int limit) { + int startIdx = 0; - ClusterNode node1 = nodes.get(idx); + StringBuilder builder = new StringBuilder(); - if (!CU.clientNode(node1)) - cluster.srvCnt++; + int cnt = 0; - for (int i = 0; i < nodeCnt; i++) { - if (i == idx || getBit(visitBitSet, i)) - continue; + for (;;) { + int idx = cluster.nextSetBit(startIdx); - ClusterNode node2 = nodes.get(i); + if (idx == -1) + break; - boolean connected = ctx.connectionAvailable(node1, node2) || - ctx.connectionAvailable(node2, node1); + startIdx = idx + 1; - if (connected) - search(cluster, i); - } - } + if (builder.length() == 0) + builder.append('['); + else + builder.append(", "); - /** - * @param words Bit set words. - * @param bitIndex Bit index. - */ - static void setBit(long words[], int bitIndex) { - int wordIndex = wordIndex(bitIndex); + builder.append(nodes.get(idx).id()); - words[wordIndex] |= (1L << bitIndex); + if (cnt++ > limit) + builder.append(", ..."); } - /** - * @param words Bit set words. - * @param bitIndex Bit index. - * @return Bit value. - */ - static boolean getBit(long[] words, int bitIndex) { - int wordIndex = wordIndex(bitIndex); + builder.append(']'); - return (words[wordIndex] & (1L << bitIndex)) != 0; - } + return builder.toString(); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cluster/graph/BitSetIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/cluster/graph/BitSetIterator.java new file mode 100644 index 0000000000000..3a5cf9f21ea70 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/cluster/graph/BitSetIterator.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.cluster.graph; + +import java.util.BitSet; +import java.util.Iterator; +import java.util.NoSuchElementException; + +/** + * Iterator over set bits in {@link BitSet}. + */ +public class BitSetIterator implements Iterator { + /** Bitset. */ + private final BitSet bitSet; + + /** Current index. */ + private int idx = -1; + + /** + * @param bitSet Bitset. + */ + public BitSetIterator(BitSet bitSet) { + this.bitSet = bitSet; + + advance(); + } + + /** + * Find index of the next set bit. + */ + private void advance() { + idx = bitSet.nextSetBit(idx + 1); + } + + /** {@inheritDoc} */ + @Override public boolean hasNext() { + return idx != -1; + } + + /** {@inheritDoc} */ + @Override public Integer next() throws NoSuchElementException { + if (idx == -1) + throw new NoSuchElementException(); + + int res = idx; + + advance(); + + return res; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cluster/graph/ClusterGraph.java b/modules/core/src/main/java/org/apache/ignite/internal/cluster/graph/ClusterGraph.java new file mode 100644 index 0000000000000..ba56c3386e34e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/cluster/graph/ClusterGraph.java @@ -0,0 +1,207 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.cluster.graph; + +import java.util.ArrayList; +import java.util.BitSet; +import java.util.Iterator; +import java.util.List; +import java.util.function.Predicate; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CommunicationFailureContext; + +/** + * Class to represent cluster nodes avalaible connections as graph. + * Provides several graph algorithms to analyze cluster nodes connections. + */ +public class ClusterGraph { + /** Number of all cluster nodes. */ + private final int nodeCnt; + + /** List of the all cluster nodes. */ + private final List nodes; + + /** Connectivity (adjacency) matrix between cluster nodes. */ + private final BitSet[] connections; + + /** Fully-connected component searcher. */ + private final FullyConnectedComponentSearcher fccSearcher; + + /** + * Constructor. + * + * @param ctx Communication failure context. + * @param nodeFilterOut Filter to exclude some cluster nodes from graph. + */ + public ClusterGraph(CommunicationFailureContext ctx, Predicate nodeFilterOut) { + nodes = ctx.topologySnapshot(); + + nodeCnt = nodes.size(); + + assert nodeCnt > 0; + + connections = buildConnectivityMatrix(ctx, nodeFilterOut); + + fccSearcher = new FullyConnectedComponentSearcher(connections); + } + + /** + * Builds connectivity matrix (adjacency matrix) for all cluster nodes. + * + * @param ctx Communication failure context. + * @param nodeFilterOut Filter to exclude some cluster nodes from graph. + * @return Connections bit set for each node, where set bit means avalable connection. + */ + private BitSet[] buildConnectivityMatrix(CommunicationFailureContext ctx, Predicate nodeFilterOut) { + BitSet[] connections = new BitSet[nodeCnt]; + + for (int i = 0; i < nodeCnt; i++) { + ClusterNode node = nodes.get(i); + + if (nodeFilterOut.test(node)) { + connections[i] = null; + continue; + } + + connections[i] = new BitSet(nodeCnt); + for (int j = 0; j < nodeCnt; j++) { + ClusterNode to = nodes.get(j); + + if (nodeFilterOut.test(to)) + continue; + + if (i == j || ctx.connectionAvailable(node, to)) + connections[i].set(j); + } + } + + // Remove unidirectional connections (node A can connect to B, but B can't connect to A). + for (int i = 0; i < nodeCnt; i++) + for (int j = i + 1; j < nodeCnt; j++) { + if (connections[i] == null || connections[j] == null) + continue; + + if (connections[i].get(j) ^ connections[j].get(i)) { + connections[i].set(j, false); + connections[j].set(i, false); + } + } + + return connections; + } + + /** + * Finds connected components in cluster graph. + * + * @return List of set of nodes, each set represents connected component. + */ + public List findConnectedComponents() { + List connectedComponets = new ArrayList<>(); + + BitSet visitSet = new BitSet(nodeCnt); + + for (int i = 0; i < nodeCnt; i++) { + if (visitSet.get(i) || connections[i] == null) + continue; + + BitSet currComponent = new BitSet(nodeCnt); + + dfs(i, currComponent, visitSet); + + connectedComponets.add(currComponent); + } + + return connectedComponets; + } + + /** + * Deep-first search to find connected components in connections graph. + * + * @param nodeIdx Current node index to traverse from. + * @param currComponent Current connected component to populate. + * @param allVisitSet Set of the visited nodes in whole graph during traversal. + */ + private void dfs(int nodeIdx, BitSet currComponent, BitSet allVisitSet) { + assert !allVisitSet.get(nodeIdx) + : "Incorrect node visit " + nodeIdx; + + assert connections[nodeIdx] != null + : "Incorrect node visit. Node has not passed filter " + nodes.get(nodeIdx); + + allVisitSet.set(nodeIdx); + + currComponent.set(nodeIdx); + + for (int toIdx = 0; toIdx < nodeCnt; toIdx++) { + if (toIdx == nodeIdx || allVisitSet.get(toIdx) || connections[toIdx] == null) + continue; + + boolean connected = connections[nodeIdx].get(toIdx) && connections[toIdx].get(nodeIdx); + + if (connected) + dfs(toIdx, currComponent, allVisitSet); + } + } + + /** + * Finds largest fully-connected component from given {@code nodesSet}. + * + * @param nodesSet Set of nodes. + * @return Set of nodes which forms largest fully-connected component. + */ + public BitSet findLargestFullyConnectedComponent(BitSet nodesSet) { + // Check that current set is already fully connected. + boolean fullyConnected = checkFullyConnected(nodesSet); + + if (fullyConnected) + return nodesSet; + + BitSet res = fccSearcher.findLargest(nodesSet); + + assert checkFullyConnected(res) + : "Not fully connected component was found [result=" + res + ", nodesSet=" + nodesSet + "]"; + + return res; + } + + /** + * Checks that given {@code nodesSet} forms fully-connected component. + * + * @param nodesSet Set of cluster nodes. + * @return {@code True} if all given cluster nodes are able to connect to each other. + */ + public boolean checkFullyConnected(BitSet nodesSet) { + int maxIdx = nodesSet.length(); + + Iterator it = new BitSetIterator(nodesSet); + + while (it.hasNext()) { + int idx = it.next(); + + for (int i = 0; i < maxIdx; i++) { + if (i == idx) + continue; + + if (nodesSet.get(i) && !connections[idx].get(i)) + return false; + } + } + + return true; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cluster/graph/FullyConnectedComponentSearcher.java b/modules/core/src/main/java/org/apache/ignite/internal/cluster/graph/FullyConnectedComponentSearcher.java new file mode 100644 index 0000000000000..9a8098eb3de70 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/cluster/graph/FullyConnectedComponentSearcher.java @@ -0,0 +1,341 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.cluster.graph; + +import java.util.Arrays; +import java.util.BitSet; +import java.util.Comparator; +import java.util.Iterator; + +/** + * Class to find (possibly) largest fully-connected component (also can be called as complete subgraph) in graph. + * This problem is also known as Clique problem which is NP-complete. + * + * For small number of nodes simple brute-force algorithm is used which finds such component guaranteed. + * For large number of nodes some sort of greedy heuristic is used which works well for real-life scenarios + * but doesn't guarantee to find largest component, however very close to ideal result. + */ +public class FullyConnectedComponentSearcher { + /** The maximal number of nodes when bruteforce algorithm will be used. */ + private static final int BRUTE_FORCE_THRESHOULD = 24; + + /** Number of nodes in connections graph. */ + private final int totalNodesCnt; + + /** Adjacency matrix. */ + private final BitSet[] connections; + + /** + * Constructor. + * + * @param connections Adjacency matrix. + */ + public FullyConnectedComponentSearcher(BitSet[] connections) { + this.connections = connections; + totalNodesCnt = connections.length; + } + + /** + * Find largest fully connected component from presented set of the nodes {@code where}. + * + * @param where Set of nodes where fully connected component must be found. + * @return Set of nodes forming fully connected component. + */ + public BitSet findLargest(BitSet where) { + int nodesCnt = where.cardinality(); + + if (nodesCnt <= BRUTE_FORCE_THRESHOULD) + return bruteforce(nodesCnt, where); + + // Return best of the 2 heuristics. + BitSet e1 = heuristic1(where); + BitSet e2 = heuristic2(where); + + return e1.cardinality() > e2.cardinality() ? e1 : e2; + } + + /** + * Extract node indexes (set bits) from given {@code selectedSet} to integer array. + * + * @param selectedNodesCnt Number of nodes. + * @param selectedSet Set of nodes. + * @return Arrays which contains node indexes. + */ + private Integer[] extractNodeIndexes(int selectedNodesCnt, BitSet selectedSet) { + Integer[] indexes = new Integer[selectedNodesCnt]; + Iterator it = new BitSetIterator(selectedSet); + int i = 0; + + while (it.hasNext()) + indexes[i++] = it.next(); + + assert i == indexes.length + : "Extracted not all indexes [nodesCnt=" + selectedNodesCnt + ", extracted=" + i + ", set=" + selectedSet + "]"; + + return indexes; + } + + /** + * Sorts nodes using {@link ConnectionsComparator} + * and runs greedy algorithm {@link #greedyIterative(int, Integer[])} on it. + * + * @param selectedSet Set of nodes used to form fully-connected component. + * @return Subset of given {@code selectedSet} which forms fully connected component. + */ + private BitSet heuristic1(BitSet selectedSet) { + int selectedNodesCnt = selectedSet.cardinality(); + Integer[] nodeIndexes = extractNodeIndexes(selectedNodesCnt, selectedSet); + + Arrays.sort(nodeIndexes, new ConnectionsComparator(totalNodesCnt)); + + return greedyIterative(selectedNodesCnt, nodeIndexes); + } + + /** + * Exactly the same thing as in {@link #heuristic1(BitSet)} but using reversed {@link ConnectionsComparator}. + * + * @param selectedSet Set of nodes used to form fully-connected component. + * @return Subset of given {@code selectedSet} which forms fully connected component. + */ + private BitSet heuristic2(BitSet selectedSet) { + int selectedNodesCnt = selectedSet.cardinality(); + Integer[] nodeIndexes = extractNodeIndexes(selectedNodesCnt, selectedSet); + + Arrays.sort(nodeIndexes, new ConnectionsComparator(totalNodesCnt).reversed()); + + return greedyIterative(selectedNodesCnt, nodeIndexes); + } + + /** + * Finds fully-connected component between given {@code nodeIndexes} and tries to maximize size of it. + * + * The main idea of the algorithm is that after specific sorting, + * nodes able to form fully-connected will be placed closer to each other in given {@code nodeIndexes} array. + * While nodes not able to form will be placed further. + * + * At the begging of algorithm we form global set of nodes can be used to form fully-connected component. + * We iterate over this set and try to add each node to current fully-connected component, which is empty at the beginning. + * + * When we add node to the component we need to check that after adding new component is also fully-connected. + * See {@link #joinNode(BitSet, int, Integer[])}. + * + * After end of iteration we exclude nodes which formed fully-connected from the global set and run iteration again and again + * on remaining nodes, while the global set will not be empty. + * + * Complexity is O(N^2), where N is number of nodes. + * + * @param selectedNodesCnt Number of nodes. + * @param nodeIndexes Node indexes used to form fully-connected component. + * @return Subset of given {@code nodeIndexes} which forms fully connected component. + */ + private BitSet greedyIterative(int selectedNodesCnt, Integer[] nodeIndexes) { + // Set of the nodes which can be used to form fully connected component. + BitSet canUse = new BitSet(selectedNodesCnt); + for (int i = 0; i < selectedNodesCnt; i++) + canUse.set(i); + + BitSet bestRes = null; + + while (!canUse.isEmpty()) { + // Even if we pick all possible nodes, their size will not be greater than current best result. + // No needs to run next iteration in this case. + if (bestRes != null && canUse.cardinality() <= bestRes.cardinality()) + break; + + BitSet currRes = new BitSet(selectedNodesCnt); + + Iterator canUseIter = new BitSetIterator(canUse); + while (canUseIter.hasNext()) { + /* Try to add node to the current set that forms fully connected component. + Node will be skipped if after adding, current set loose fully connectivity. */ + int pickedIdx = canUseIter.next(); + + if (joinNode(currRes, pickedIdx, nodeIndexes)) { + currRes.set(pickedIdx); + canUse.set(pickedIdx, false); + } + } + + if (bestRes == null || currRes.cardinality() > bestRes.cardinality()) + bestRes = currRes; + } + + // Try to improve our best result, if it was formed on second or next iteration. + for (int nodeIdx = 0; nodeIdx < selectedNodesCnt; nodeIdx++) + if (!bestRes.get(nodeIdx) && joinNode(bestRes, nodeIdx, nodeIndexes)) + bestRes.set(nodeIdx); + + // Replace relative node indexes (used in indexes) to absolute node indexes (used in whole graph connections). + BitSet reindexedBestRes = new BitSet(totalNodesCnt); + Iterator it = new BitSetIterator(bestRes); + while (it.hasNext()) + reindexedBestRes.set(nodeIndexes[it.next()]); + + return reindexedBestRes; + } + + /** + * Checks that given {@code nodeIdx} can be joined to current fully-connected component, + * so after join result component will be also fully-connected. + * + * @param currComponent Current fully-connected component. + * @param nodeIdx Node relative index. + * @param nodeIndexes Node absolute indexes. + * @return {@code True} if given node can be joined to {@code currentComponent}. + */ + private boolean joinNode(BitSet currComponent, int nodeIdx, Integer[] nodeIndexes) { + boolean fullyConnected = true; + + Iterator alreadyUsedIter = new BitSetIterator(currComponent); + while (alreadyUsedIter.hasNext()) { + int existedIdx = alreadyUsedIter.next(); + + // If no connection between existing node and picked node, skip picked node. + if (!connections[nodeIndexes[nodeIdx]].get(nodeIndexes[existedIdx])) { + fullyConnected = false; + + break; + } + } + + return fullyConnected; + } + + /** + * Simple bruteforce implementation which works in O(2^N * N^2), where N is number of nodes. + * + * @param selectedNodesCnt Nodes count. + * @param selectedSet Set of nodes. + * @return Subset of given {@code set} of nodes which forms fully connected component. + */ + private BitSet bruteforce(int selectedNodesCnt, BitSet selectedSet) { + Integer[] indexes = extractNodeIndexes(selectedNodesCnt, selectedSet); + + int resMask = -1; + int maxCardinality = -1; + + // Iterate over all possible combinations of used nodes. + for (int mask = (1 << selectedNodesCnt) - 1; mask > 0; mask--) { + int cardinality = Integer.bitCount(mask); + + if (cardinality <= maxCardinality) + continue; + + // Check that selected set of nodes forms fully connected component. + boolean fullyConnected = true; + + for (int i = 0; i < selectedNodesCnt && fullyConnected; i++) + if ((mask & (1 << i)) != 0) + for (int j = 0; j < selectedNodesCnt; j++) + if ((mask & (1 << j)) != 0) { + boolean connected = connections[indexes[i]].get(indexes[j]); + + if (!connected) { + fullyConnected = false; + + break; + } + } + + if (fullyConnected) { + resMask = mask; + maxCardinality = cardinality; + } + } + + BitSet resSet = new BitSet(selectedNodesCnt); + + for (int i = 0; i < selectedNodesCnt; i++) { + if ((resMask & (1 << i)) != 0) { + int idx = indexes[i]; + + assert selectedSet.get(idx) + : "Result contains node which is not presented in income set [nodeIdx" + idx + ", set=" + selectedSet + "]"; + + resSet.set(idx); + } + } + + assert resSet.cardinality() > 0 + : "No nodes selected as fully connected component [set=" + selectedSet + "]"; + + return resSet; + } + + /** + * Comparator which sorts nodes by their connections array. + * + * Suppose you have connections matrix: + * + * 1111 + * 1101 + * 1010 + * 1101 + * + * Each connection row associated with some node. + * Comparator will sort node indexes using their connection rows as very big binary numbers, as in example: + * + * 1111 + * 1101 + * 1101 + * 1011 + * + * Note: Comparator sorts only node indexes, actual connection rows swapping will be not happened. + */ + private class ConnectionsComparator implements Comparator { + /** Cache each connection long array representation, to avoid creating new object for each comparison. */ + private final long[][] cachedConnRows; + + /** + * Constructor + * @param totalNodesCnt Total number of nodes in the cluster. + */ + ConnectionsComparator(int totalNodesCnt) { + cachedConnRows = new long[totalNodesCnt][]; + } + + /** + * Returns long array representation of connection row for given node {@code idx}. + * + * @param idx Node index. + * @return Long array connection row representation. + */ + private long[] connectionRow(int idx) { + if (cachedConnRows[idx] != null) + return cachedConnRows[idx]; + + return cachedConnRows[idx] = connections[idx].toLongArray(); + } + + /** {@inheritDoc} */ + @Override public int compare(Integer node1, Integer node2) { + long[] conn1 = connectionRow(node1); + long[] conn2 = connectionRow(node2); + + int len = Math.min(conn1.length, conn2.length); + for (int i = 0; i < len; i++) { + int res = Long.compare(conn1[i], conn2[i]); + + if (res != 0) + return res; + } + + return conn1.length - conn2.length; + } + } +} 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 df37dff1f6f10..f9fd6fd504ad1 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 @@ -3442,50 +3442,66 @@ else if (X.hasCause(e, SocketTimeoutException.class)) break; } - if (client == null) { - assert errs != null; + if (client == null) + processClientCreationError(node, addrs, errs); - if (X.hasCause(errs, ConnectException.class)) - LT.warn(log, "Failed to connect to a remote node " + - "(make sure that destination node is alive and " + - "operating system firewall is disabled on local and remote hosts) " + - "[addrs=" + addrs + ']'); + return client; + } + + /** + * Process errors if TCP client to remote node hasn't been created. + * + * @param node Remote node. + * @param addrs Remote node addresses. + * @param errs TCP client creation errors. + * + * @throws IgniteCheckedException If failed. + */ + protected void processClientCreationError( + ClusterNode node, + Collection addrs, + IgniteCheckedException errs + ) throws IgniteCheckedException { + assert errs != null; - boolean commErrResolve = false; + if (X.hasCause(errs, ConnectException.class)) + LT.warn(log, "Failed to connect to a remote node " + + "(make sure that destination node is alive and " + + "operating system firewall is disabled on local and remote hosts) " + + "[addrs=" + addrs + ']'); - IgniteSpiContext ctx = getSpiContext(); + boolean commErrResolve = false; - if (connectionError(errs) && ctx.communicationFailureResolveSupported()) { - commErrResolve = true; + IgniteSpiContext ctx = getSpiContext(); - ctx.resolveCommunicationFailure(node, errs); - } + if (connectionError(errs) && ctx.communicationFailureResolveSupported()) { + commErrResolve = true; - if (!commErrResolve && enableForcibleNodeKill) { - if (ctx.node(node.id()) != null - && (CU.clientNode(node) || !CU.clientNode(getLocalNode())) && - connectionError(errs)) { - String msg = "TcpCommunicationSpi failed to establish connection to node, node will be dropped from " + - "cluster [" + "rmtNode=" + node + ']'; + ctx.resolveCommunicationFailure(node, errs); + } - if (enableTroubleshootingLog) - U.error(log, msg, errs); - else - U.warn(log, msg); + if (!commErrResolve && enableForcibleNodeKill) { + if (ctx.node(node.id()) != null + && (CU.clientNode(node) || !CU.clientNode(getLocalNode())) && + connectionError(errs)) { + String msg = "TcpCommunicationSpi failed to establish connection to node, node will be dropped from " + + "cluster [" + "rmtNode=" + node + ']'; - ctx.failNode(node.id(), "TcpCommunicationSpi failed to establish connection to node [" + - "rmtNode=" + node + - ", errs=" + errs + - ", connectErrs=" + X.getSuppressedList(errs) + ']'); - } - } + if (enableTroubleshootingLog) + U.error(log, msg, errs); + else + U.warn(log, msg); - if (!X.hasCause(errs, SocketTimeoutException.class, HandshakeTimeoutException.class, - IgniteSpiOperationTimeoutException.class)) - throw errs; + ctx.failNode(node.id(), "TcpCommunicationSpi failed to establish connection to node [" + + "rmtNode=" + node + + ", errs=" + errs + + ", connectErrs=" + X.getSuppressedList(errs) + ']'); + } } - return client; + if (!X.hasCause(errs, SocketTimeoutException.class, HandshakeTimeoutException.class, + IgniteSpiOperationTimeoutException.class)) + throw errs; } /** 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 new file mode 100644 index 0000000000000..d6680cf0867f8 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/cluster/FullyConnectedComponentSearcherTest.java @@ -0,0 +1,323 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.cluster; + +import java.util.Arrays; +import java.util.BitSet; +import java.util.Collection; +import java.util.Random; +import org.apache.ignite.internal.cluster.graph.FullyConnectedComponentSearcher; +import org.apache.ignite.internal.util.typedef.internal.A; +import org.jetbrains.annotations.NotNull; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +/** + * Class to test correctness of fully-connectet component searching algorithm. + */ +@RunWith(Parameterized.class) +public class FullyConnectedComponentSearcherTest { + /** Adjacency matrix provider for each test. */ + private AdjacencyMatrixProvider provider; + + /** Minimul acceptable result of size of fully-connected component for each test. */ + private int minAcceptableRes; + + /** + * @param provider Adjacency matrix. + * @param minAcceptableRes Expected result. + */ + public FullyConnectedComponentSearcherTest(AdjacencyMatrixProvider provider, int minAcceptableRes) { + this.provider = provider; + this.minAcceptableRes = minAcceptableRes; + } + + /** + * + */ + @Test + public void testFind() { + BitSet[] matrix = provider.provide(); + + int nodes = matrix.length; + + BitSet all = new BitSet(nodes); + for (int i = 0; i < nodes; i++) + all.set(i); + + FullyConnectedComponentSearcher searcher = new FullyConnectedComponentSearcher(matrix); + + BitSet res = searcher.findLargest(all); + int size = res.cardinality(); + + Assert.assertTrue("Actual = " + size + ", Expected = " + minAcceptableRes, + size >= minAcceptableRes); + } + + /** + * @return Test dataset. + */ + @Parameterized.Parameters(name = "{index}: search({0}) >= {1}") + public static Collection data() { + return Arrays.asList(new Object[][]{ + {new StaticMatrix(new String[] { + "100", + "010", + "001", + }), 1}, + {new StaticMatrix(new String[] { + "101", + "010", + "101", + }), 2}, + {new StaticMatrix(new String[] { + "1101", + "1111", + "0110", + "1101", + }), 3}, + {new StaticMatrix(new String[] { + "1111001", + "1111000", + "1111000", + "1111000", + "0000111", + "0000111", + "1000111", + }), 4}, + {new AlmostSplittedMatrix(30, 100, 200), 200}, + {new AlmostSplittedMatrix(500, 1000, 2000), 2000}, + {new AlmostSplittedMatrix(1000, 2000, 3000), 3000}, + {new AlmostSplittedMatrix(30, 22, 25, 33, 27), 33}, + {new AlmostSplittedMatrix(1000, 400, 1000, 800), 1000}, + {new SeveralConnectionsAreLostMatrix(200, 10), 190}, + {new SeveralConnectionsAreLostMatrix(2000, 100), 1900}, + {new SeveralConnectionsAreLostMatrix(2000, 500), 1500}, + {new SeveralConnectionsAreLostMatrix(4000, 2000), 2000} + }); + } + + /** + * Provider for adjacency matrix for each test. + */ + interface AdjacencyMatrixProvider { + /** + * @return Adjacency matrix. + */ + BitSet[] provide(); + } + + /** + * Static graph represented as array of strings. Each cell (i, j) in such matrix means that there is connection + * between node(i) and node(j). Needed mostly to test bruteforce algorithm implementation. + */ + static class StaticMatrix implements AdjacencyMatrixProvider { + /** Matrix. */ + private final BitSet[] matrix; + + /** + * @param strMatrix String matrix. + */ + public StaticMatrix(@NotNull String[] strMatrix) { + A.ensure(strMatrix.length > 0, "Matrix should not be empty"); + for (int i = 0; i < strMatrix.length; i++) + A.ensure(strMatrix[i].length() == strMatrix.length, + "Matrix should be quadratic. Problem row: " + i); + + int nodes = strMatrix.length; + + matrix = init(nodes); + + for (int i = 0; i < nodes; i++) + for (int j = 0; j < nodes; j++) + matrix[i].set(j, strMatrix[i].charAt(j) == '1'); + } + + /** {@inheritDoc} */ + @Override public BitSet[] provide() { + return matrix; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "StaticMatrix{" + + "matrix=" + Arrays.toString(matrix) + + '}'; + } + } + + /** + * A graph splitted on several isolated fully-connected components, + * but each of such component have some connections to another to reach graph connectivity. + * Answer is this case should be the size max(Pi), where Pi size of each fully-connected component. + */ + static class AlmostSplittedMatrix implements AdjacencyMatrixProvider { + /** Partition sizes. */ + private final int[] partSizes; + + /** Connections between parts. */ + private final int connectionsBetweenParts; + + /** Matrix. */ + private final BitSet[] matrix; + + /** + * @param connectionsBetweenParts Connections between parts. + * @param partSizes Partition sizes. + */ + public AlmostSplittedMatrix(int connectionsBetweenParts, int... partSizes) { + A.ensure(connectionsBetweenParts >= 1 + partSizes.length, "There should be at least 1 connection between parts"); + A.ensure(partSizes.length >= 2, "The should be at least 2 parts of cluster"); + for (int i = 0; i < partSizes.length; i++) + A.ensure(partSizes[i] > 0, "Part size " + (i + 1) + " shouldn't be empty"); + + this.partSizes = partSizes.clone(); + this.connectionsBetweenParts = connectionsBetweenParts; + + int nodes = 0; + for (int i = 0; i < partSizes.length; i++) + nodes += partSizes[i]; + + matrix = init(nodes); + + int[] startIdx = new int[partSizes.length]; + + for (int i = 0, total = 0; i < partSizes.length; i++) { + startIdx[i] = total; + + fillAll(matrix, total, total + partSizes[i]); + + total += partSizes[i]; + } + + Random random = new Random(777); + + for (int i = 0, part1 = 0; i < connectionsBetweenParts; i++) { + int part2 = (part1 + 1) % partSizes.length; + + // Pick 2 random nodes from 2 parts and add connection between them. + int idx1 = random.nextInt(partSizes[part1]) + startIdx[part1]; + int idx2 = random.nextInt(partSizes[part2]) + startIdx[part2]; + + matrix[idx1].set(idx2); + matrix[idx2].set(idx1); + } + } + + /** {@inheritDoc} */ + @Override public BitSet[] provide() { + return matrix; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "AlmostSplittedGraph{" + + "partSizes=" + Arrays.toString(partSizes) + + ", connectionsBetweenParts=" + connectionsBetweenParts + + '}'; + } + } + + /** + * Complete graph with several connections lost choosen randomly. + * In worst case each lost connection decreases potential size of maximal fully-connected component. + * So answer in this test case should be at least N - L, where N - nodes, L - lost connections. + */ + static class SeveralConnectionsAreLostMatrix implements AdjacencyMatrixProvider { + /** Nodes. */ + private final int nodes; + + /** Lost connections. */ + private final int lostConnections; + + /** Matrix. */ + private final BitSet[] matrix; + + /** + * @param nodes Nodes. + * @param lostConnections Lost connections. + */ + public SeveralConnectionsAreLostMatrix(int nodes, int lostConnections) { + A.ensure(nodes > 0, "There should be at least 1 node"); + + this.nodes = nodes; + this.lostConnections = lostConnections; + + this.matrix = init(nodes); + + fillAll(matrix, 0, nodes); + + Random random = new Random(777); + + for (int i = 0; i < lostConnections; i++) { + int idx1 = random.nextInt(nodes); + int idx2 = random.nextInt(nodes); + + if (idx1 == idx2) + continue; + + matrix[idx1].set(idx2, false); + matrix[idx2].set(idx1, false); + } + } + + /** {@inheritDoc} */ + @Override public BitSet[] provide() { + return matrix; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "SeveralConnectionsAreLost{" + + "nodes=" + nodes + + ", lostConnections=" + lostConnections + + '}'; + } + } + + /** + * Utility method to pre-create adjacency matrix. + * + * @param nodes Nodes in graph. + * @return Adjacency matrix. + */ + private static BitSet[] init(int nodes) { + BitSet[] matrix = new BitSet[nodes]; + for (int i = 0; i < nodes; i++) + matrix[i] = new BitSet(nodes); + + return matrix; + } + + /** + * Utility method to fill all connections between all nodes from {@code fromIdx} and {@code endIdx} exclusive. + * + * @param matrix Adjacency matrix. + * @param fromIdx Lower bound node index inclusive. + * @param endIdx Upper bound node index exclusive. + */ + private static void fillAll(BitSet[] matrix, int fromIdx, int endIdx) { + for (int i = fromIdx; i < endIdx; i++) + for (int j = fromIdx; j < endIdx; j++) { + matrix[i].set(j); + matrix[j].set(i); + } + } +} 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 408af30c87e05..03b874dce1bf9 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 @@ -17,8 +17,13 @@ package org.apache.ignite.spi.discovery.zk.internal; +import java.io.InputStream; +import java.io.OutputStream; import java.io.Serializable; import java.lang.management.ManagementFactory; +import java.net.InetSocketAddress; +import java.net.SocketTimeoutException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.BitSet; @@ -41,6 +46,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; import javax.management.JMX; import javax.management.MBeanServer; import javax.management.ObjectName; @@ -84,6 +90,8 @@ import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl; import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.internal.util.lang.IgniteInClosure2X; +import org.apache.ignite.internal.util.nio.GridCommunicationClient; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.T3; import org.apache.ignite.internal.util.typedef.X; @@ -169,6 +177,9 @@ public class ZookeeperDiscoverySpiTest extends GridCommonAbstractTest { /** */ private boolean testCommSpi; + /** */ + private boolean failCommSpi; + /** */ private long sesTimeout; @@ -335,6 +346,9 @@ public class ZookeeperDiscoverySpiTest extends GridCommonAbstractTest { if (testCommSpi) cfg.setCommunicationSpi(new ZkTestCommunicationSpi()); + if (failCommSpi) + cfg.setCommunicationSpi(new PeerToPeerCommunicationFailureSpi()); + if (commFailureRslvr != null) cfg.setCommunicationFailureResolver(commFailureRslvr.apply()); @@ -3558,6 +3572,369 @@ public void testReconnectServersRestart_2() throws Exception { reconnectServersRestart(3); } + /** + * A simple split-brain test, where cluster spliited on 2 parts of server nodes (2 and 3). + * There is also client which sees both parts of splitted cluster. + * + * Result cluster should be: 3 server nodes + 1 client. + * + * @throws Exception If failed. + */ + public void testSimpleSplitBrain() throws Exception { + failCommSpi = true; + + startGridsMultiThreaded(5); + + client = true; + + startGridsMultiThreaded(5, 3); + + List all = G.allGrids().stream() + .map(g -> g.cluster().localNode()) + .collect(Collectors.toList());; + + List part1 = all.subList(0, 3); + List part2 = all.subList(3, all.size()); + + ConnectionsFailureMatrix matrix = ConnectionsFailureMatrix.buildFrom(part1, part2); + + ClusterNode lastClient = startGrid(8).cluster().localNode(); + + // Make last client connected to other nodes. + for (ClusterNode node : all) { + if (node.id().equals(lastClient.id())) + continue; + + matrix.addConnection(lastClient, node); + matrix.addConnection(node, lastClient); + } + + PeerToPeerCommunicationFailureSpi.fail(matrix); + + waitForTopology(4); + } + + /** + * A simple not actual split-brain test, where some connections between server nodes are lost. + * Server nodes: 5. + * Client nodes: 5. + * Lost connections between server nodes: 2. + * + * Result cluster should be: 3 server nodes + 5 clients. + * + * @throws Exception If failed. + */ + public void testNotActualSplitBrain() throws Exception { + failCommSpi = true; + + startGridsMultiThreaded(5); + + List srvNodes = G.allGrids().stream() + .map(g -> g.cluster().localNode()) + .collect(Collectors.toList()); + + client = true; + + startGridsMultiThreaded(5, 3); + + client = false; + + ConnectionsFailureMatrix matrix = new ConnectionsFailureMatrix(); + + matrix.addAll(G.allGrids().stream().map(g -> g.cluster().localNode()).collect(Collectors.toList())); + + // Remove 2 connections between server nodes. + matrix.removeConnection(srvNodes.get(0), srvNodes.get(1)); + matrix.removeConnection(srvNodes.get(1), srvNodes.get(0)); + matrix.removeConnection(srvNodes.get(2), srvNodes.get(3)); + matrix.removeConnection(srvNodes.get(3), srvNodes.get(2)); + + PeerToPeerCommunicationFailureSpi.fail(matrix); + + waitForTopology(8); + } + + /** + * Almost split-brain test, server nodes splitted on 2 parts and there are some connections between these 2 parts. + * Server nodes: 5. + * Client nodes: 5. + * Splitted on: 3 servers + 2 clients and 3 servers + 2 clients. + * Extra connections between server nodes: 3. + * + * Result cluster should be: 3 server nodes + 2 clients. + * + * @throws Exception If failed. + */ + public void testAlmostSplitBrain() throws Exception { + failCommSpi = true; + + startGridsMultiThreaded(6); + + List srvNodes = G.allGrids().stream() + .map(g -> g.cluster().localNode()) + .collect(Collectors.toList()); + + List srvPart1 = srvNodes.subList(0, 3); + List srvPart2 = srvNodes.subList(3, srvNodes.size()); + + client = true; + + startGridsMultiThreaded(6, 5); + + client = false; + + List clientNodes = G.allGrids().stream() + .map(g -> g.cluster().localNode()) + .filter(ClusterNode::isClient) + .collect(Collectors.toList()); + + List clientPart1 = clientNodes.subList(0, 2); + List clientPart2 = clientNodes.subList(2, 4); + + List splittedPart1 = new ArrayList<>(); + splittedPart1.addAll(srvPart1); + splittedPart1.addAll(clientPart1); + + List splittedPart2 = new ArrayList<>(); + splittedPart2.addAll(srvPart2); + splittedPart2.addAll(clientPart2); + + ConnectionsFailureMatrix matrix = new ConnectionsFailureMatrix(); + + matrix.addAll(splittedPart1); + matrix.addAll(splittedPart2); + + matrix.addConnection(srvPart1.get(0), srvPart2.get(1)); + matrix.addConnection(srvPart2.get(1), srvPart1.get(0)); + + matrix.addConnection(srvPart1.get(1), srvPart2.get(2)); + matrix.addConnection(srvPart2.get(2), srvPart1.get(1)); + + matrix.addConnection(srvPart1.get(2), srvPart2.get(0)); + matrix.addConnection(srvPart2.get(0), srvPart1.get(2)); + + PeerToPeerCommunicationFailureSpi.fail(matrix); + + waitForTopology(5); + } + + /** + * Class represents available connections between cluster nodes. + * This is needed to simulate network problems in {@link PeerToPeerCommunicationFailureSpi}. + */ + static class ConnectionsFailureMatrix { + /** Available connections per each node id. */ + private Map> availableConnections = new HashMap<>(); + + /** + * @param from Cluster node 1. + * @param to Cluster node 2. + * @return {@code True} if there is connection between nodes {@code from} and {@code to}. + */ + public boolean hasConnection(ClusterNode from, ClusterNode to) { + return availableConnections.getOrDefault(from.id(), Collections.emptySet()).contains(to.id()); + } + + /** + * Adds connection between nodes {@code from} and {@code to}. + * @param from Cluster node 1. + * @param to Cluster node 2. + */ + public void addConnection(ClusterNode from, ClusterNode to) { + availableConnections.computeIfAbsent(from.id(), s -> new HashSet<>()).add(to.id()); + } + + /** + * Removes connection between nodes {@code from} and {@code to}. + * @param from Cluster node 1. + * @param to Cluster node 2. + */ + public void removeConnection(ClusterNode from, ClusterNode to) { + availableConnections.getOrDefault(from.id(), Collections.emptySet()).remove(to.id()); + } + + /** + * Adds connections between all nodes presented in given {@code nodeSet}. + * + * @param nodeSet Set of the cluster nodes. + */ + public void addAll(List nodeSet) { + for (int i = 0; i < nodeSet.size(); i++) { + for (int j = 0; j < nodeSet.size(); j++) { + if (i == j) + continue; + + addConnection(nodeSet.get(i), nodeSet.get(j)); + } + } + } + + /** + * Builds connections failure matrix from two part of the cluster nodes. + * Each part has all connections inside, but hasn't any connection to another part. + * + * @param part1 Part 1. + * @param part2 Part 2. + * @return Connections failure matrix. + */ + static ConnectionsFailureMatrix buildFrom(List part1, List part2) { + ConnectionsFailureMatrix matrix = new ConnectionsFailureMatrix(); + matrix.addAll(part1); + matrix.addAll(part2); + return matrix; + } + } + + /** + * Communication SPI with possibility to simulate network problems between some of the cluster nodes. + */ + static class PeerToPeerCommunicationFailureSpi extends TcpCommunicationSpi { + /** Flag indicates that connections according to {@code matrix} should be failed. */ + private static volatile boolean failure; + + /** Connections failure matrix. */ + private static ConnectionsFailureMatrix matrix; + + /** + * Start failing connections according to given matrix {@code with}. + * @param with Failure matrix. + */ + public static void fail(ConnectionsFailureMatrix with) { + matrix = with; + failure = true; + } + + /** {@inheritDoc} */ + @Override public IgniteFuture checkConnection(List nodes) { + // Creates connections statuses according to failure matrix. + BitSet bitSet = new BitSet(); + + ClusterNode localNode = getLocalNode(); + + int idx = 0; + + for (ClusterNode remoteNode : nodes) { + if (localNode.id().equals(remoteNode.id())) + bitSet.set(idx); + else { + if (matrix.hasConnection(localNode, remoteNode)) + bitSet.set(idx); + } + idx++; + } + + return new IgniteFinishedFutureImpl<>(bitSet); + } + + /** {@inheritDoc} */ + @Override protected GridCommunicationClient createTcpClient( + ClusterNode node, + int connIdx + ) throws IgniteCheckedException { + if (failure && !matrix.hasConnection(getLocalNode(), node)) { + processClientCreationError(node, null, new IgniteCheckedException("Test", new SocketTimeoutException())); + + return null; + } + + return new FailingCommunicationClient(getLocalNode(), node, + super.createTcpClient(node, connIdx)); + } + + /** + * Communication client with possibility to simulate network error between peers. + */ + class FailingCommunicationClient implements GridCommunicationClient { + /** Delegate. */ + private final GridCommunicationClient delegate; + + /** Local node which sends messages. */ + private final ClusterNode localNode; + + /** Remote node which receives messages. */ + private final ClusterNode remoteNode; + + FailingCommunicationClient(ClusterNode localNode, ClusterNode remoteNode, GridCommunicationClient delegate) { + this.delegate = delegate; + this.localNode = localNode; + this.remoteNode = remoteNode; + } + + /** {@inheritDoc} */ + @Override public void doHandshake(IgniteInClosure2X handshakeC) throws IgniteCheckedException { + if (failure && !matrix.hasConnection(localNode, remoteNode)) + throw new IgniteCheckedException("Test", new SocketTimeoutException()); + + delegate.doHandshake(handshakeC); + } + + /** {@inheritDoc} */ + @Override public boolean close() { + return delegate.close(); + } + + /** {@inheritDoc} */ + @Override public void forceClose() { + delegate.forceClose(); + } + + /** {@inheritDoc} */ + @Override public boolean closed() { + return delegate.closed(); + } + + /** {@inheritDoc} */ + @Override public boolean reserve() { + return delegate.reserve(); + } + + /** {@inheritDoc} */ + @Override public void release() { + delegate.release(); + } + + /** {@inheritDoc} */ + @Override public long getIdleTime() { + return delegate.getIdleTime(); + } + + /** {@inheritDoc} */ + @Override public void sendMessage(ByteBuffer data) throws IgniteCheckedException { + if (failure && !matrix.hasConnection(localNode, remoteNode)) + throw new IgniteCheckedException("Test", new SocketTimeoutException()); + + delegate.sendMessage(data); + } + + /** {@inheritDoc} */ + @Override public void sendMessage(byte[] data, int len) throws IgniteCheckedException { + if (failure && !matrix.hasConnection(localNode, remoteNode)) + throw new IgniteCheckedException("Test", new SocketTimeoutException()); + + delegate.sendMessage(data, len); + } + + /** {@inheritDoc} */ + @Override public boolean sendMessage(@Nullable UUID nodeId, Message msg, @Nullable IgniteInClosure c) throws IgniteCheckedException { + // This will enforce SPI to create new client. + if (failure && !matrix.hasConnection(localNode, remoteNode)) + return true; + + return delegate.sendMessage(nodeId, msg, c); + } + + /** {@inheritDoc} */ + @Override public boolean async() { + return delegate.async(); + } + + /** {@inheritDoc} */ + @Override public int connectionIndex() { + return delegate.connectionIndex(); + } + } + } + /** * @param srvs Number of server nodes in test. * @throws Exception If failed. @@ -3886,6 +4263,8 @@ private void reset() { err = false; + failCommSpi = false; + evts.clear(); try { From 2d9c4294a15a6fdf7d8a3d69b9e557d06c710285 Mon Sep 17 00:00:00 2001 From: Ivan Daschinskiy Date: Thu, 10 May 2018 12:02:20 +0300 Subject: [PATCH 0155/1463] IGNITE-7912: control.sh script should show used WAL-segments. This closes #3636. This closes #3965. (cherry picked from commit 07cbe22) --- .../apache/ignite/IgniteSystemProperties.java | 3 + .../internal/commandline/Arguments.java | 80 +++-- .../ignite/internal/commandline/Command.java | 5 +- .../internal/commandline/CommandHandler.java | 228 +++++++++++- .../wal/IgniteWriteAheadLogManager.java | 10 + .../GridCacheDatabaseSharedManager.java | 20 ++ .../wal/FileWriteAheadLogManager.java | 28 ++ .../FsyncModeFileWriteAheadLogManager.java | 30 ++ .../internal/visor/misc/VisorClusterNode.java | 129 +++++++ .../internal/visor/misc/VisorWalTask.java | 336 ++++++++++++++++++ .../internal/visor/misc/VisorWalTaskArg.java | 98 +++++ .../visor/misc/VisorWalTaskOperation.java | 44 +++ .../visor/misc/VisorWalTaskResult.java | 113 ++++++ .../resources/META-INF/classnames.properties | 6 + .../CommandHandlerParsingTest.java | 119 ++++++- .../db/IgnitePdsUnusedWalSegmentsTest.java | 202 +++++++++++ .../persistence/pagemem/NoOpWALManager.java | 7 +- .../testsuites/IgnitePdsTestSuite2.java | 3 + .../testsuites/IgniteUtilSelfTestSuite.java | 2 + .../ignite/util/GridCommandHandlerTest.java | 72 ++++ ...GridInternalTaskUnusedWalSegmentsTest.java | 153 ++++++++ 21 files changed, 1634 insertions(+), 54 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorClusterNode.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTask.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTaskArg.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTaskOperation.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTaskResult.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsUnusedWalSegmentsTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/util/GridInternalTaskUnusedWalSegmentsTest.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 32fed05b369ab..008974c6ff68e 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -277,6 +277,9 @@ public final class IgniteSystemProperties { /** System property to hold SSH host for visor-started nodes. */ public static final String IGNITE_SSH_HOST = "IGNITE_SSH_HOST"; + /** System property to enable experimental commands in control.sh script. */ + public static final String IGNITE_ENABLE_EXPERIMENTAL_COMMAND = "IGNITE_ENABLE_EXPERIMENTAL_COMMAND"; + /** System property to hold SSH user name for visor-started nodes. */ public static final String IGNITE_SSH_USER_NAME = "IGNITE_SSH_USER_NAME"; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/Arguments.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/Arguments.java index ce7269378fc20..0d4b38e83851b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/Arguments.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/Arguments.java @@ -53,12 +53,6 @@ public class Arguments { */ private String baselineArgs; - /** Ping timeout for grid client. See {@link GridClientConfiguration#pingTimeout}.*/ - private long pingTimeout; - - /** Ping interval for grid client. See {@link GridClientConfiguration#pingInterval}.*/ - private long pingInterval; - /** Transaction arguments. */ private final VisorTxTaskArg txArg; @@ -67,6 +61,22 @@ public class Arguments { */ private CacheArguments cacheArgs; + /** + * Action for WAL command. + */ + private String walAct; + + /** + * Arguments for WAL command. + */ + private String walArgs; + + /** Ping timeout for grid client. See {@link GridClientConfiguration#pingTimeout}.*/ + private long pingTimeout; + + /** Ping interval for grid client. See {@link GridClientConfiguration#pingInterval}.*/ + private long pingInterval; + /** * @param cmd Command. * @param host Host. @@ -76,14 +86,16 @@ public class Arguments { * @param baselineAct Baseline action. * @param baselineArgs Baseline args. * @param txArg TX arg. - * @param force Force flag. + * @param cacheArgs --cache subcommand arguments. + * @param walAct WAL action. + * @param walArgs WAL args. * @param pingTimeout Ping timeout. See {@link GridClientConfiguration#pingTimeout}. * @param pingInterval Ping interval. See {@link GridClientConfiguration#pingInterval}. - * @param cacheArgs --cache subcommand arguments. + * @param force Force flag. */ public Arguments(Command cmd, String host, String port, String user, String pwd, String baselineAct, - String baselineArgs, long pingTimeout, long pingInterval, VisorTxTaskArg txArg, boolean force, - CacheArguments cacheArgs) { + String baselineArgs, VisorTxTaskArg txArg, CacheArguments cacheArgs, String walAct, String walArgs, + Long pingTimeout, Long pingInterval, boolean force) { this.cmd = cmd; this.host = host; this.port = port; @@ -91,11 +103,13 @@ public Arguments(Command cmd, String host, String port, String user, String pwd, this.pwd = pwd; this.baselineAct = baselineAct; this.baselineArgs = baselineArgs; + this.txArg = txArg; + this.cacheArgs = cacheArgs; + this.walAct = walAct; + this.walArgs = walArgs; this.pingTimeout = pingTimeout; this.pingInterval = pingInterval; this.force = force; - this.txArg = txArg; - this.cacheArgs = cacheArgs; } /** @@ -147,6 +161,34 @@ public String baselineArguments() { return baselineArgs; } + /** + * @return Transaction arguments. + */ + public VisorTxTaskArg transactionArguments() { + return txArg; + } + + /** + * @return Arguments for --cache subcommand. + */ + public CacheArguments cacheArgs() { + return cacheArgs; + } + + /** + * @return WAL action. + */ + public String walAction() { + return walAct; + } + + /** + * @return WAL arguments. + */ + public String walArguments() { + return walArgs; + } + /** * See {@link GridClientConfiguration#pingTimeout}. * @@ -165,24 +207,10 @@ public long pingInterval() { return pingInterval; } - /** - * @return Transaction arguments. - */ - public VisorTxTaskArg transactionArguments() { - return txArg; - } - /** * @return Force option. */ public boolean force() { return force; } - - /** - * @return Arguments for --cache subcommand. - */ - public CacheArguments cacheArgs() { - return cacheArgs; - } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/Command.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/Command.java index 52098d629fb0d..c64e488db4ffd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/Command.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/Command.java @@ -37,7 +37,10 @@ public enum Command { TX("--tx"), /** */ - CACHE("--cache"); + CACHE("--cache"), + + /** */ + WAL("--wal"); /** */ private final String text; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java index ff05684bb47a8..7d457fd1f1265 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java @@ -32,6 +32,7 @@ import java.util.regex.Pattern; import java.util.regex.PatternSyntaxException; import java.util.stream.Collectors; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.client.GridClient; import org.apache.ignite.internal.client.GridClientAuthenticationException; @@ -61,6 +62,11 @@ import org.apache.ignite.internal.visor.baseline.VisorBaselineTask; import org.apache.ignite.internal.visor.baseline.VisorBaselineTaskArg; import org.apache.ignite.internal.visor.baseline.VisorBaselineTaskResult; +import org.apache.ignite.internal.visor.misc.VisorClusterNode; +import org.apache.ignite.internal.visor.misc.VisorWalTask; +import org.apache.ignite.internal.visor.misc.VisorWalTaskArg; +import org.apache.ignite.internal.visor.misc.VisorWalTaskOperation; +import org.apache.ignite.internal.visor.misc.VisorWalTaskResult; import org.apache.ignite.internal.visor.tx.VisorTxInfo; import org.apache.ignite.internal.visor.tx.VisorTxOperation; import org.apache.ignite.internal.visor.tx.VisorTxProjection; @@ -86,6 +92,7 @@ import org.apache.ignite.plugin.security.SecurityCredentials; import org.apache.ignite.plugin.security.SecurityCredentialsBasicProvider; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_ENABLE_EXPERIMENTAL_COMMAND; import static org.apache.ignite.internal.IgniteVersionUtils.ACK_VER_STR; import static org.apache.ignite.internal.IgniteVersionUtils.COPYRIGHT; import static org.apache.ignite.internal.commandline.Command.ACTIVATE; @@ -93,6 +100,7 @@ import static org.apache.ignite.internal.commandline.Command.CACHE; import static org.apache.ignite.internal.commandline.Command.DEACTIVATE; import static org.apache.ignite.internal.commandline.Command.STATE; +import static org.apache.ignite.internal.commandline.Command.WAL; import static org.apache.ignite.internal.commandline.Command.TX; import static org.apache.ignite.internal.visor.baseline.VisorBaselineOperation.ADD; import static org.apache.ignite.internal.visor.baseline.VisorBaselineOperation.COLLECT; @@ -133,6 +141,12 @@ public class CommandHandler { /** Force option is used for auto confirmation. */ private static final String CMD_FORCE = "--force"; + /** */ + protected static final String CMD_PING_INTERVAL = "--ping-interval"; + + /** */ + protected static final String CMD_PING_TIMEOUT = "--ping-timeout"; + /** List of optional auxiliary commands. */ private static final Set AUX_COMMANDS = new HashSet<>(); static { @@ -142,17 +156,13 @@ public class CommandHandler { AUX_COMMANDS.add(CMD_PASSWORD); AUX_COMMANDS.add(CMD_USER); AUX_COMMANDS.add(CMD_FORCE); + AUX_COMMANDS.add(CMD_PING_INTERVAL); + AUX_COMMANDS.add(CMD_PING_TIMEOUT); } /** Broadcast uuid. */ private static final UUID BROADCAST_UUID = UUID.randomUUID(); - /** */ - protected static final String CMD_PING_INTERVAL = "--ping-interval"; - - /** */ - protected static final String CMD_PING_TIMEOUT = "--ping-timeout"; - /** */ public static final String CONFIRM_MSG = "y"; @@ -172,7 +182,13 @@ public class CommandHandler { private static final String BASELINE_SET_VERSION = "version"; /** */ - private static final String DELIM = "--------------------------------------------------------------------------------"; + static final String WAL_PRINT = "print"; + + /** */ + static final String WAL_DELETE = "delete"; + + /** */ + static final String DELIM = "--------------------------------------------------------------------------------"; /** */ public static final int EXIT_CODE_OK = 0; @@ -240,6 +256,9 @@ public class CommandHandler { /** */ private Object lastOperationRes; + /** Check if experimental commands are enabled. Default {@code false}. */ + private final boolean enableExperimental = IgniteSystemProperties.getBoolean(IGNITE_ENABLE_EXPERIMENTAL_COMMAND, false); + /** * Output specified string to console. * @@ -332,6 +351,12 @@ private String confirmationPrompt(Arguments args) { break; + case WAL: + if (WAL_DELETE.equals(args.walAction())) + str = "Warning: the command will delete unused WAL segments."; + + break; + case TX: if (args.transactionArguments().getOperation() == VisorTxOperation.KILL) str = "Warning: the command will kill some transactions."; @@ -941,6 +966,147 @@ else if (arg.getOperation() == VisorTxOperation.KILL) } } + /** + * Execute WAL command. + * + * @param client Client. + * @param walAct WAL action to execute. + * @param walArgs WAL args. + * @throws Throwable If failed to execute wal action. + */ + private void wal(GridClient client, String walAct, String walArgs) throws Throwable { + switch (walAct){ + case WAL_DELETE: + deleteUnusedWalSegments(client, walArgs); + + break; + + case WAL_PRINT: + default: + printUnusedWalSegments(client, walArgs); + + break; + } + } + + /** + * Execute delete unused WAL segments task. + * + * @param client Client. + * @param walArgs WAL args. + */ + private void deleteUnusedWalSegments(GridClient client, String walArgs) throws Throwable { + VisorWalTaskResult res = executeTask(client, VisorWalTask.class, + walArg(VisorWalTaskOperation.DELETE_UNUSED_WAL_SEGMENTS, walArgs)); + printDeleteWalSegments0(res); + } + + /** + * Execute print unused WAL segments task. + * + * @param client Client. + * @param walArgs Wal args. + */ + private void printUnusedWalSegments(GridClient client, String walArgs) throws Throwable { + VisorWalTaskResult res = executeTask(client, VisorWalTask.class, + walArg(VisorWalTaskOperation.PRINT_UNUSED_WAL_SEGMENTS, walArgs)); + printUnusedWalSegments0(res); + } + + /** + * Prepare WAL task argument. + * + * @param op Operation. + * @param s Argument from command line. + * @return Task argument. + */ + private VisorWalTaskArg walArg(VisorWalTaskOperation op, String s){ + List consistentIds = null; + + if (!F.isEmpty(s)) { + consistentIds = new ArrayList<>(); + + for (String consistentId : s.split(",")) + consistentIds.add(consistentId.trim()); + } + + switch (op) { + case DELETE_UNUSED_WAL_SEGMENTS: + case PRINT_UNUSED_WAL_SEGMENTS: + return new VisorWalTaskArg(op, consistentIds); + + default: + return new VisorWalTaskArg(VisorWalTaskOperation.PRINT_UNUSED_WAL_SEGMENTS, consistentIds); + } + + } + + /** + * Print list of unused wal segments. + * + * @param taskRes Task result with baseline topology. + */ + private void printUnusedWalSegments0(VisorWalTaskResult taskRes) { + log("Unused wal segments per node:"); + nl(); + + Map> res = taskRes.results(); + Map failRes = taskRes.exceptions(); + Map nodesInfo = taskRes.getNodesInfo(); + + for(Map.Entry> entry: res.entrySet()) { + VisorClusterNode node = nodesInfo.get(entry.getKey()); + + log("Node=" + node.getConsistentId()); + log(" addresses " + U.addressesAsString(node.getAddresses(),node.getHostNames())); + + for(String fileName: entry.getValue()) + log(" " + fileName); + + nl(); + } + + for(Map.Entry entry: failRes.entrySet()) { + VisorClusterNode node = nodesInfo.get(entry.getKey()); + + log("Node=" + node.getConsistentId()); + log(" addresses " + U.addressesAsString(node.getAddresses(),node.getHostNames())); + log(" failed with error: " + entry.getValue().getMessage()); + nl(); + } + } + + /** + * Print list of unused wal segments. + * + * @param taskRes Task result with baseline topology. + */ + private void printDeleteWalSegments0(VisorWalTaskResult taskRes) { + log("WAL segments deleted for nodes:"); + nl(); + + Map> res = taskRes.results(); + Map errors = taskRes.exceptions(); + Map nodesInfo = taskRes.getNodesInfo(); + + for(Map.Entry> entry: res.entrySet()) { + VisorClusterNode node = nodesInfo.get(entry.getKey()); + + log("Node=" + node.getConsistentId()); + log(" addresses " + U.addressesAsString(node.getAddresses(),node.getHostNames())); + nl(); + } + + for(Map.Entry entry: errors.entrySet()) { + VisorClusterNode node = nodesInfo.get(entry.getKey()); + + log("Node=" + node.getConsistentId()); + log(" addresses " + U.addressesAsString(node.getAddresses(),node.getHostNames())); + log(" failed with error: " + entry.getValue().getMessage()); + nl(); + } + } + /** * @param e Exception to check. * @return {@code true} if specified exception is {@link GridClientAuthenticationException}. @@ -1031,6 +1197,10 @@ Arguments parseAndValidate(List rawArgs) { Long pingTimeout = DFLT_PING_TIMEOUT; + String walAct = ""; + + String walArgs = ""; + boolean force = false; CacheArguments cacheArgs = null; @@ -1052,6 +1222,7 @@ Arguments parseAndValidate(List rawArgs) { case DEACTIVATE: case STATE: commands.add(cmd); + break; case TX: @@ -1088,6 +1259,24 @@ Arguments parseAndValidate(List rawArgs) { break; + case WAL: + if (!enableExperimental) + throw new IllegalArgumentException("Experimental command is disabled."); + + commands.add(WAL); + + str = nextArg("Expected arguments for " + WAL.text()); + + walAct = str.toLowerCase(); + + if (WAL_PRINT.equals(walAct) || WAL_DELETE.equals(walAct)) + walArgs = (str = peekNextArg()) != null && !isCommandOrOption(str) + ? nextArg("Unexpected argument for " + WAL.text() + ": " + walAct) + : ""; + else + throw new IllegalArgumentException("Unexpected action " + walAct + " for " + WAL.text()); + + break; default: throw new IllegalArgumentException("Unexpected command: " + str); } @@ -1161,8 +1350,8 @@ Arguments parseAndValidate(List rawArgs) { if (hasUsr != hasPwd) throw new IllegalArgumentException("Both user and password should be specified"); - return new Arguments(cmd, host, port, user, pwd, baselineAct, baselineArgs, - pingTimeout, pingInterval, txArgs, force, cacheArgs); + return new Arguments(cmd, host, port, user, pwd, baselineAct, baselineArgs, txArgs, cacheArgs, walAct, walArgs, + pingTimeout, pingInterval, force); } /** @@ -1429,6 +1618,15 @@ private long nextLongArg(String lb) { } } + /** + * Check if raw arg is command or option. + * + * @return {@code true} If raw arg is command, overwise {@code false}. + */ + private boolean isCommandOrOption(String raw) { + return raw != null && raw.contains("--"); + } + /** * Parse and execute command. * @@ -1457,6 +1655,13 @@ public int execute(List rawArgs) { "[minSize SIZE] [label PATTERN_REGEX] [servers|clients] " + "[nodes consistentId1[,consistentId2,....,consistentIdN] [limit NUMBER] [order DURATION|SIZE] [kill] [--force]"); + if(enableExperimental) { + usage(" Print absolute paths of unused archived wal segments on each node:", WAL, + " print [consistentId1,consistentId2,....,consistentIdN]"); + usage(" Delete unused archived wal segments on each node:", WAL, + " delete [consistentId1,consistentId2,....,consistentIdN] [--force]"); + } + log("The utility has --cache subcommand to view and control state of caches in cluster."); log(" More info: control.sh --cache help"); nl(); @@ -1533,6 +1738,11 @@ public int execute(List rawArgs) { case CACHE: cache(client, args.cacheArgs()); + break; + + case WAL: + wal(client, args.walAction(), args.walArguments()); + break; } } 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 8e8a1b3a43e9b..b5c22c92341c4 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 @@ -123,11 +123,21 @@ public interface IgniteWriteAheadLogManager extends GridCacheSharedManager, Igni /** * Checks if WAL segment is under lock or reserved + * * @param ptr Pointer to check. * @return True if given pointer is located in reserved segment. */ public boolean reserved(WALPointer ptr); + /** + * Checks if WAL segments is under lock or reserved. + * + * @param low Pointer since which WAL is locked or reserved. If {@code null}, checks from the oldest segment. + * @param high Pointer for which WAL is locked or reserved. + * @return Number of reserved WAL segments. + */ + public int reserved(WALPointer low, WALPointer high); + /** * Checks WAL disabled for cache group. * 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 d847132611e5f..12cfc05d72e2e 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 @@ -3721,6 +3721,26 @@ private CheckpointEntry entry(Long cpTs) throws IgniteCheckedException { return entry; } + /** + * @return First checkpoint entry if exists. Otherwise {@code null}. + */ + private CheckpointEntry firstEntry() { + Map.Entry entry = histMap.firstEntry(); + + return entry != null ? entry.getValue() : null; + } + + /** + * Get WAL pointer to low checkpoint bound. + * + * @return WAL pointer to low checkpoint bound. + */ + public WALPointer lowCheckpointBound() { + CheckpointEntry entry = firstEntry(); + + return entry != null ? entry.cpMark : null; + } + /** * @return Collection of checkpoint timestamps. */ 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 8755e1bf31f4b..e69cc8040f7e9 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 @@ -946,6 +946,34 @@ private boolean segmentReservedOrLocked(long absIdx) { return segmentReservedOrLocked(fPtr.index()); } + /** {@inheritDoc} */ + @Override public int reserved(WALPointer low, WALPointer high) { + // It is not clear now how to get the highest WAL pointer. So when high is null method returns 0. + if (high == null) + return 0; + + assert high instanceof FileWALPointer : high; + + assert low == null || low instanceof FileWALPointer : low; + + FileWALPointer lowPtr = (FileWALPointer)low; + + FileWALPointer highPtr = (FileWALPointer)high; + + long lowIdx = lowPtr != null ? lowPtr.index() : 0; + + long highIdx = highPtr.index(); + + while (lowIdx < highIdx) { + if (segmentReservedOrLocked(lowIdx)) + break; + + lowIdx++; + } + + return (int)(highIdx - lowIdx + 1); + } + /** {@inheritDoc} */ @Override public boolean disabled(int grpId) { CacheGroupContext ctx = cctx.cache().cacheGroup(grpId); 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 c446f7f228649..7bb2ce4dfc48c 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 @@ -848,6 +848,36 @@ private boolean hasIndex(long absIdx) { return archiver0 != null && archiver0.reserved(fPtr.index()); } + /** {@inheritDoc} */ + @Override public int reserved(WALPointer low, WALPointer high) { + // It is not clear now how to get the highest WAL pointer. So when high is null method returns 0. + if (high == null) + return 0; + + assert high instanceof FileWALPointer : high; + + assert low == null || low instanceof FileWALPointer : low; + + FileWALPointer lowPtr = (FileWALPointer)low; + + FileWALPointer highPtr = (FileWALPointer)high; + + FileArchiver archiver0 = archiver; + + long lowIdx = lowPtr != null ? lowPtr.index() : 0; + + long highIdx = highPtr.index(); + + while (lowIdx < highIdx) { + if(archiver0 != null && archiver0.reserved(lowIdx)) + break; + + lowIdx++; + } + + return (int)(highIdx - lowIdx + 1); + } + /** {@inheritDoc} */ @Override public boolean disabled(int grpId) { CacheGroupContext ctx = cctx.cache().cacheGroup(grpId); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorClusterNode.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorClusterNode.java new file mode 100644 index 0000000000000..5706444e086dd --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorClusterNode.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.visor.misc; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.Collection; +import java.util.Map; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * Data transfer object for {@link ClusterNode}. + */ +public class VisorClusterNode extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Cluster node consistent id. */ + @GridToStringInclude + private String consistentId; + + /** Cluster node attributes. */ + private Map attrs; + + /** Cluster node addresses. */ + @GridToStringInclude + private Collection addrs; + + /** Cluster node host names. */ + @GridToStringInclude + private Collection hostNames; + + /** + * Default constructor. + */ + public VisorClusterNode() { + // No-op. + } + + /** + * Create data transfer object for baseline node. + * + * @param node Baseline node. + */ + public VisorClusterNode(ClusterNode node) { + consistentId = String.valueOf(node.consistentId()); + addrs = node.addresses(); + hostNames = node.hostNames(); + attrs = node.attributes(); + } + + /** + * Get cluster node consistent id. + * + * @return Cluster node consistent id. + */ + public String getConsistentId() { + return consistentId; + } + + /** + * Get cluster node attributes. + * + * @return Cluster node attributes. + */ + public Map getAttributes() { + return attrs; + } + + /** + * Get cluster node addresses. + * + * @return Node addresses. + */ + public Collection getAddresses() { + return addrs; + } + + /** + * Get cluster node host names. + * + * @return Node host names. + */ + public Collection getHostNames() { + return hostNames; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + U.writeString(out, consistentId); + U.writeMap(out, attrs); + U.writeCollection(out, hostNames); + U.writeCollection(out, addrs); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + consistentId = U.readString(in); + attrs = U.readMap(in); + hostNames = U.readCollection(in); + addrs = U.readCollection(in); + } + + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorClusterNode.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTask.java new file mode 100644 index 0000000000000..7edcea9d459b2 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTask.java @@ -0,0 +1,336 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.visor.misc; + +import java.io.File; +import java.io.FileFilter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.regex.Pattern; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.compute.ComputeJobResult; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.pagemem.wal.WALPointer; +import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings; +import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer; +import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager; +import org.apache.ignite.internal.processors.task.GridInternal; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorJob; +import org.apache.ignite.internal.visor.VisorMultiNodeTask; +import org.apache.ignite.internal.visor.VisorTaskArgument; +import org.apache.ignite.resources.LoggerResource; +import org.jetbrains.annotations.Nullable; + +/** + * Performs WAL cleanup clusterwide. + */ +@GridInternal +public class VisorWalTask extends VisorMultiNodeTask> { + /** */ + private static final long serialVersionUID = 0L; + + /** Pattern for segment file names. */ + private static final Pattern WAL_NAME_PATTERN = Pattern.compile("\\d{16}\\.wal"); + + /** Pattern for compacted segment file names. */ + private static final Pattern WAL_SEGMENT_FILE_COMPACTED_PATTERN = Pattern.compile("\\d{16}\\.wal\\.zip"); + + /** WAL archive file filter. */ + private static final FileFilter WAL_ARCHIVE_FILE_FILTER = new FileFilter() { + @Override public boolean accept(File file) { + return !file.isDirectory() && (WAL_NAME_PATTERN.matcher(file.getName()).matches() || + WAL_SEGMENT_FILE_COMPACTED_PATTERN.matcher(file.getName()).matches()); + } + }; + + /** {@inheritDoc} */ + @Override protected VisorWalJob job(VisorWalTaskArg arg) { + return new VisorWalJob(arg, debug); + } + + /** {@inheritDoc} */ + @Override protected Collection jobNodes(VisorTaskArgument arg) { + Collection srvNodes = ignite.cluster().forServers().nodes(); + Collection ret = new ArrayList<>(srvNodes.size()); + + VisorWalTaskArg taskArg = arg.getArgument(); + + Set nodeIds = taskArg.getConsistentIds() != null ? new HashSet<>(arg.getArgument().getConsistentIds()) + : null; + + if (nodeIds == null) { + for (ClusterNode node : srvNodes) + ret.add(node.id()); + } + else { + for (ClusterNode node : srvNodes) { + if (nodeIds.contains(node.consistentId().toString())) + ret.add(node.id()); + } + } + + return ret; + } + + /** {@inheritDoc} */ + @Nullable @Override protected VisorWalTaskResult reduce0(List results) throws IgniteException { + Map exRes = U.newHashMap(0); + Map> res = U.newHashMap(results.size()); + Map nodesInfo = U.newHashMap(results.size()); + + for (ComputeJobResult result: results){ + ClusterNode node = result.getNode(); + + String nodeId = node.consistentId().toString(); + + if(result.getException() != null) + exRes.put(nodeId, result.getException()); + else if (result.getData() != null) { + Collection data = result.getData(); + + if(data != null) + res.put(nodeId, data); + } + + nodesInfo.put(nodeId, new VisorClusterNode(node)); + } + + return new VisorWalTaskResult(res, exRes, nodesInfo); + } + + /** + * Performs WAL cleanup per node. + */ + private static class VisorWalJob extends VisorJob> { + /** */ + private static final long serialVersionUID = 0L; + + /** Auto injected logger */ + @LoggerResource + private transient IgniteLogger log; + + /** + * @param arg WAL task argument. + * @param debug Debug flag. + */ + public VisorWalJob(VisorWalTaskArg arg, boolean debug) { + super(arg, debug); + } + + /** {@inheritDoc} */ + @Nullable @Override protected Collection run(@Nullable VisorWalTaskArg arg) throws IgniteException { + try { + GridKernalContext cctx = ignite.context(); + + GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager)cctx.cache().context().database(); + FileWriteAheadLogManager wal = (FileWriteAheadLogManager)cctx.cache().context().wal(); + + if (dbMgr == null || arg == null || wal == null) + return null; + + switch (arg.getOperation()) { + case DELETE_UNUSED_WAL_SEGMENTS: + return deleteUnusedWalSegments(dbMgr, wal); + + case PRINT_UNUSED_WAL_SEGMENTS: + default: + return getUnusedWalSegments(dbMgr, wal); + + } + } + catch (IgniteCheckedException e){ + U.error(log, "Failed to perform WAL task", e); + + throw new IgniteException("Failed to perform WAL task", e); + } + } + + /** + * Get unused wal segments. + * + * @param wal Database manager. + * @return {@link Collection} of absolute paths of unused WAL segments. + * @throws IgniteCheckedException if failed. + */ + Collection getUnusedWalSegments( + GridCacheDatabaseSharedManager dbMgr, + FileWriteAheadLogManager wal + ) throws IgniteCheckedException{ + WALPointer lowBoundForTruncate = dbMgr.checkpointHistory().lowCheckpointBound(); + + if (lowBoundForTruncate == null) + return Collections.emptyList(); + + int maxIdx = resolveMaxReservedIndex(wal, lowBoundForTruncate); + + File[] walFiles = getWalArchiveDir().listFiles(WAL_ARCHIVE_FILE_FILTER); + + Collection res = new ArrayList<>(walFiles != null && walFiles.length > 0 ? walFiles.length - 1 : 0); + + if(walFiles != null && walFiles.length > 0) { + sortWalFiles(walFiles); + + // Obtain index of last archived WAL segment, it will not be deleted. + long lastArchIdx = getIndex(walFiles[walFiles.length - 1]); + + for (File f : walFiles) { + long fileIdx = getIndex(f); + + if (fileIdx < maxIdx && fileIdx < lastArchIdx) + res.add(f.getAbsolutePath()); + else + break; + } + } + + return res; + } + + /** + * Delete unused wal segments. + * + * @param dbMgr Database manager. + * @return {@link Collection} of deleted WAL segment's files. + * @throws IgniteCheckedException if failed. + */ + Collection deleteUnusedWalSegments( + GridCacheDatabaseSharedManager dbMgr, + FileWriteAheadLogManager wal + ) throws IgniteCheckedException { + WALPointer lowBoundForTruncate = dbMgr.checkpointHistory().lowCheckpointBound(); + + if (lowBoundForTruncate == null) + return Collections.emptyList(); + + int maxIdx = resolveMaxReservedIndex(wal, lowBoundForTruncate); + + File[] walFiles = getWalArchiveDir().listFiles(WAL_ARCHIVE_FILE_FILTER); + + dbMgr.onWalTruncated(lowBoundForTruncate); + + int num = wal.truncate(null, lowBoundForTruncate); + + if (walFiles != null) { + sortWalFiles(walFiles); + + Collection res = new ArrayList<>(num); + + for (File walFile: walFiles) { + if (getIndex(walFile) < maxIdx && num > 0) + res.add(walFile.getAbsolutePath()); + else + break; + + num--; + } + + return res; + } + else + return Collections.emptyList(); + + } + + /** + * + */ + private int resolveMaxReservedIndex(FileWriteAheadLogManager wal, WALPointer lowBoundForTruncate) { + FileWALPointer low = (FileWALPointer)lowBoundForTruncate; + + int resCnt = wal.reserved(null, lowBoundForTruncate); + + long highIdx = low.index(); + + return (int)(highIdx - resCnt + 1); + } + + /** + * Get WAL archive directory from configuration. + * + * @return WAL archive directory. + * @throws IgniteCheckedException if failed. + */ + private File getWalArchiveDir() throws IgniteCheckedException { + IgniteConfiguration igCfg = ignite.context().config(); + + DataStorageConfiguration dsCfg = igCfg.getDataStorageConfiguration(); + + PdsFolderSettings resFldrs = ignite.context().pdsFolderResolver().resolveFolders(); + + String consId = resFldrs.folderName(); + + File dir; + + if (dsCfg.getWalArchivePath() != null) { + File workDir0 = new File(dsCfg.getWalArchivePath()); + + dir = workDir0.isAbsolute() ? + new File(workDir0, consId) : + new File(U.resolveWorkDirectory(igCfg.getWorkDirectory(), dsCfg.getWalArchivePath(), false), + consId); + } + else + dir = new File(U.resolveWorkDirectory(igCfg.getWorkDirectory(), + DataStorageConfiguration.DFLT_WAL_ARCHIVE_PATH, false), consId); + + if (!dir.exists()) + throw new IgniteCheckedException("WAL archive directory does not exists" + dir.getAbsolutePath()); + + return dir; + } + + + /** + * Sort WAL files according their indices. + * + * @param files Array of WAL segment files. + */ + private void sortWalFiles(File[] files) { + Arrays.sort(files, new Comparator() { + @Override public int compare(File o1, File o2) { + return Long.compare(getIndex(o1), getIndex(o2)); + } + }); + } + } + + /** + * Get index from WAL segment file. + * + * @param file WAL segment file. + * @return Index of WAL segment file. + */ + private static long getIndex(File file) { + return Long.parseLong(file.getName().substring(0, 16)); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTaskArg.java new file mode 100644 index 0000000000000..afd3b5836fc70 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTaskArg.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.visor.misc; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.List; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * Argument for {@link VisorWalTask}. + */ +public class VisorWalTaskArg extends VisorDataTransferObject{ + /** */ + private static final long serialVersionUID = 0L; + + /** WAL task operation. */ + private VisorWalTaskOperation op; + + /** List of nodes' consistent ids. */ + private List consistentIds; + + /** + * Default constructor. + */ + public VisorWalTaskArg() { + // No-op. + } + + /** + * @param op Task operation. + */ + public VisorWalTaskArg(VisorWalTaskOperation op) { + this.op = op; + } + + /** + * @param op WAL task operation. + * @param consistentIds Nodes consistent ids. + */ + public VisorWalTaskArg(VisorWalTaskOperation op, List consistentIds) { + this.op = op; + this.consistentIds = consistentIds; + } + + /** + * Get WAL task operation. + * + * @return WAL task operation. + */ + public VisorWalTaskOperation getOperation() { + return op == null ? VisorWalTaskOperation.PRINT_UNUSED_WAL_SEGMENTS : op; + } + + /** + * Get nodes consistent ids. + * + * @return Consistent ids. + */ + public List getConsistentIds() { + return consistentIds; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + U.writeEnum(out, op); + U.writeCollection(out, consistentIds); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + op = VisorWalTaskOperation.fromOrdinal(in.readByte()); + consistentIds = U.readList(in); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorWalTaskArg.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTaskOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTaskOperation.java new file mode 100644 index 0000000000000..be2ff18456840 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTaskOperation.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.visor.misc; + +import org.jetbrains.annotations.Nullable; + +/** + * WAL Task operation types. + */ +public enum VisorWalTaskOperation { + /** Print unused wal segments. */ + PRINT_UNUSED_WAL_SEGMENTS, + + /** Delete unused wal segments. */ + DELETE_UNUSED_WAL_SEGMENTS; + + /** Enumerated values. */ + private static final VisorWalTaskOperation[] VALS = values(); + + /** + * Efficiently gets enumerated value from its ordinal. + * + * @param ord Ordinal value. + * @return Enumerated value or {@code null} if ordinal out of range. + */ + @Nullable public static VisorWalTaskOperation fromOrdinal(int ord) { + return ord >= 0 && ord < VALS.length ? VALS[ord] : null; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTaskResult.java new file mode 100644 index 0000000000000..104c2f3059508 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTaskResult.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.visor.misc; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.Collection; +import java.util.Map; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * Result of {@link VisorWalTask}. + */ +public class VisorWalTaskResult extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Exceptions by node consistent id. */ + @GridToStringInclude + private Map exceptions; + + /** Archived wal segments path search results by node consistent id. */ + @GridToStringInclude + private Map> results; + + /** Nodes info by node consistent id. */ + @GridToStringInclude + private Map nodesInfo; + + /** + * Default constructor. + */ + public VisorWalTaskResult() { + // No-op. + } + + /** + * Create {@link VisorWalTask } result with given parameters. + * + * @param results List of log search results. + * @param exceptions List of exceptions by node id. + * @param nodesInfo Nodes info. + */ + public VisorWalTaskResult(Map> results, Map exceptions, + Map nodesInfo) { + this.exceptions = exceptions; + this.results = results; + this.nodesInfo = nodesInfo; + } + + /** + * Get occurred errors by node consistent id. + * + * @return Exceptions by node consistent id. + */ + public Map exceptions() { + return exceptions; + } + + /** + * @return List of archived wal segments path search results by node consistent id. + */ + public Map> results() { + return results; + } + + /** + * Get nodes info by node consistent id. + * + * @return Nodes info. + */ + public Map getNodesInfo() { + return nodesInfo; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + U.writeMap(out, exceptions); + U.writeMap(out, results); + U.writeMap(out, nodesInfo); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + exceptions = U.readMap(in); + results = U.readMap(in); + nodesInfo = U.readMap(in); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorWalTaskResult.class, this); + } +} diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties index 8ca47c8430830..f1aec976e4c53 100644 --- a/modules/core/src/main/resources/META-INF/classnames.properties +++ b/modules/core/src/main/resources/META-INF/classnames.properties @@ -2028,6 +2028,12 @@ org.apache.ignite.internal.visor.misc.VisorNopTask org.apache.ignite.internal.visor.misc.VisorNopTask$VisorNopJob org.apache.ignite.internal.visor.misc.VisorResolveHostNameTask org.apache.ignite.internal.visor.misc.VisorResolveHostNameTask$VisorResolveHostNameJob +org.apache.ignite.internal.visor.misc.VisorClusterNode +org.apache.ignite.internal.visor.misc.VisorWalTask +org.apache.ignite.internal.visor.misc.VisorWalTask$VisorWalJob +org.apache.ignite.internal.visor.misc.VisorWalTaskArg +org.apache.ignite.internal.visor.misc.VisorWalTaskOperation +org.apache.ignite.internal.visor.misc.VisorWalTaskResult org.apache.ignite.internal.visor.node.VisorAffinityTopologyVersion org.apache.ignite.internal.visor.node.VisorAtomicConfiguration org.apache.ignite.internal.visor.node.VisorBasicConfiguration diff --git a/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java index 98b8e01c99ea3..2fc40ca167944 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java @@ -1,46 +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 * - * * Licensed to the Apache Software Foundation (ASF) under one or more - * * contributor license agreements. See the NOTICE file distributed with - * * this work for additional information regarding copyright ownership. - * * The ASF licenses this file to You under the Apache License, Version 2.0 - * * (the "License"); you may not use this file except in compliance with - * * the License. You may obtain a copy of the License at - * * - * * http://www.apache.org/licenses/LICENSE-2.0 - * * - * * Unless required by applicable law or agreed to in writing, software - * * distributed under the License is distributed on an "AS IS" BASIS, - * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * * See the License for the specific language governing permissions and - * * limitations under the License. + * http://www.apache.org/licenses/LICENSE-2.0 * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. */ package org.apache.ignite.internal.commandline; import java.util.Arrays; +import java.util.Collections; +import java.util.UUID; import junit.framework.TestCase; import org.apache.ignite.internal.visor.tx.VisorTxProjection; import org.apache.ignite.internal.visor.tx.VisorTxSortOrder; import org.apache.ignite.internal.visor.tx.VisorTxTaskArg; import static java.util.Arrays.asList; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_ENABLE_EXPERIMENTAL_COMMAND; +import static org.apache.ignite.internal.commandline.Command.WAL; import static org.apache.ignite.internal.commandline.CommandHandler.DFLT_HOST; import static org.apache.ignite.internal.commandline.CommandHandler.DFLT_PORT; +import static org.apache.ignite.internal.commandline.CommandHandler.WAL_DELETE; +import static org.apache.ignite.internal.commandline.CommandHandler.WAL_PRINT; /** * Tests Command Handler parsing arguments. */ public class CommandHandlerParsingTest extends TestCase { + /** {@inheritDoc} */ + @Override protected void setUp() throws Exception { + System.setProperty(IGNITE_ENABLE_EXPERIMENTAL_COMMAND, "true"); + + super.setUp(); + } + + /** {@inheritDoc} */ + @Override public void tearDown() throws Exception { + System.clearProperty(IGNITE_ENABLE_EXPERIMENTAL_COMMAND); + + super.tearDown(); + } + + /** + * Test that experimental command (i.e. WAL command) is disabled by default. + */ + public void testExperimentalCommandIsDisabled() { + System.clearProperty(IGNITE_ENABLE_EXPERIMENTAL_COMMAND); + + CommandHandler hnd = new CommandHandler(); + + try { + hnd.parseAndValidate(Arrays.asList(WAL.text(), WAL_PRINT)); + } + catch (Throwable e) { + e.printStackTrace(); + + assertTrue(e instanceof IllegalArgumentException); + } + + try { + hnd.parseAndValidate(Arrays.asList(WAL.text(), WAL_DELETE)); + } + catch (Throwable e) { + e.printStackTrace(); + + assertTrue(e instanceof IllegalArgumentException); + } + } + /** - * Test parsing and validation for user and password arguments. + * Tests parsing and validation for user and password arguments. */ public void testParseAndValidateUserAndPassword() { CommandHandler hnd = new CommandHandler(); for (Command cmd : Command.values()) { - if (cmd == Command.CACHE) + if (cmd == Command.CACHE || cmd == Command.WAL) continue; // --cache subcommand requires its own specific arguments. try { @@ -88,13 +133,53 @@ public void testParseAndValidateUserAndPassword() { } /** + * Tests parsing and validation of WAL commands. + */ + public void testParseAndValidateWalActions() { + CommandHandler hnd = new CommandHandler(); + + Arguments args = hnd.parseAndValidate(Arrays.asList(WAL.text(), WAL_PRINT)); + + assertEquals(WAL, args.command()); + + assertEquals(WAL_PRINT, args.walAction()); + + String nodes = UUID.randomUUID().toString() + "," + UUID.randomUUID().toString(); + + args = hnd.parseAndValidate(Arrays.asList(WAL.text(), WAL_DELETE, nodes)); + + assertEquals(WAL_DELETE, args.walAction()); + + assertEquals(nodes, args.walArguments()); + + try { + hnd.parseAndValidate(Collections.singletonList(WAL.text())); + + fail("expected exception: invalid arguments for --wal command"); + } + catch (IllegalArgumentException e) { + e.printStackTrace(); + } + + try { + hnd.parseAndValidate(Arrays.asList(WAL.text(), UUID.randomUUID().toString())); + + fail("expected exception: invalid arguments for --wal command"); + } + catch (IllegalArgumentException e) { + e.printStackTrace(); + } + } + + /** + * Tests host and port arguments. * Tests connection settings arguments. */ public void testConnectionSettings() { CommandHandler hnd = new CommandHandler(); for (Command cmd : Command.values()) { - if (cmd == Command.CACHE) + if (cmd == Command.CACHE || cmd == Command.WAL) continue; // --cache subcommand requires its own specific arguments. Arguments args = hnd.parseAndValidate(asList(cmd.text())); 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/IgnitePdsUnusedWalSegmentsTest.java new file mode 100644 index 0000000000000..0f522544107e4 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsUnusedWalSegmentsTest.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.persistence.db; + +import org.apache.ignite.IgniteCache; +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.pagemem.wal.IgniteWriteAheadLogManager; +import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer; +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.IgniteSystemProperties.IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE; + +/** + * Test correctness of truncating unused WAL segments. + */ +public class IgnitePdsUnusedWalSegmentsTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + System.setProperty(IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE, "2"); + + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER)); + + CacheConfiguration ccfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME); + + ccfg.setAffinity(new RendezvousAffinityFunction(false, 32)); + + cfg.setCacheConfiguration(ccfg); + + DataStorageConfiguration dbCfg = new DataStorageConfiguration(); + + dbCfg.setPageSize(4 * 1024); + + 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)); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + cleanPersistenceDir(); + } + + /** + * Tests that range reserved method return correct number of reserved WAL segments. + * + * @throws Exception if failed. + */ + public void testWalManagerRangeReservation() throws Exception { + try{ + IgniteEx ig0 = prepareGrid(4); + + GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager) ig0.context().cache().context() + .database(); + + IgniteWriteAheadLogManager wal = ig0.context().cache().context().wal(); + + long resIdx = getReservedWalSegmentIndex(dbMgr); + + assertTrue("Expected that at least resIdx greater than 0, real is " + resIdx, resIdx > 0); + + FileWALPointer lowPtr = (FileWALPointer)dbMgr.checkpointHistory().lowCheckpointBound(); + + assertTrue("Expected that dbMbr returns valid resIdx", lowPtr.index() == resIdx); + + // 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)); + + assertTrue("Expected resCnt is 2, real is " + resCnt, resCnt == 2); + } + finally { + stopAllGrids(); + } + } + + /** + * Tests that grid cache manager correctly truncates unused WAL segments; + * + * @throws Exception if failed. + */ + public void testUnusedWalTruncate() throws Exception { + try{ + IgniteEx ig0 = prepareGrid(4); + + GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager) ig0.context().cache().context() + .database(); + + IgniteWriteAheadLogManager wal = ig0.context().cache().context().wal(); + + long resIdx = getReservedWalSegmentIndex(dbMgr); + + assertTrue("Expected that at least resIdx greater than 0, real is " + resIdx, resIdx > 0); + + FileWALPointer lowPtr = (FileWALPointer) dbMgr.checkpointHistory().lowCheckpointBound(); + + assertTrue("Expected that dbMbr returns valid resIdx", lowPtr.index() == resIdx); + + // Reserve previous WAL segment. + wal.reserve(new FileWALPointer(resIdx - 1, 0, 0)); + + int numDel = wal.truncate(null, lowPtr); + + int expNumDel = (int)resIdx - 1; + + assertTrue("Expected del segments is " + expNumDel + ", real is " + numDel, expNumDel == numDel); + } + finally { + stopAllGrids(); + } + } + + /** + * Starts grid and populates test data. + * + * @param cnt Grid count. + * @return First started grid. + * @throws Exception If failed. + */ + private IgniteEx prepareGrid(int cnt) throws Exception { + IgniteEx ig0 = (IgniteEx)startGrids(cnt); + + ig0.cluster().active(true); + + IgniteCache cache = ig0.cache(DEFAULT_CACHE_NAME); + + for (int k = 0; k < 10_000; k++) + cache.put(k, new byte[1024]); + + forceCheckpoint(); + + for (int k = 0; k < 1_000; k++) + cache.put(k, new byte[1024]); + + forceCheckpoint(); + + return ig0; + } + + + /** + * Get index of reserved WAL segment by checkpointer. + * + * @param dbMgr Database shared manager. + * @throws Exception If failed. + */ + private long getReservedWalSegmentIndex(GridCacheDatabaseSharedManager dbMgr) throws Exception{ + GridCacheDatabaseSharedManager.CheckpointHistory cpHist = dbMgr.checkpointHistory(); + + Object histMap = GridTestUtils.getFieldValue(cpHist, "histMap"); + + Object cpEntry = GridTestUtils.getFieldValue(GridTestUtils.invoke(histMap, "firstEntry"), "value"); + + FileWALPointer walPtr = GridTestUtils.getFieldValue(cpEntry, "cpMark"); + + return walPtr.index(); + } +} 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 16d1d0a6872c1..c95d1f4a61340 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 @@ -91,6 +91,11 @@ public class NoOpWALManager implements IgniteWriteAheadLogManager { return false; } + /** {@inheritDoc} */ + @Override public int reserved(WALPointer low, WALPointer high) { + return 0; + } + /** {@inheritDoc} */ @Override public boolean disabled(int grpId) { return false; @@ -127,7 +132,7 @@ public class NoOpWALManager implements IgniteWriteAheadLogManager { } /** {@inheritDoc} */ - @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException { + @Override public void onActivate(GridKernalContext kctx) { // No-op. } 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 6d953cdd7b2de..71c6c32a983cb 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 @@ -36,6 +36,7 @@ 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.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.IgniteWalFlushBackgroundSelfTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushDefaultSelfTest; @@ -127,6 +128,8 @@ public static void addRealPageStoreTests(TestSuite suite) { suite.addTestSuite(IgnitePdsExchangeDuringCheckpointTest.class); + suite.addTestSuite(IgnitePdsUnusedWalSegmentsTest.class); + // new style folders with generated consistent ID test suite.addTestSuite(IgniteUidAsConsistentIdMigrationTest.class); 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 e3be1e3f22312..124300585b4e7 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 @@ -43,6 +43,7 @@ import org.apache.ignite.thread.IgniteThreadPoolSizeTest; import org.apache.ignite.util.GridCommandHandlerTest; import org.apache.ignite.util.GridIntListSelfTest; +import org.apache.ignite.util.GridInternalTaskUnusedWalSegmentsTest; import org.apache.ignite.util.GridLongListSelfTest; import org.apache.ignite.util.GridMessageCollectionTest; import org.apache.ignite.util.GridPartitionMapSelfTest; @@ -116,6 +117,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { // control.sh suite.addTestSuite(CommandHandlerParsingTest.class); suite.addTestSuite(GridCommandHandlerTest.class); + suite.addTestSuite(GridInternalTaskUnusedWalSegmentsTest.class); return suite; } diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java index dbd6107f737b1..4daa92cdf000c 100644 --- a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java @@ -30,6 +30,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.List; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteAtomicSequence; import org.apache.ignite.IgniteCache; @@ -57,6 +58,7 @@ import org.apache.ignite.transactions.Transaction; import org.apache.ignite.transactions.TransactionRollbackException; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_ENABLE_EXPERIMENTAL_COMMAND; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; import static org.apache.ignite.internal.commandline.CommandHandler.EXIT_CODE_OK; @@ -85,6 +87,8 @@ protected File folder(String folder) throws IgniteCheckedException { /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { + System.setProperty(IGNITE_ENABLE_EXPERIMENTAL_COMMAND, "true"); + cleanPersistenceDir(); stopAllGrids(); @@ -100,6 +104,8 @@ protected File folder(String folder) throws IgniteCheckedException { cleanPersistenceDir(); + System.clearProperty(IGNITE_ENABLE_EXPERIMENTAL_COMMAND); + System.setOut(sysOut); if (testOut != null) @@ -787,4 +793,70 @@ private Map generate(int from, int cnt) { return map; } + + /** + * Test execution of --wal print command. + * + * @throws Exception if failed. + */ + public void testUnusedWalPrint() throws Exception { + Ignite ignite = startGrids(2); + + ignite.cluster().active(true); + + List nodes = new ArrayList<>(2); + + for (ClusterNode node: ignite.cluster().forServers().nodes()) + nodes.add(node.consistentId().toString()); + + injectTestSystemOut(); + + assertEquals(EXIT_CODE_OK, execute("--wal", "print")); + + for(String id: nodes) + assertTrue(testOut.toString().contains(id)); + + assertTrue(!testOut.toString().contains("error")); + + testOut.reset(); + + assertEquals(EXIT_CODE_OK, execute("--wal", "print", nodes.get(0))); + + assertTrue(!testOut.toString().contains(nodes.get(1))); + + assertTrue(!testOut.toString().contains("error")); + } + + /** + * Test execution of --wal delete command. + * + * @throws Exception if failed. + */ + public void testUnusedWalDelete() throws Exception { + Ignite ignite = startGrids(2); + + ignite.cluster().active(true); + + List nodes = new ArrayList<>(2); + + for (ClusterNode node: ignite.cluster().forServers().nodes()) + nodes.add(node.consistentId().toString()); + + injectTestSystemOut(); + + assertEquals(EXIT_CODE_OK, execute("--wal", "delete")); + + for(String id: nodes) + assertTrue(testOut.toString().contains(id)); + + assertTrue(!testOut.toString().contains("error")); + + testOut.reset(); + + assertEquals(EXIT_CODE_OK, execute("--wal", "delete", nodes.get(0))); + + assertTrue(!testOut.toString().contains(nodes.get(1))); + + assertTrue(!testOut.toString().contains("error")); + } } diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridInternalTaskUnusedWalSegmentsTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridInternalTaskUnusedWalSegmentsTest.java new file mode 100644 index 0000000000000..148ecef37d3f9 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/util/GridInternalTaskUnusedWalSegmentsTest.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.util; + +import java.io.File; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import org.apache.ignite.IgniteCache; +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.visor.VisorTaskArgument; +import org.apache.ignite.internal.visor.misc.VisorWalTask; +import org.apache.ignite.internal.visor.misc.VisorWalTaskArg; +import org.apache.ignite.internal.visor.misc.VisorWalTaskOperation; +import org.apache.ignite.internal.visor.misc.VisorWalTaskResult; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE; + +/** + * Test correctness of VisorWalTask. + */ +public class GridInternalTaskUnusedWalSegmentsTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + System.setProperty(IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE, "2"); + + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER)); + + CacheConfiguration ccfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME); + + ccfg.setAffinity(new RendezvousAffinityFunction(false, 32)); + + cfg.setCacheConfiguration(ccfg); + + DataStorageConfiguration dbCfg = new DataStorageConfiguration(); + + dbCfg.setPageSize(4 * 1024); + + 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)); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + cleanPersistenceDir(); + } + + /** + * Tests correctness of {@link VisorWalTaskOperation}. + * + * @throws Exception if failed. + */ + public void testCorrectnessOfDeletionTaskSegments() throws Exception { + try { + IgniteEx ig0 = (IgniteEx)startGrids(4); + + ig0.cluster().active(true); + + IgniteCache cache = ig0.cache(DEFAULT_CACHE_NAME); + + for (int k = 0; k < 10_000; k++) + cache.put(k, new byte[1024]); + + forceCheckpoint(); + + for (int k = 0; k < 1_000; k++) + cache.put(k, new byte[1024]); + + forceCheckpoint(); + + VisorWalTaskResult printRes = ig0.compute().execute(VisorWalTask.class, + new VisorTaskArgument<>(ig0.cluster().node().id(), + new VisorWalTaskArg(VisorWalTaskOperation.PRINT_UNUSED_WAL_SEGMENTS), false)); + + assertEquals("Check that print task finished without exceptions", printRes.results().size(), 4); + + List walArchives = new ArrayList<>(); + + for (Collection pathsPerNode : printRes.results().values()) { + for (String path : pathsPerNode) + walArchives.add(Paths.get(path).toFile()); + } + + VisorWalTaskResult delRes = ig0.compute().execute(VisorWalTask.class, + new VisorTaskArgument<>(ig0.cluster().node().id(), + new VisorWalTaskArg(VisorWalTaskOperation.DELETE_UNUSED_WAL_SEGMENTS), false)); + + assertEquals("Check that delete task finished with no exceptions", delRes.results().size(), 4); + + List walDeletedArchives = new ArrayList<>(); + + for (Collection pathsPerNode : delRes.results().values()) { + for (String path : pathsPerNode) + walDeletedArchives.add(Paths.get(path).toFile()); + } + + for (File f : walDeletedArchives) + assertTrue("Checking existing of deleted WAL archived segments: " + f.getAbsolutePath(), !f.exists()); + + for (File f : walArchives) + assertTrue("Checking existing of WAL archived segments from print task after delete: " + f.getAbsolutePath(), + !f.exists()); + } + finally { + stopAllGrids(); + } + } +} From ed6fcce7a5f0e48adf5d0203c8dcf00e04473523 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Thu, 10 May 2018 18:12:41 +0300 Subject: [PATCH 0156/1463] IGNITE-7535: SQL: Added encoding option to COPY command. This closes #3487. --- .../examples/sql/SqlJdbcCopyExample.java | 4 +- .../JdbcThinBulkLoadAbstractSelfTest.java | 408 ++++++++++++++---- .../{bulkload2_utf.csv => bulkload2_utf8.csv} | 0 .../test/resources/bulkload2_windows1251.csv | 2 + .../ignite/internal/jdbc2/JdbcQueryTask.java | 2 +- .../bulkload/BulkLoadCacheWriter.java | 4 +- .../bulkload/BulkLoadCsvFormat.java | 25 +- .../bulkload/BulkLoadCsvParser.java | 19 +- .../processors/bulkload/BulkLoadFormat.java | 4 +- .../pipeline/CharsetDecoderBlock.java | 15 +- .../bulkload/pipeline/PipelineBlock.java | 5 +- .../ignite/internal/sql/SqlKeyword.java | 3 + .../internal/sql/SqlParseException.java | 13 + .../apache/ignite/internal/sql/SqlParser.java | 11 +- .../internal/sql/SqlStrictParseException.java | 32 ++ .../sql/command/SqlBulkLoadCommand.java | 32 +- .../sql/SqlParserBulkLoadSelfTest.java | 64 ++- .../processors/query/h2/IgniteH2Indexing.java | 5 +- .../yardstick/upload/model/QueryFactory.java | 2 +- 19 files changed, 538 insertions(+), 112 deletions(-) rename modules/clients/src/test/resources/{bulkload2_utf.csv => bulkload2_utf8.csv} (100%) create mode 100644 modules/clients/src/test/resources/bulkload2_windows1251.csv create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/sql/SqlStrictParseException.java diff --git a/examples/src/main/java/org/apache/ignite/examples/sql/SqlJdbcCopyExample.java b/examples/src/main/java/org/apache/ignite/examples/sql/SqlJdbcCopyExample.java index 1271b3935ff1f..394c3b0425dbb 100644 --- a/examples/src/main/java/org/apache/ignite/examples/sql/SqlJdbcCopyExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/sql/SqlJdbcCopyExample.java @@ -58,8 +58,8 @@ public static void main(String[] args) throws Exception { print("Created database objects."); // Load data from CSV file. - executeCommand(conn, "COPY FROM \"" + - IgniteUtils.resolveIgnitePath("examples/src/main/resources/sql/city.csv") + "\" " + + executeCommand(conn, "COPY FROM '" + + IgniteUtils.resolveIgnitePath("examples/src/main/resources/sql/city.csv") + "' " + "INTO City (ID, Name, CountryCode, District, Population) FORMAT CSV"); // Read data. diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAbstractSelfTest.java index e9cb8325a1ecf..2a4c7995fbdf6 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAbstractSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAbstractSelfTest.java @@ -22,9 +22,16 @@ import org.apache.ignite.cache.QueryEntity; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; +import org.apache.ignite.internal.processors.bulkload.BulkLoadCsvFormat; +import org.apache.ignite.internal.processors.bulkload.BulkLoadCsvParser; import org.apache.ignite.internal.processors.query.QueryUtils; +import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.testframework.GridTestUtils; +import java.nio.ByteBuffer; +import java.nio.charset.Charset; +import java.nio.charset.CodingErrorAction; +import java.nio.charset.UnsupportedCharsetException; import java.sql.BatchUpdateException; import java.sql.PreparedStatement; import java.sql.ResultSet; @@ -35,7 +42,6 @@ import java.util.Objects; import java.util.concurrent.Callable; -import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_PARSER_DISABLE_H2_FALLBACK; import static org.apache.ignite.cache.CacheMode.PARTITIONED; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; import static org.apache.ignite.internal.util.IgniteUtils.resolveIgnitePath; @@ -47,9 +53,13 @@ public abstract class JdbcThinBulkLoadAbstractSelfTest extends JdbcThinAbstractD /** Subdirectory with CSV files */ private static final String CSV_FILE_SUBDIR = "/modules/clients/src/test/resources/"; + /** Default table name. */ + private static final String TBL_NAME = "Person"; + /** A CSV file with zero records */ private static final String BULKLOAD_EMPTY_CSV_FILE = - Objects.requireNonNull(resolveIgnitePath(CSV_FILE_SUBDIR + "bulkload0.csv")).getAbsolutePath(); + Objects.requireNonNull(resolveIgnitePath(CSV_FILE_SUBDIR + "bulkload0.csv")) + .getAbsolutePath(); /** A CSV file with one record. */ private static final String BULKLOAD_ONE_LINE_CSV_FILE = @@ -60,18 +70,19 @@ public abstract class JdbcThinBulkLoadAbstractSelfTest extends JdbcThinAbstractD Objects.requireNonNull(resolveIgnitePath(CSV_FILE_SUBDIR + "bulkload2.csv")).getAbsolutePath(); /** A CSV file in UTF-8. */ - private static final String BULKLOAD_UTF_CSV_FILE = - Objects.requireNonNull(resolveIgnitePath(CSV_FILE_SUBDIR + "bulkload2_utf.csv")).getAbsolutePath(); + private static final String BULKLOAD_UTF8_CSV_FILE = + Objects.requireNonNull(resolveIgnitePath(CSV_FILE_SUBDIR + "bulkload2_utf8.csv")).getAbsolutePath(); - /** Default table name. */ - private static final String TBL_NAME = "Person"; + /** A CSV file in windows-1251. */ + private static final String BULKLOAD_CP1251_CSV_FILE = + Objects.requireNonNull(resolveIgnitePath(CSV_FILE_SUBDIR + "bulkload2_windows1251.csv")).getAbsolutePath(); /** Basic COPY statement used in majority of the tests. */ public static final String BASIC_SQL_COPY_STMT = - "copy from \"" + BULKLOAD_TWO_LINES_CSV_FILE + "\"" + - " into " + TBL_NAME + - " (_key, age, firstName, lastName)" + - " format csv"; + "copy from '" + BULKLOAD_TWO_LINES_CSV_FILE + "'" + + " into " + TBL_NAME + + " (_key, age, firstName, lastName)" + + " format csv"; /** JDBC statement. */ private Statement stmt; @@ -161,8 +172,6 @@ private CacheConfiguration cacheConfigWithQueryEntity() { @Override protected void beforeTest() throws Exception { super.beforeTest(); - System.setProperty(IGNITE_SQL_PARSER_DISABLE_H2_FALLBACK, "TRUE"); - stmt = conn.createStatement(); assertNotNull(stmt); @@ -176,8 +185,6 @@ private CacheConfiguration cacheConfigWithQueryEntity() { assertTrue(stmt.isClosed()); - System.clearProperty(IGNITE_SQL_PARSER_DISABLE_H2_FALLBACK); - super.afterTest(); } @@ -203,9 +210,9 @@ public void testBasicStatement() throws SQLException { */ public void testEmptyFile() throws SQLException { int updatesCnt = stmt.executeUpdate( - "copy from \"" + BULKLOAD_EMPTY_CSV_FILE + "\" into " + TBL_NAME + - " (_key, age, firstName, lastName)" + - " format csv"); + "copy from '" + BULKLOAD_EMPTY_CSV_FILE + "' into " + TBL_NAME + + " (_key, age, firstName, lastName)" + + " format csv"); assertEquals(0, updatesCnt); @@ -219,35 +226,161 @@ public void testEmptyFile() throws SQLException { */ public void testOneLineFile() throws SQLException { int updatesCnt = stmt.executeUpdate( - "copy from \"" + BULKLOAD_ONE_LINE_CSV_FILE + "\" into " + TBL_NAME + - " (_key, age, firstName, lastName)" + - " format csv"); + "copy from '" + BULKLOAD_ONE_LINE_CSV_FILE + "' into " + TBL_NAME + + " (_key, age, firstName, lastName)" + + " format csv"); assertEquals(1, updatesCnt); checkCacheContents(TBL_NAME, true, 1); } + /** + * Verifies that error is reported for empty charset name. + */ + public void testEmptyCharset() { + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + stmt.executeUpdate( + "copy from 'any.file' into Person " + + "(_key, age, firstName, lastName) " + + "format csv charset ''"); + + return null; + } + }, SQLException.class, "Unknown charset name: ''"); + } + + /** + * Verifies that error is reported for unsupported charset name. + */ + public void testNotSupportedCharset() { + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + stmt.executeUpdate( + "copy from 'any.file' into Person " + + "(_key, age, firstName, lastName) " + + "format csv charset 'nonexistent'"); + + return null; + } + }, SQLException.class, "Charset is not supported: 'nonexistent'"); + } + + /** + * Verifies that error is reported for unknown charset name. + */ + public void testUnknownCharset() { + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + stmt.executeUpdate( + "copy from 'any.file' into Person " + + "(_key, age, firstName, lastName) " + + "format csv charset '8^)\'"); + + return null; + } + }, SQLException.class, "Unknown charset name: '8^)'"); + } + + /** + * Verifies that ASCII encoding is recognized and imported. + * + * @throws SQLException If failed. + */ + public void testAsciiCharset() throws SQLException { + int updatesCnt = stmt.executeUpdate( + "copy from '" + BULKLOAD_TWO_LINES_CSV_FILE + "'" + + " into " + TBL_NAME + + " (_key, age, firstName, lastName)" + + " format csv charset 'ascii'"); + + assertEquals(2, updatesCnt); + + checkCacheContents(TBL_NAME, true, 2); + } + /** * Imports two-entry CSV file with UTF-8 characters into a table and checks * the created entries using SELECT statement. * * @throws SQLException If failed. */ - public void testUtf() throws SQLException { + public void testUtf8Charset() throws SQLException { + checkBulkLoadWithCharset(BULKLOAD_UTF8_CSV_FILE, "utf-8"); + } + + /** + * Verifies that ASCII encoding is recognized and imported. + * + * @throws SQLException If failed. + */ + public void testWin1251Charset() throws SQLException { + checkBulkLoadWithCharset(BULKLOAD_CP1251_CSV_FILE, "windows-1251"); + } + + /** + * Bulk-loads specified file specifying charset in the command + * and verifies the entries imported. + * + * @param fileName CSV file to load. + * @param charsetName Charset name to specify in the command. + * @throws SQLException If failed. + */ + private void checkBulkLoadWithCharset(String fileName, String charsetName) throws SQLException { int updatesCnt = stmt.executeUpdate( - "copy from \"" + BULKLOAD_UTF_CSV_FILE + "\" into " + TBL_NAME + - " (_key, age, firstName, lastName)" + - " format csv"); + "copy from '" + fileName + "' into " + TBL_NAME + + " (_key, age, firstName, lastName)" + + " format csv charset '" + charsetName + "'"); assertEquals(2, updatesCnt); - checkUtfCacheContents(TBL_NAME, true, 2); + checkNationalCacheContents(TBL_NAME); + } + + /** + * Verifies that no error is reported and characters are converted improperly when we import + * UTF-8 as windows-1251. + * + * @throws SQLException If failed. + */ + public void testWrongCharset_Utf8AsWin1251() throws SQLException { + checkBulkLoadWithWrongCharset(BULKLOAD_UTF8_CSV_FILE, "UTF-8", "windows-1251"); + } + + /** + * Verifies that no error is reported and characters are converted improperly when we import + * windows-1251 as UTF-8. + * + * @throws SQLException If failed. + */ + public void testWrongCharset_Win1251AsUtf8() throws SQLException { + checkBulkLoadWithWrongCharset(BULKLOAD_CP1251_CSV_FILE, "windows-1251", "UTF-8"); + } + + /** + * Verifies that no error is reported and characters are converted improperly when we import + * UTF-8 as ASCII. + * + * @throws SQLException If failed. + */ + public void testWrongCharset_Utf8AsAscii() throws SQLException { + checkBulkLoadWithWrongCharset(BULKLOAD_UTF8_CSV_FILE, "UTF-8", "ascii"); + } + + /** + * Verifies that no error is reported and characters are converted improperly when we import + * windows-1251 as ASCII. + * + * @throws SQLException If failed. + */ + public void testWrongCharset_Win1251AsAscii() throws SQLException { + checkBulkLoadWithWrongCharset(BULKLOAD_CP1251_CSV_FILE, "windows-1251", "ascii"); } /** - * Checks that bulk load works when we use batch size of 1 byte and thus - * create multiple batches per COPY. + * Checks that bulk load works when we use packet size of 1 byte and thus + * create multiple packets per COPY. * * @throws SQLException If failed. */ @@ -259,22 +392,39 @@ public void testPacketSize_1() throws SQLException { checkCacheContents(TBL_NAME, true, 2); } + /** + * Imports two-entry CSV file with UTF-8 characters into a table and checks + * the created entries using SELECT statement. + * + * @throws SQLException If failed. + */ + public void testDefaultCharset() throws SQLException { + int updatesCnt = stmt.executeUpdate( + "copy from '" + BULKLOAD_UTF8_CSV_FILE + "' into " + TBL_NAME + + " (_key, age, firstName, lastName)" + + " format csv"); + + assertEquals(2, updatesCnt); + + checkNationalCacheContents(TBL_NAME); + } + /** * Imports two-entry CSV file with UTF-8 characters into a table using packet size of one byte - * (thus splitting each two-byte UTF-8 character into two batches) + * (thus splitting each two-byte UTF-8 character into two packets) * and checks the created entries using SELECT statement. * * @throws SQLException If failed. */ - public void testUtfPacketSize_1() throws SQLException { + public void testDefaultCharsetPacketSize1() throws SQLException { int updatesCnt = stmt.executeUpdate( - "copy from \"" + BULKLOAD_UTF_CSV_FILE + "\" into " + TBL_NAME + - " (_key, age, firstName, lastName)" + - " format csv packet_size 1"); + "copy from '" + BULKLOAD_UTF8_CSV_FILE + "' into " + TBL_NAME + + " (_key, age, firstName, lastName)" + + " format csv packet_size 1"); assertEquals(2, updatesCnt); - checkUtfCacheContents(TBL_NAME, true, 2); + checkNationalCacheContents(TBL_NAME); } /** @@ -284,9 +434,9 @@ public void testWrongFileName() { GridTestUtils.assertThrows(log, new Callable() { @Override public Object call() throws Exception { stmt.executeUpdate( - "copy from \"nonexistent\" into Person" + - " (_key, age, firstName, lastName)" + - " format csv"); + "copy from 'nonexistent' into Person" + + " (_key, age, firstName, lastName)" + + " format csv"); return null; } @@ -300,9 +450,9 @@ public void testMissingTable() { GridTestUtils.assertThrows(log, new Callable() { @Override public Object call() throws Exception { stmt.executeUpdate( - "copy from \"" + BULKLOAD_TWO_LINES_CSV_FILE + "\" into Peterson" + - " (_key, age, firstName, lastName)" + - " format csv"); + "copy from '" + BULKLOAD_TWO_LINES_CSV_FILE + "' into Peterson" + + " (_key, age, firstName, lastName)" + + " format csv"); return null; } @@ -316,9 +466,9 @@ public void testWrongColumnName() { GridTestUtils.assertThrows(log, new Callable() { @Override public Object call() throws Exception { stmt.executeUpdate( - "copy from \"" + BULKLOAD_TWO_LINES_CSV_FILE + "\" into Person" + - " (_key, age, firstName, lostName)" + - " format csv"); + "copy from '" + BULKLOAD_TWO_LINES_CSV_FILE + "' into Person" + + " (_key, age, firstName, lostName)" + + " format csv"); return null; } @@ -332,9 +482,9 @@ public void testWrongColumnType() { GridTestUtils.assertThrows(log, new Callable() { @Override public Object call() throws Exception { stmt.executeUpdate( - "copy from \"" + BULKLOAD_TWO_LINES_CSV_FILE + "\" into Person" + - " (_key, firstName, age, lastName)" + - " format csv"); + "copy from '" + BULKLOAD_TWO_LINES_CSV_FILE + "' into Person" + + " (_key, firstName, age, lastName)" + + " format csv"); return null; } @@ -348,9 +498,10 @@ public void testWrongColumnType() { */ public void testFieldsSubset() throws SQLException { int updatesCnt = stmt.executeUpdate( - "copy from \"" + BULKLOAD_TWO_LINES_CSV_FILE + "\" into " + TBL_NAME + - " (_key, age, firstName)" + - " format csv"); + "copy from '" + BULKLOAD_TWO_LINES_CSV_FILE + "'" + + " into " + TBL_NAME + + " (_key, age, firstName)" + + " format csv"); assertEquals(2, updatesCnt); @@ -372,9 +523,9 @@ public void testCreateAndBulkLoadTable() throws SQLException { " (id int primary key, age int, firstName varchar(30), lastName varchar(30))"); int updatesCnt = stmt.executeUpdate( - "copy from \"" + BULKLOAD_TWO_LINES_CSV_FILE + "\" into " + tblName + - "(_key, age, firstName, lastName)" + - " format csv"); + "copy from '" + BULKLOAD_TWO_LINES_CSV_FILE + "' into " + tblName + + "(_key, age, firstName, lastName)" + + " format csv"); assertEquals(2, updatesCnt); @@ -401,18 +552,20 @@ public void testConfigureQueryEntityAndBulkLoad() throws SQLException { } /** - * Verifies exception thrown if COPY is added into a batch. + * Verifies exception thrown if COPY is added into a packet. + * + * @throws SQLException If failed. */ - public void testMultipleStatement() { + public void testMultipleStatement() throws SQLException { GridTestUtils.assertThrows(log, new Callable() { @Override public Object call() throws Exception { stmt.addBatch(BASIC_SQL_COPY_STMT); - stmt.addBatch("copy from \"" + BULKLOAD_ONE_LINE_CSV_FILE + "\" into " + TBL_NAME + + stmt.addBatch("copy from '" + BULKLOAD_ONE_LINE_CSV_FILE + "' into " + TBL_NAME + " (_key, age, firstName, lastName)" + " format csv"); - stmt.addBatch("copy from \"" + BULKLOAD_UTF_CSV_FILE + "\" into " + TBL_NAME + + stmt.addBatch("copy from '" + BULKLOAD_UTF8_CSV_FILE + "' into " + TBL_NAME + " (_key, age, firstName, lastName)" + " format csv"); @@ -425,8 +578,10 @@ public void testMultipleStatement() { /** * Verifies that COPY command is rejected by Statement.executeQuery(). + * + * @throws SQLException If failed. */ - public void testExecuteQuery() { + public void testExecuteQuery() throws SQLException { GridTestUtils.assertThrows(log, new Callable() { @Override public Object call() throws Exception { stmt.executeQuery(BASIC_SQL_COPY_STMT); @@ -466,22 +621,24 @@ public void testPreparedStatementWithExecuteUpdate() throws SQLException { /** * Verifies that COPY command reports an error when used with PreparedStatement parameter. + * + * @throws SQLException If failed. */ - public void testPreparedStatementWithParameter() { + public void testPreparedStatementWithParameter() throws SQLException { GridTestUtils.assertThrows(log, new Callable() { - @Override public Object call() throws Exception { - PreparedStatement pstmt = conn.prepareStatement( - "copy from \"" + BULKLOAD_TWO_LINES_CSV_FILE + "\" into " + TBL_NAME + + @Override public Object call() throws Exception { + PreparedStatement pstmt = conn.prepareStatement( + "copy from '" + BULKLOAD_TWO_LINES_CSV_FILE + "' into " + TBL_NAME + " (_key, age, firstName, lastName)" + " format ?"); - pstmt.setString(1, "csv"); + pstmt.setString(1, "csv"); - pstmt.executeUpdate(); + pstmt.executeUpdate(); - return null; - } - }, SQLException.class, "Unexpected token: \"?\" (expected: \"[identifier]\""); + return null; + } + }, SQLException.class, "Unexpected token: \"?\" (expected: \"[identifier]\""); } /** @@ -515,7 +672,9 @@ public void testPreparedStatementWithExecuteQuery() { } /** - * Checks cache contents for a typical test using SQL SELECT command. + * Checks cache contents after bulk loading data in the above tests: ASCII version. + *

    + * Uses SQL SELECT command for querying entries. * * @param tblName Table name to query. * @param checkLastName Check 'lastName' column (not imported in some tests). @@ -554,18 +713,47 @@ else if (id == 456) { } /** - * Checks cache contents for a UTF-8 bulk load tests using SQL SELECT command. + * Checks cache contents after bulk loading data in the above tests: + * national charset version. + *

    + * Uses SQL SELECT command for querying entries. * * @param tblName Table name to query. - * @param checkLastName Check 'lastName' column (not imported in some tests). - * @param recCnt Number of records to expect. * @throws SQLException When one of checks has failed. */ - private void checkUtfCacheContents(String tblName, boolean checkLastName, int recCnt) throws SQLException { + private void checkNationalCacheContents(String tblName) throws SQLException { + checkRecodedNationalCacheContents(tblName, null, null); + } + + /** + * Checks cache contents after bulk loading data in the tests: + * normal and erroneously recoded national charset version. + *

    + * Uses SQL SELECT command for querying entries. + * + * @param tblName Table name to query. + * @param csvCharsetName Either null or the charset used in CSV file + * Note that the both {@code csvCharsetName} and {@code stmtCharsetName} should be either null or non-null. + * @param stmtCharsetName Either null or the charset specified in COPY statement. + * @throws SQLException When one of checks has failed. + */ + private void checkRecodedNationalCacheContents(String tblName, + String csvCharsetName, String stmtCharsetName) throws SQLException { + assert (csvCharsetName != null) == (stmtCharsetName != null); + ResultSet rs = stmt.executeQuery("select _key, age, firstName, lastName from " + tblName); assert rs != null; + IgniteClosure recoder = + (csvCharsetName != null) + ? new WrongCharsetRecoder(csvCharsetName, stmtCharsetName) + : new IgniteClosure() { + @Override public String apply(String input) { + return input; + } + }; + int cnt = 0; while (rs.next()) { @@ -573,15 +761,17 @@ private void checkUtfCacheContents(String tblName, boolean checkLastName, int re if (id == 123) { assertEquals(12, rs.getInt("age")); - assertEquals("ИмÑ123 ОтчеÑтво123", rs.getString("firstName")); - if (checkLastName) - assertEquals("ФамилиÑ123", rs.getString("lastName")); + + assertEquals(recoder.apply("ИмÑ123 ОтчеÑтво123"), rs.getString("firstName")); + + assertEquals(recoder.apply("ФамилиÑ123"), rs.getString("lastName")); } else if (id == 456) { assertEquals(45, rs.getInt("age")); - assertEquals("ИмÑ456", rs.getString("firstName")); - if (checkLastName) - assertEquals("ФамилиÑ456", rs.getString("lastName")); + + assertEquals(recoder.apply("ИмÑ456"), rs.getString("firstName")); + + assertEquals(recoder.apply("ФамилиÑ456"), rs.getString("lastName")); } else fail("Wrong ID: " + id); @@ -589,6 +779,70 @@ else if (id == 456) { cnt++; } - assertEquals(recCnt, cnt); + assertEquals(2, cnt); + } + + /** + * Checks that no error is reported and characters are converted improperly when we import + * file having a different charset than the one specified in the SQL statement. + * + * @param csvFileName Imported file name. + * @param csvCharsetName Imported file charset. + * @param stmtCharsetName Charset to specify in the SQL statement. + * @throws SQLException If failed. + */ + private void checkBulkLoadWithWrongCharset(String csvFileName, String csvCharsetName, String stmtCharsetName) + throws SQLException { + int updatesCnt = stmt.executeUpdate( + "copy from '" + csvFileName + "' into " + TBL_NAME + + " (_key, age, firstName, lastName)" + + " format csv charset '" + stmtCharsetName + "'"); + + assertEquals(2, updatesCnt); + + checkRecodedNationalCacheContents(TBL_NAME, csvCharsetName, stmtCharsetName); + } + + /** + * Recodes an input string as if it was encoded in one charset and was read using + * another charset using {@link CodingErrorAction#REPLACE} settings for + * unmappable and malformed characters. + */ + private static class WrongCharsetRecoder implements IgniteClosure { + /** Charset in which the string we are reading is actually encoded. */ + private final Charset actualCharset; + + /** Charset which we use to read the string. */ + private final Charset appliedCharset; + + /** + * Creates the recoder. + * + * @param actualCharset Charset in which the string we are reading is actually encoded. + * @param appliedCharset Charset which we use to read the string. + * @throws UnsupportedCharsetException if the charset name is wrong. + */ + WrongCharsetRecoder(String actualCharset, String appliedCharset) { + this.actualCharset = Charset.forName(actualCharset); + this.appliedCharset = Charset.forName(appliedCharset); + } + + /** + * Converts string as it was read using a wrong charset. + *

    + * First the method converts the string into {@link #actualCharset} and puts bytes into a buffer. + * Then it tries to read these bytes from the buffer using {@link #appliedCharset} and + * {@link CodingErrorAction#REPLACE} settings for unmappable and malformed characters + * (NB: these settings implicitly come from {@link Charset#decode(ByteBuffer)} implementation, while + * being explicitly set in {@link BulkLoadCsvParser#BulkLoadCsvParser(BulkLoadCsvFormat)}). + * + * @param input The input string (in Java encoding). + * @return The converted string. + */ + @Override public String apply(String input) { + ByteBuffer encodedBuf = actualCharset.encode(input); + + return appliedCharset.decode(encodedBuf).toString(); + } } } diff --git a/modules/clients/src/test/resources/bulkload2_utf.csv b/modules/clients/src/test/resources/bulkload2_utf8.csv similarity index 100% rename from modules/clients/src/test/resources/bulkload2_utf.csv rename to modules/clients/src/test/resources/bulkload2_utf8.csv diff --git a/modules/clients/src/test/resources/bulkload2_windows1251.csv b/modules/clients/src/test/resources/bulkload2_windows1251.csv new file mode 100644 index 0000000000000..b0dcde05b4a77 --- /dev/null +++ b/modules/clients/src/test/resources/bulkload2_windows1251.csv @@ -0,0 +1,2 @@ +123,12,"Èìÿ123 Îò÷åñòâî123",Ôàìèëèÿ123 +456,45,"Èìÿ456","Ôàìèëèÿ456" \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java index 07034f4579e41..fce046dd752a9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java @@ -174,7 +174,7 @@ public JdbcQueryTask(Ignite ignite, String cacheName, String schemaName, String if (fldQryCursor instanceof BulkLoadContextCursor) { fldQryCursor.close(); - + throw new SQLException("COPY command is currently supported only in thin JDBC driver."); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCacheWriter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCacheWriter.java index 90714c8308de4..0f7444ceceba0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCacheWriter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCacheWriter.java @@ -20,7 +20,9 @@ import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteInClosure; -/** A proxy, which stores given key+value pair to a cache. */ +/** + * A proxy, which stores given key+value pair to a cache. + */ public abstract class BulkLoadCacheWriter implements IgniteInClosure>, AutoCloseable { /** * Returns number of entry updates made by the writer. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCsvFormat.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCsvFormat.java index ec1dfd1a889b5..69cb341daacf3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCsvFormat.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCsvFormat.java @@ -22,7 +22,9 @@ import java.util.regex.Pattern; -/** A placeholder for bulk load CSV format parser options. */ +/** + * A placeholder for bulk load CSV format parser options. + */ public class BulkLoadCsvFormat extends BulkLoadFormat { /** Line separator pattern. */ @NotNull public static final Pattern DEFAULT_LINE_SEPARATOR = Pattern.compile("[\r\n]+"); @@ -57,6 +59,9 @@ public class BulkLoadCsvFormat extends BulkLoadFormat { /** Set of escape start characters. */ @Nullable private String escapeChars; + /** File charset. */ + @Nullable private String inputCharsetName; + /** * Returns the name of the format. * @@ -155,4 +160,22 @@ public void commentChars(@Nullable Pattern commentChars) { public void escapeChars(@Nullable String escapeChars) { this.escapeChars = escapeChars; } + + /** + * Returns the input file charset name, null if not specified. + * + * @return The input file charset name, null if not specified. + */ + @Nullable public String inputCharsetName() { + return inputCharsetName; + } + + /** + * Sets the input file charset name. The null here means "not specified". + * + * @param inputCharsetName The input file charset name. + */ + public void inputCharsetName(@Nullable String inputCharsetName) { + this.inputCharsetName = inputCharsetName; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCsvParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCsvParser.java index 0511596012477..98e994d1fcde4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCsvParser.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadCsvParser.java @@ -24,8 +24,12 @@ import org.apache.ignite.internal.processors.bulkload.pipeline.StrListAppenderBlock; import org.apache.ignite.internal.processors.bulkload.pipeline.LineSplitterBlock; +import java.nio.charset.Charset; +import java.nio.charset.IllegalCharsetNameException; +import java.nio.charset.UnsupportedCharsetException; import java.util.LinkedList; import java.util.List; +import org.apache.ignite.internal.processors.query.IgniteSQLException; /** CSV parser for COPY command. */ public class BulkLoadCsvParser extends BulkLoadParser { @@ -41,7 +45,20 @@ public class BulkLoadCsvParser extends BulkLoadParser { * @param format Format options (parsed from COPY command). */ public BulkLoadCsvParser(BulkLoadCsvFormat format) { - inputBlock = new CharsetDecoderBlock(BulkLoadFormat.DEFAULT_INPUT_CHARSET); + try { + Charset charset = format.inputCharsetName() == null ? BulkLoadFormat.DEFAULT_INPUT_CHARSET : + Charset.forName(format.inputCharsetName()); + + inputBlock = new CharsetDecoderBlock(charset); + } + catch (IllegalCharsetNameException e) { + throw new IgniteSQLException("Unknown charset name: '" + format.inputCharsetName() + "': " + + e.getMessage()); + } + catch (UnsupportedCharsetException e) { + throw new IgniteSQLException("Charset is not supported: '" + format.inputCharsetName() + "': " + + e.getMessage()); + } collectorBlock = new StrListAppenderBlock(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadFormat.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadFormat.java index cff93c5788552..bb4beb07eef52 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadFormat.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/BulkLoadFormat.java @@ -19,7 +19,9 @@ import java.nio.charset.Charset; -/** A superclass and a factory for bulk load format options. */ +/** + * A superclass and a factory for bulk load format options. + */ public abstract class BulkLoadFormat { /** The default input charset. */ public static final Charset DEFAULT_INPUT_CHARSET = Charset.forName("UTF-8"); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/CharsetDecoderBlock.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/CharsetDecoderBlock.java index 5b18def1a37cc..f9031b161745c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/CharsetDecoderBlock.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/CharsetDecoderBlock.java @@ -34,6 +34,9 @@ * the erroneous input, appending the coder's replacement value to the output buffer, and resuming the coding operation. */ public class CharsetDecoderBlock extends PipelineBlock { + /** Empty portion. */ + public static final char[] EMPTY_PORTION = new char[0]; + /** Charset decoder */ private final CharsetDecoder charsetDecoder; @@ -67,7 +70,8 @@ public CharsetDecoderBlock(Charset charset) { isEndOfInput = isLastAppend; if (leftover == null && data.length == 0) { - nextBlock.accept(new char[0], isLastAppend); + nextBlock.accept(EMPTY_PORTION, isLastAppend); + return; } @@ -78,8 +82,7 @@ public CharsetDecoderBlock(Charset charset) { else { dataBuf = ByteBuffer.allocate(leftover.length + data.length); - dataBuf.put(leftover) - .put(data); + dataBuf.put(leftover).put(data); dataBuf.flip(); @@ -101,8 +104,9 @@ public CharsetDecoderBlock(Charset charset) { leftover = Arrays.copyOfRange(dataBuf.array(), dataBuf.arrayOffset() + dataBuf.position(), dataBuf.limit()); + // See {@link CharsetDecoder} class javadoc for the protocol. if (isEndOfInput) - charsetDecoder.flush(outBuf); // See {@link CharsetDecoder} class javadoc for the protocol. + charsetDecoder.flush(outBuf); if (outBuf.position() > 0) nextBlock.accept(Arrays.copyOfRange(outBuf.array(), outBuf.arrayOffset(), outBuf.position()), @@ -111,7 +115,8 @@ public CharsetDecoderBlock(Charset charset) { break; } - if (res.isOverflow()) { // Not enough space in the output buffer, flush it and retry. + // Not enough space in the output buffer, flush it and retry. + if (res.isOverflow()) { assert outBuf.position() > 0; nextBlock.accept(Arrays.copyOfRange(outBuf.array(), outBuf.arrayOffset(), outBuf.position()), diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/PipelineBlock.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/PipelineBlock.java index 914b4b4d4aaa4..f0a7339d6425d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/PipelineBlock.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/bulkload/pipeline/PipelineBlock.java @@ -27,7 +27,7 @@ */ public abstract class PipelineBlock { /** The next block in pipeline or null if this block is a terminator. */ - @Nullable protected PipelineBlock nextBlock; + @Nullable PipelineBlock nextBlock; /** * Creates a pipeline block. @@ -35,7 +35,7 @@ public abstract class PipelineBlock { *

    (There is no nextBlock argument in the constructor: setting the next block using * {@link #append(PipelineBlock)} method is more convenient. */ - protected PipelineBlock() { + PipelineBlock() { nextBlock = null; } @@ -61,6 +61,7 @@ public PipelineBlock append(PipelineBlock next) { * * @param inputPortion Portion of input. * @param isLastPortion Is this the last portion. + * @throws IgniteCheckedException On error. */ public abstract void accept(I inputPortion, boolean isLastPortion) throws IgniteCheckedException; } 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 e25096daeff20..0fd08f48c4340 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 @@ -62,6 +62,9 @@ public class SqlKeyword { /** Keyword: CHARACTER. */ public static final String CHARACTER = "CHARACTER"; + /** Keyword: CHARSET. */ + public static final String CHARSET = "CHARSET"; + /** Keyword: COPY. */ public static final String COPY = "COPY"; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParseException.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParseException.java index 96d385da51fff..c31824c48462d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParseException.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParseException.java @@ -52,6 +52,19 @@ public SqlParseException(String sql, int pos, int code, String msg) { this.code = code; } + /** + * Copy constructor. + * + * @param e Copied exception. + */ + protected SqlParseException(SqlParseException e) { + super(e.getMessage()); + + sql = e.sql; + pos = e.pos; + code = e. code; + } + /** * Prepare message. * 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 cbde93663bfc8..d46863a8e34dc 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 @@ -113,9 +113,14 @@ private SqlCommand nextCommand0() { break; case COPY: - cmd = processCopy(); - - break; + try { + cmd = processCopy(); + + break; + } + catch (SqlParseException e) { + throw new SqlStrictParseException(e); + } case SET: cmd = processSet(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlStrictParseException.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlStrictParseException.java new file mode 100644 index 0000000000000..7c831971b07ba --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlStrictParseException.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.sql; + +/** + * Parse exception guarantees parse error without. Such error deliver to user + * statement isn't passed to H2 parser. + */ +public class SqlStrictParseException extends SqlParseException { + /** + * Constructor. + * @param e SQL parse exception. + */ + public SqlStrictParseException(SqlParseException e) { + super(e); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBulkLoadCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBulkLoadCommand.java index 9b0db969caf67..a1e4aecc0fdbb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBulkLoadCommand.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlBulkLoadCommand.java @@ -33,6 +33,7 @@ import static org.apache.ignite.internal.sql.SqlParserUtils.parseIdentifier; import static org.apache.ignite.internal.sql.SqlParserUtils.parseInt; import static org.apache.ignite.internal.sql.SqlParserUtils.parseQualifiedIdentifier; +import static org.apache.ignite.internal.sql.SqlParserUtils.parseString; import static org.apache.ignite.internal.sql.SqlParserUtils.skipCommaOrRightParenthesis; import static org.apache.ignite.internal.sql.SqlParserUtils.skipIfMatches; import static org.apache.ignite.internal.sql.SqlParserUtils.skipIfMatchesKeyword; @@ -84,8 +85,8 @@ public class SqlBulkLoadCommand implements SqlCommand { * @param lex The lexer. */ private void parseFileName(SqlLexer lex) { - if (lex.lookAhead().tokenType() != SqlLexerTokenType.QUOTED) - throw errorUnexpectedToken(lex.lookAhead(), "[quoted file name]"); + if (lex.lookAhead().tokenType() != SqlLexerTokenType.STRING) + throw errorUnexpectedToken(lex.lookAhead(), "[file name: string]"); lex.shift(); @@ -150,6 +151,8 @@ private void parseFormat(SqlLexer lex) { fmt.commentChars(BulkLoadCsvFormat.DEFAULT_COMMENT_CHARS); fmt.escapeChars(BulkLoadCsvFormat.DEFAULT_ESCAPE_CHARS); + parseCsvOptions(lex, fmt); + inputFormat = fmt; break; @@ -160,6 +163,31 @@ private void parseFormat(SqlLexer lex) { } } + /** + * Parses CSV format options. + * + * @param lex The lexer. + * @param format CSV format object to configure. + */ + private void parseCsvOptions(SqlLexer lex, BulkLoadCsvFormat format) { + while (lex.lookAhead().tokenType() == SqlLexerTokenType.DEFAULT) { + switch (lex.lookAhead().token()) { + case SqlKeyword.CHARSET: { + lex.shift(); + + String charsetName = parseString(lex); + + format.inputCharsetName(charsetName); + + break; + } + + default: + return; + } + } + } + /** * Parses the optional parameters. * diff --git a/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserBulkLoadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserBulkLoadSelfTest.java index 96fe57ffbcb4d..b6716ffbf6a8f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserBulkLoadSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserBulkLoadSelfTest.java @@ -24,63 +24,99 @@ public class SqlParserBulkLoadSelfTest extends SqlParserAbstractSelfTest { /** Tests for COPY command. */ public void testCopy() { assertParseError(null, - "copy grom \"any.file\" into Person (_key, age, firstName, lastName) format csv", + "copy grom 'any.file' into Person (_key, age, firstName, lastName) format csv", "Unexpected token: \"GROM\" (expected: \"FROM\")"); assertParseError(null, "copy from into Person (_key, age, firstName, lastName) format csv", - "Unexpected token: \"INTO\" (expected: \"[quoted file name]\""); + "Unexpected token: \"INTO\" (expected: \"[file name: string]\""); assertParseError(null, "copy from unquoted into Person (_key, age, firstName, lastName) format csv", - "Unexpected token: \"UNQUOTED\" (expected: \"[quoted file name]\""); + "Unexpected token: \"UNQUOTED\" (expected: \"[file name: string]\""); assertParseError(null, "copy from unquoted.file into Person (_key, age, firstName, lastName) format csv", - "Unexpected token: \"UNQUOTED\" (expected: \"[quoted file name]\""); + "Unexpected token: \"UNQUOTED\" (expected: \"[file name: string]\""); new SqlParser(null, - "copy from \"\" into Person (_key, age, firstName, lastName) format csv") + "copy from '' into Person (_key, age, firstName, lastName) format csv") .nextCommand(); new SqlParser(null, - "copy from \"d:/copy/from/into/format.csv\" into Person (_key, age, firstName, lastName) format csv") + "copy from 'd:/copy/from/into/format.csv' into Person (_key, age, firstName, lastName) format csv") .nextCommand(); new SqlParser(null, - "copy from \"/into\" into Person (_key, age, firstName, lastName) format csv") + "copy from '/into' into Person (_key, age, firstName, lastName) format csv") .nextCommand(); new SqlParser(null, - "copy from \"into\" into Person (_key, age, firstName, lastName) format csv") + "copy from 'into' into Person (_key, age, firstName, lastName) format csv") .nextCommand(); assertParseError(null, - "copy from \"any.file\" to Person (_key, age, firstName, lastName) format csv", + "copy from 'any.file' to Person (_key, age, firstName, lastName) format csv", "Unexpected token: \"TO\" (expected: \"INTO\")"); // Column list assertParseError(null, - "copy from \"any.file\" into Person () format csv", + "copy from '" + + "any.file' into Person () format csv", "Unexpected token: \")\" (expected: \"[identifier]\")"); assertParseError(null, - "copy from \"any.file\" into Person (,) format csv", + "copy from 'any.file' into Person (,) format csv", "Unexpected token: \",\" (expected: \"[identifier]\")"); assertParseError(null, - "copy from \"any.file\" into Person format csv", + "copy from 'any.file' into Person format csv", "Unexpected token: \"FORMAT\" (expected: \"(\")"); // FORMAT assertParseError(null, - "copy from \"any.file\" into Person (_key, age, firstName, lastName)", + "copy from 'any.file' into Person (_key, age, firstName, lastName)", "Unexpected end of command (expected: \"FORMAT\")"); assertParseError(null, - "copy from \"any.file\" into Person (_key, age, firstName, lastName) format lsd", + "copy from 'any.file' into Person (_key, age, firstName, lastName) format lsd", "Unknown format name: LSD"); + + // FORMAT CSV CHARSET + + new SqlParser(null, + "copy from 'any.file' into Person (_key, age, firstName, lastName) format csv") + .nextCommand(); + + new SqlParser(null, + "copy from 'any.file' into Person (_key, age, firstName, lastName) format csv charset" + + " 'utf-8'") + .nextCommand(); + + new SqlParser(null, + "copy from 'any.file' into Person (_key, age, firstName, lastName) format csv charset" + + " 'UTF-8'") + .nextCommand(); + + new SqlParser(null, + "copy from 'any.file' into Person (_key, age, firstName, lastName) format csv charset" + + " 'UtF-8'") + .nextCommand(); + + new SqlParser(null, + "copy from 'any.file' into Person (_key, age, firstName, lastName) format csv charset" + + " 'windows-1251'") + .nextCommand(); + + new SqlParser(null, + "copy from 'any.file' into Person (_key, age, firstName, lastName) format csv charset" + + " 'ISO-2022-JP'") + .nextCommand(); + + assertParseError(null, + "copy from 'any.file' into Person (_key, age, firstName, lastName) format csv charset ", + "Unexpected end of command (expected: \"[string]\")"); } } 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 b113a8b065b95..7b1a093a1ab95 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 @@ -122,10 +122,10 @@ import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.sql.SqlParseException; import org.apache.ignite.internal.sql.SqlParser; +import org.apache.ignite.internal.sql.SqlStrictParseException; import org.apache.ignite.internal.sql.command.SqlAlterTableCommand; import org.apache.ignite.internal.sql.command.SqlBulkLoadCommand; 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.SqlCreateIndexCommand; import org.apache.ignite.internal.sql.command.SqlCreateUserCommand; @@ -1525,6 +1525,9 @@ private List>> tryQueryDistributedSqlFieldsNative(Stri || cmd instanceof SqlDropUserCommand)) return null; } + catch (SqlStrictParseException e) { + throw new IgniteSQLException(e.getMessage(), IgniteQueryErrorCode.PARSING, e); + } catch (Exception e) { // Cannot parse, return. if (log.isDebugEnabled()) 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 8021a3cac8d06..87efa83bc6a3d 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 @@ -111,7 +111,7 @@ public String copyFrom(String csvFilePath, @Nullable Long packetSize) { if (packetSize != null) pSizeExpr = " packet_size " + packetSize; - return "COPY FROM \"" + csvFilePath + "\" " + + return "COPY FROM '" + csvFilePath + "' " + "INTO test_upload " + attributes() + " " + "FORMAT CSV" + pSizeExpr + ";"; } From bf1b14a3179d03d14e4379ba917c9025b33bedee Mon Sep 17 00:00:00 2001 From: dpavlov Date: Thu, 10 May 2018 19:48:24 +0300 Subject: [PATCH 0157/1463] IGNITE-7535: SerialVersionUID added to exception. (cherry picked from commit 619d6c5) --- .../apache/ignite/internal/sql/SqlStrictParseException.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlStrictParseException.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlStrictParseException.java index 7c831971b07ba..119b8545ae7c0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlStrictParseException.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlStrictParseException.java @@ -22,6 +22,9 @@ * statement isn't passed to H2 parser. */ public class SqlStrictParseException extends SqlParseException { + /** */ + private static final long serialVersionUID = 0L; + /** * Constructor. * @param e SQL parse exception. From 906b3d932edaa3ee44aab93641356de342ad32a2 Mon Sep 17 00:00:00 2001 From: Ilya Borisov Date: Mon, 23 Apr 2018 15:12:18 +0700 Subject: [PATCH 0158/1463] IGNITE-8334 Web Console: Added ability to show/hide password field value. (cherry picked from commit e0d1752) --- modules/web-console/frontend/app/app.js | 4 +- .../components/password-visibility/index.js | 26 +++ .../password-visibility/index.spec.js | 65 +++++++ .../password-visibility/root.directive.js | 49 +++++ .../components/password-visibility/style.scss | 50 +++++ .../toggle-button.component.js | 49 +++++ .../app/directives/match.directive.js | 32 +++- .../app/directives/match.directive.spec.js | 81 +++++++++ .../generator/Docker.service.spec.js | 2 +- .../app/primitives/form-field/index.scss | 26 +++ .../app/primitives/form-field/password.pug | 12 +- .../web-console/frontend/package-lock.json | 171 +++++++++--------- .../public/images/icons/eyeClosed.svg | 6 + .../public/images/icons/eyeOpened.svg | 7 + .../frontend/public/images/icons/index.js | 3 +- 15 files changed, 492 insertions(+), 91 deletions(-) create mode 100644 modules/web-console/frontend/app/components/password-visibility/index.js create mode 100644 modules/web-console/frontend/app/components/password-visibility/index.spec.js create mode 100644 modules/web-console/frontend/app/components/password-visibility/root.directive.js create mode 100644 modules/web-console/frontend/app/components/password-visibility/style.scss create mode 100644 modules/web-console/frontend/app/components/password-visibility/toggle-button.component.js create mode 100644 modules/web-console/frontend/app/directives/match.directive.spec.js create mode 100644 modules/web-console/frontend/public/images/icons/eyeClosed.svg create mode 100644 modules/web-console/frontend/public/images/icons/eyeOpened.svg diff --git a/modules/web-console/frontend/app/app.js b/modules/web-console/frontend/app/app.js index 692acc54f3d64..c6720bc7c54c6 100644 --- a/modules/web-console/frontend/app/app.js +++ b/modules/web-console/frontend/app/app.js @@ -138,6 +138,7 @@ import clusterSelector from './components/cluster-selector'; import connectedClusters from './components/connected-clusters'; import pageSignIn from './components/page-signin'; import pageLanding from './components/page-landing'; +import passwordVisibility from './components/password-visibility'; import pageProfile from './components/page-profile'; import pagePasswordChanged from './components/page-password-changed'; @@ -239,6 +240,7 @@ angular.module('ignite-console', [ uiAceJava.name, uiAceSpring.name, breadcrumbs.name, + passwordVisibility.name, // Ignite modules. IgniteModules.name ]) @@ -250,7 +252,7 @@ angular.module('ignite-console', [ .directive(...igniteCopyToClipboard) .directive(...igniteHideOnStateChange) .directive(...igniteInformation) -.directive(...igniteMatch) +.directive('igniteMatch', igniteMatch) .directive(...igniteOnClickFocus) .directive(...igniteOnEnter) .directive(...igniteOnEnterFocusMove) diff --git a/modules/web-console/frontend/app/components/password-visibility/index.js b/modules/web-console/frontend/app/components/password-visibility/index.js new file mode 100644 index 0000000000000..d735869d03354 --- /dev/null +++ b/modules/web-console/frontend/app/components/password-visibility/index.js @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import angular from 'angular'; +import './style.scss'; +import {directive as visibilityRoot} from './root.directive'; +import {component as toggleButton} from './toggle-button.component'; + +export default angular + .module('ignite-console.passwordVisibility', []) + .directive('passwordVisibilityRoot', visibilityRoot) + .component('passwordVisibilityToggleButton', toggleButton); diff --git a/modules/web-console/frontend/app/components/password-visibility/index.spec.js b/modules/web-console/frontend/app/components/password-visibility/index.spec.js new file mode 100644 index 0000000000000..236e497d55023 --- /dev/null +++ b/modules/web-console/frontend/app/components/password-visibility/index.spec.js @@ -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. + */ + +import 'mocha'; +import {assert} from 'chai'; +import angular from 'angular'; +import module from './index'; + +const PASSWORD_VISIBLE_CLASS = 'password-visibility__password-visible'; + +suite('password-visibility', () => { + /** @type {ng.IScope} */ + let $scope; + /** @type {ng.ICompileService} */ + let $compile; + + angular.module('test', [module.name]); + + setup(() => { + angular.module('test', [module.name]); + angular.mock.module('test'); + angular.mock.inject((_$rootScope_, _$compile_) => { + $compile = _$compile_; + $scope = _$rootScope_.$new(); + }); + }); + + test('Visibility toggle', () => { + const el = angular.element(` +

    + +
    + `); + $compile(el)($scope); + const toggleButton = el.find('password-visibility-toggle-button button'); + $scope.$digest(); + + assert.isFalse(el.hasClass(PASSWORD_VISIBLE_CLASS), 'Password is hidden by default'); + + toggleButton.click(); + $scope.$digest(); + + assert.isTrue(el.hasClass(PASSWORD_VISIBLE_CLASS), 'Password is visible after click on toggle button'); + assert.equal(true, $scope.visible, 'Event emits current visibility value'); + + toggleButton.click(); + $scope.$digest(); + + assert.isFalse(el.hasClass(PASSWORD_VISIBLE_CLASS), 'Password is hidden again after two clicks on button'); + }); +}); diff --git a/modules/web-console/frontend/app/components/password-visibility/root.directive.js b/modules/web-console/frontend/app/components/password-visibility/root.directive.js new file mode 100644 index 0000000000000..a041398a187e7 --- /dev/null +++ b/modules/web-console/frontend/app/components/password-visibility/root.directive.js @@ -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. + */ + +const PASSWORD_VISIBLE_CLASS = 'password-visibility__password-visible'; + +export class PasswordVisibilityRoot { + /** @type {ng.ICompiledExpression} */ + onPasswordVisibilityToggle; + + isVisible = false; + static $inject = ['$element']; + + /** + * @param {JQLite} $element + */ + constructor($element) { + this.$element = $element; + } + toggleVisibility() { + this.isVisible = !this.isVisible; + this.$element.toggleClass(PASSWORD_VISIBLE_CLASS, this.isVisible); + if (this.onPasswordVisibilityToggle) this.onPasswordVisibilityToggle({$event: this.isVisible}); + } +} + +export function directive() { + return { + restrict: 'A', + scope: false, + controller: PasswordVisibilityRoot, + bindToController: { + onPasswordVisibilityToggle: '&?' + } + }; +} diff --git a/modules/web-console/frontend/app/components/password-visibility/style.scss b/modules/web-console/frontend/app/components/password-visibility/style.scss new file mode 100644 index 0000000000000..6f5130b285b83 --- /dev/null +++ b/modules/web-console/frontend/app/components/password-visibility/style.scss @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +[password-visibility-root] { + &:not(.password-visibility__password-visible) { + .password-visibility__icon-visible, + .password-visibility__password-visible { + display: none; + } + } + + &.password-visibility__password-visible { + .password-visibility__icon-hidden, + .password-visibility__password-hidden { + display: none; + } + } +} + +password-visibility-toggle-button { + display: inline-block; + width: 36px; + + button { + display: inline-flex; + align-items: center; + justify-content: center; + width: 100%; + height: 100%; + background: none; + border: none; + outline: none; + padding: 0 !important; + margin: 0 !important; + } +} diff --git a/modules/web-console/frontend/app/components/password-visibility/toggle-button.component.js b/modules/web-console/frontend/app/components/password-visibility/toggle-button.component.js new file mode 100644 index 0000000000000..2e16201f018d5 --- /dev/null +++ b/modules/web-console/frontend/app/components/password-visibility/toggle-button.component.js @@ -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. + */ + +import {PasswordVisibilityRoot} from './root.directive'; + +class Controller { + /** @type {PasswordVisibilityRoot} */ + visibilityRoot; + + toggleVisibility() { + this.visibilityRoot.toggleVisibility(); + } + get isVisible() { + return this.visibilityRoot.isVisible; + } +} + +export const component = { + template: ` + + `, + require: { + visibilityRoot: '^passwordVisibilityRoot' + }, + controller: Controller +}; diff --git a/modules/web-console/frontend/app/directives/match.directive.js b/modules/web-console/frontend/app/directives/match.directive.js index 3a45f6da8daae..940ca086de4db 100644 --- a/modules/web-console/frontend/app/directives/match.directive.js +++ b/modules/web-console/frontend/app/directives/match.directive.js @@ -16,12 +16,32 @@ */ // Directive to enable validation to match specified value. -export default ['igniteMatch', ['$parse', ($parse) => { +export default function() { return { - require: 'ngModel', - link(scope, elem, attrs, ctrl) { - scope.$watch(() => $parse(attrs.igniteMatch)(scope) === ctrl.$modelValue, - (currentValue) => ctrl.$setValidity('mismatch', currentValue)); + require: { + ngModel: 'ngModel' + }, + scope: false, + bindToController: { + igniteMatch: '<' + }, + controller: class { + /** @type {ng.INgModelController} */ + ngModel; + /** @type {string} */ + igniteMatch; + + $postLink() { + this.ngModel.$overrideModelOptions({allowInvalid: true}); + this.ngModel.$validators.mismatch = (value) => value === this.igniteMatch; + } + + /** + * @param {{igniteMatch: ng.IChangesObject}} changes + */ + $onChanges(changes) { + if ('igniteMatch' in changes) this.ngModel.$validate(); + } } }; -}]]; +} diff --git a/modules/web-console/frontend/app/directives/match.directive.spec.js b/modules/web-console/frontend/app/directives/match.directive.spec.js new file mode 100644 index 0000000000000..2a00b03d0bb89 --- /dev/null +++ b/modules/web-console/frontend/app/directives/match.directive.spec.js @@ -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. + */ + +import 'mocha'; +import {assert} from 'chai'; +import angular from 'angular'; +import directive from './match.directive'; + +/** + * @param {HTMLInputElement} el + * @returns {ng.INgModelController} + */ +const ngModel = (el) => angular.element(el).data().$ngModelController; + +suite('ignite-match', () => { + /** @type {ng.IScope} */ + let $scope; + /** @type {ng.ICompileService} */ + let $compile; + + setup(() => { + angular.module('test', []).directive('igniteMatch', directive); + angular.mock.module('test'); + angular.mock.inject((_$rootScope_, _$compile_) => { + $compile = _$compile_; + $scope = _$rootScope_.$new(); + }); + }); + + test('Matching', () => { + const el = angular.element(` + + + `); + + const setValue = (el, value) => { + ngModel(el).$setViewValue(value, 'input'); + $scope.$digest(); + }; + + $scope.data = {}; + $compile(el)($scope); + $scope.$digest(); + + const [master, , slave] = el; + + setValue(slave, '123'); + $scope.$digest(); + + assert.isTrue( + slave.classList.contains('ng-invalid-mismatch'), + `Invalidates if slave input changes value and it doesn't match master value` + ); + assert.equal( + $scope.data.slave, + '123', + 'Allows invalid value into model' + ); + + setValue(master, '123'); + + assert.isFalse( + slave.classList.contains('ng-invalid-mismatch'), + `Runs validation on master value change` + ); + }); +}); diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.spec.js b/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.spec.js index becc35988dc30..0ecada79d3b5c 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.spec.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.spec.js @@ -19,7 +19,7 @@ import DockerGenerator from './Docker.service'; import {assert} from 'chai'; import {outdent} from 'outdent/lib'; -suite.only('Dockerfile generator', () => { +suite('Dockerfile generator', () => { const generator = new DockerGenerator(); test('Target 2.0', () => { diff --git a/modules/web-console/frontend/app/primitives/form-field/index.scss b/modules/web-console/frontend/app/primitives/form-field/index.scss index 1035adecfc6c9..800daa86669c3 100644 --- a/modules/web-console/frontend/app/primitives/form-field/index.scss +++ b/modules/web-console/frontend/app/primitives/form-field/index.scss @@ -242,6 +242,8 @@ } // Added right offset to appearance of input for invalid password + & > input[type='email'].ng-invalid.ng-touched, + & > input[type='text'].ng-invalid.ng-touched, & > input[type='password'].ng-invalid.ng-touched { padding-right: 36px; } @@ -328,3 +330,27 @@ } } } + +.form-field__password { + // Validation error notification will overlap with visibility button if it's not moved more to the left + input[type='password'].ng-invalid.ng-touched, + input[type='password'].ng-invalid.ng-touched + input { + padding-right: 62px; + } + + // Extra space for visibility button + input { + padding-right: 36px; + } + + // Distance between error notification and visibility button + .form-field__errors { + right: 26px; + } + + password-visibility-toggle-button { + position: absolute; + right: 0; + height: 36px; + } +} diff --git a/modules/web-console/frontend/app/primitives/form-field/password.pug b/modules/web-console/frontend/app/primitives/form-field/password.pug index ba38cce8e833d..40e1aa90b87d7 100644 --- a/modules/web-console/frontend/app/primitives/form-field/password.pug +++ b/modules/web-console/frontend/app/primitives/form-field/password.pug @@ -17,13 +17,23 @@ mixin form-field__password({ label, model, name, disabled, required, placeholder, tip }) -var errLbl = label.substring(0, label.length - 1) - .form-field + .form-field.form-field__password( + password-visibility-root + on-password-visibility-toggle=`${form}[${name}].$setTouched()` + ) +form-field__label({ label, name, required }) +form-field__tooltip({ title: tip, options: tipOpts }) .form-field__control - attributes.type='password' + - attributes.class = 'password-visibility__password-hidden' +form-field__input({ name, model, disabled, required, placeholder })(attributes=attributes) + - attributes.class = 'password-visibility__password-visible' + - attributes.type='text' + - attributes.autocomplete = 'off' + +form-field__input({ name: name + `+"Text"`, model, disabled, required, placeholder })(attributes=attributes) + + password-visibility-toggle-button .form-field__errors( ng-messages=`(${form}[${name}].$dirty || ${form}[${name}].$touched || ${form}[${name}].$submitted) && ${form}[${name}].$invalid ? ${form}[${name}].$error : {}` diff --git a/modules/web-console/frontend/package-lock.json b/modules/web-console/frontend/package-lock.json index 1fec909e3860e..c966cb7b9a770 100644 --- a/modules/web-console/frontend/package-lock.json +++ b/modules/web-console/frontend/package-lock.json @@ -71,9 +71,9 @@ } }, "@types/chai": { - "version": "4.1.2", - "resolved": "https://registry.npmjs.org/@types/chai/-/chai-4.1.2.tgz", - "integrity": "sha512-D8uQwKYUw2KESkorZ27ykzXgvkDJYXVEihGklgfp5I4HUP8D6IxtcdLTMB1emjQiWzV7WZ5ihm1cxIzVwjoleQ==", + "version": "4.1.3", + "resolved": "https://registry.npmjs.org/@types/chai/-/chai-4.1.3.tgz", + "integrity": "sha512-f5dXGzOJycyzSMdaXVhiBhauL4dYydXwVpavfQ1mVCaGjR56a9QfklXObUxlIY9bGTmCPHEEZ04I16BZ/8w5ww==", "dev": true }, "@types/jquery": { @@ -95,9 +95,9 @@ "dev": true }, "@types/node": { - "version": "9.6.5", - "resolved": "https://registry.npmjs.org/@types/node/-/node-9.6.5.tgz", - "integrity": "sha512-NOLEgsT6UiDTjnWG5Hd2Mg25LRyz/oe8ql3wbjzgSFeRzRROhPmtlsvIrei4B46UjERF0td9SZ1ZXPLOdcrBHg==", + "version": "9.6.6", + "resolved": "https://registry.npmjs.org/@types/node/-/node-9.6.6.tgz", + "integrity": "sha512-SJe0g5cZeGNDP5sD8mIX3scb+eq8LQQZ60FXiKZHipYSeEFZ5EKml+NNMiO76F74TY4PoMWlNxF/YRY40FOvZQ==", "dev": true }, "@types/sinon": { @@ -145,7 +145,7 @@ "integrity": "sha512-NoGVTCumOsyFfuy3934f3ktiJi+wcXHJFxT47tby3iCpuo6M/WjFA9VqT5bYO+FE46i3R0N00RpJX75HxHKDaQ==", "dev": true, "requires": { - "@types/node": "9.6.5", + "@types/node": "9.6.6", "@types/tapable": "1.0.2", "@types/uglify-js": "3.0.2", "source-map": "0.6.1" @@ -465,9 +465,9 @@ "integrity": "sha1-ClsQZgGLQOcAuMbuNLDJf8MhlSs=" }, "angular-ui-grid": { - "version": "4.4.6", - "resolved": "https://registry.npmjs.org/angular-ui-grid/-/angular-ui-grid-4.4.6.tgz", - "integrity": "sha512-0d14HDY4XeqFHI508thxeufiR0AlFoZQ8ihk0x8TRCQc+b9CCk1/F63W2zihirxF0cdOAqBCY2pVSM7vfZvXBQ==", + "version": "4.4.7", + "resolved": "https://registry.npmjs.org/angular-ui-grid/-/angular-ui-grid-4.4.7.tgz", + "integrity": "sha512-z592OgLcVkyfGVMU2M0ymTPjBh/F+0/fAy/Yzj19Cs7wmuYcb13+iGFPskas27uDdxKv4mkJqmGeklyh5NxJEw==", "requires": { "angular": "1.6.6" } @@ -3250,7 +3250,7 @@ "cipher-base": "1.0.4", "inherits": "2.0.3", "md5.js": "1.3.4", - "ripemd160": "2.0.1", + "ripemd160": "2.0.2", "sha.js": "2.4.11" } }, @@ -3262,7 +3262,7 @@ "cipher-base": "1.0.4", "create-hash": "1.2.0", "inherits": "2.0.3", - "ripemd160": "2.0.1", + "ripemd160": "2.0.2", "safe-buffer": "5.1.1", "sha.js": "2.4.11" } @@ -3297,7 +3297,7 @@ "create-hmac": "1.1.7", "diffie-hellman": "5.0.3", "inherits": "2.0.3", - "pbkdf2": "3.0.14", + "pbkdf2": "3.0.16", "public-encrypt": "4.0.2", "randombytes": "2.0.6", "randomfill": "1.0.4" @@ -3972,9 +3972,9 @@ "integrity": "sha1-WQxhFWsK4vTwJVcyoViyZrxWsh0=" }, "ejs": { - "version": "2.5.8", - "resolved": "https://registry.npmjs.org/ejs/-/ejs-2.5.8.tgz", - "integrity": "sha512-QIDZL54fyV8MDcAsO91BMH1ft2qGGaHIJsJIA/+t+7uvXol1dm413fPcUgUb4k8F/9457rx4/KFE4XfDifrQxQ==" + "version": "2.5.9", + "resolved": "https://registry.npmjs.org/ejs/-/ejs-2.5.9.tgz", + "integrity": "sha512-GJCAeDBKfREgkBtgrYSf9hQy9kTb3helv0zGdzqhM7iAkW8FA/ZF97VQDbwFiwIT8MQLLOe5VlPZOEvZAqtUAQ==" }, "electron-to-chromium": { "version": "1.3.42", @@ -4402,9 +4402,9 @@ "integrity": "sha1-Qa4u62XvpiJorr/qg6x9eSmbCIc=" }, "eventemitter3": { - "version": "1.2.0", - "resolved": "https://registry.npmjs.org/eventemitter3/-/eventemitter3-1.2.0.tgz", - "integrity": "sha1-HIaZHYFq0eUEdQ5zh0Ik7PO+xQg=" + "version": "3.0.1", + "resolved": "https://registry.npmjs.org/eventemitter3/-/eventemitter3-3.0.1.tgz", + "integrity": "sha512-QOCPu979MMWX9XNlfRZoin+Wm+bK1SP7vv3NGUniYwuSJK/+cPA10blMaeRgzg31RvoSFk6FsCDVa4vNryBTGA==" }, "events": { "version": "1.1.1", @@ -4622,9 +4622,9 @@ "integrity": "sha1-wFNHeBfIa1HaqFPIHgWbcz0CNhQ=" }, "fast-glob": { - "version": "2.2.0", - "resolved": "https://registry.npmjs.org/fast-glob/-/fast-glob-2.2.0.tgz", - "integrity": "sha512-4F75PTznkNtSKs2pbhtBwRkw8sRwa7LfXx5XaQJOe4IQ6yTjceLDTwM5gj1s80R2t/5WeDC1gVfm3jLE+l39Tw==", + "version": "2.2.1", + "resolved": "https://registry.npmjs.org/fast-glob/-/fast-glob-2.2.1.tgz", + "integrity": "sha512-wSyW1TBK3ia5V+te0rGPXudeMHoUQW6O5Y9oATiaGhpENmEifPDlOdhpsnlj5HoG6ttIvGiY1DdCmI9X2xGMhg==", "dev": true, "requires": { "@mrmlnc/readdir-enhanced": "2.2.1", @@ -5118,6 +5118,24 @@ "readable-stream": "2.3.6" } }, + "follow-redirects": { + "version": "1.4.1", + "resolved": "https://registry.npmjs.org/follow-redirects/-/follow-redirects-1.4.1.tgz", + "integrity": "sha512-uxYePVPogtya1ktGnAAXOacnbIuRMB4dkvqeNz2qTtTQsuzSfbDolV+wMMKxAmCx0bLgAKLbBOkjItMbbkR1vg==", + "requires": { + "debug": "3.1.0" + }, + "dependencies": { + "debug": { + "version": "3.1.0", + "resolved": "https://registry.npmjs.org/debug/-/debug-3.1.0.tgz", + "integrity": "sha512-OX8XqP7/1a9cqkxYw2yXss15f26NKWBpDXQd0/uK/KPqdQhxbPa994hnzjcE2VqQpDslf55723cKPUOGSmMY3g==", + "requires": { + "ms": "2.0.0" + } + } + } + }, "font-awesome": { "version": "4.7.0", "resolved": "https://registry.npmjs.org/font-awesome/-/font-awesome-4.7.0.tgz", @@ -5491,7 +5509,7 @@ "requires": { "array-union": "1.0.2", "dir-glob": "2.0.0", - "fast-glob": "2.2.0", + "fast-glob": "2.2.1", "glob": "7.1.2", "ignore": "3.3.7", "pify": "3.0.0", @@ -5842,7 +5860,7 @@ "he": "1.1.1", "param-case": "2.1.1", "relateurl": "0.2.7", - "uglify-js": "3.3.21" + "uglify-js": "3.3.22" }, "dependencies": { "commander": { @@ -5856,9 +5874,9 @@ "integrity": "sha512-UjgapumWlbMhkBgzT7Ykc5YXUT46F0iKu8SGXq0bcwP5dz/h0Plj6enJqjz1Zbq2l5WaqYnrVbwWOWMyF3F47g==" }, "uglify-js": { - "version": "3.3.21", - "resolved": "https://registry.npmjs.org/uglify-js/-/uglify-js-3.3.21.tgz", - "integrity": "sha512-uy82472lH8tshK3jS3c5IFb5MmNKd/5qyBd0ih8sM42L3jWvxnE339U9gZU1zufnLVs98Stib9twq8dLm2XYCA==", + "version": "3.3.22", + "resolved": "https://registry.npmjs.org/uglify-js/-/uglify-js-3.3.22.tgz", + "integrity": "sha512-tqw96rL6/BG+7LM5VItdhDjTQmL5zG/I0b2RqWytlgeHe2eydZHuBHdA9vuGpCDhH/ZskNGcqDhivoR2xt8RIw==", "requires": { "commander": "2.15.1", "source-map": "0.6.1" @@ -6032,11 +6050,12 @@ "integrity": "sha512-QCR5O2AjjMW8Mo4HyI1ctFcv+O99j/0g367V3YoVnrNw5hkDvAWZD0lWGcc+F4yN3V55USPCVix4efb75HxFfA==" }, "http-proxy": { - "version": "1.16.2", - "resolved": "https://registry.npmjs.org/http-proxy/-/http-proxy-1.16.2.tgz", - "integrity": "sha1-Bt/ykpUr9k2+hHH6nfcwZtTzd0I=", + "version": "1.17.0", + "resolved": "https://registry.npmjs.org/http-proxy/-/http-proxy-1.17.0.tgz", + "integrity": "sha512-Taqn+3nNvYRfJ3bGvKfBSRwy1v6eePlm3oc/aWVxZp57DQr5Eq3xhKJi7Z4hZpS8PC3H4qI+Yly5EmFacGuA/g==", "requires": { - "eventemitter3": "1.2.0", + "eventemitter3": "3.0.1", + "follow-redirects": "1.4.1", "requires-port": "1.0.0" } }, @@ -6045,7 +6064,7 @@ "resolved": "https://registry.npmjs.org/http-proxy-middleware/-/http-proxy-middleware-0.17.4.tgz", "integrity": "sha1-ZC6ISIUdZvCdTxJJEoRtuutBuDM=", "requires": { - "http-proxy": "1.16.2", + "http-proxy": "1.17.0", "is-glob": "3.1.0", "lodash": "4.17.5", "micromatch": "2.3.11" @@ -6798,7 +6817,7 @@ "babel-preset-es2015": "6.24.1", "babel-preset-stage-1": "6.24.1", "babel-register": "6.26.0", - "babylon": "7.0.0-beta.44", + "babylon": "7.0.0-beta.46", "colors": "1.1.2", "flow-parser": "0.70.0", "lodash": "4.17.5", @@ -6812,9 +6831,9 @@ }, "dependencies": { "babylon": { - "version": "7.0.0-beta.44", - "resolved": "https://registry.npmjs.org/babylon/-/babylon-7.0.0-beta.44.tgz", - "integrity": "sha512-5Hlm13BJVAioCHpImtFqNOF2H3ieTOHd0fmFGMxOJ9jgeFqeAwsv3u5P5cR7CSeFrkgHsT19DgFJkHV0/Mcd8g==" + "version": "7.0.0-beta.46", + "resolved": "https://registry.npmjs.org/babylon/-/babylon-7.0.0-beta.46.tgz", + "integrity": "sha512-WFJlg2WatdkXRFMpk7BN/Uzzkjkcjk+WaqnrSCpay+RYl4ypW9ZetZyT9kNt22IH/BQNst3M6PaaBn9IXsUNrg==" } } }, @@ -7036,7 +7055,7 @@ "expand-braces": "0.1.2", "glob": "7.1.2", "graceful-fs": "4.1.11", - "http-proxy": "1.16.2", + "http-proxy": "1.17.0", "isbinaryfile": "3.0.2", "lodash": "4.17.5", "log4js": "2.5.3", @@ -7882,7 +7901,7 @@ "requires": { "commondir": "1.0.1", "deep-extend": "0.4.2", - "ejs": "2.5.8", + "ejs": "2.5.9", "glob": "7.1.2", "globby": "6.1.0", "mkdirp": "0.5.1", @@ -8543,7 +8562,7 @@ "stream-browserify": "2.0.1", "stream-http": "2.8.1", "string_decoder": "1.1.1", - "timers-browserify": "2.0.9", + "timers-browserify": "2.0.10", "tty-browserify": "0.0.0", "url": "0.11.0", "util": "0.10.3", @@ -9136,7 +9155,7 @@ "browserify-aes": "1.2.0", "create-hash": "1.2.0", "evp_bytestokey": "1.0.3", - "pbkdf2": "3.0.14" + "pbkdf2": "3.0.16" } }, "parse-glob": { @@ -9258,13 +9277,13 @@ "dev": true }, "pbkdf2": { - "version": "3.0.14", - "resolved": "https://registry.npmjs.org/pbkdf2/-/pbkdf2-3.0.14.tgz", - "integrity": "sha512-gjsZW9O34fm0R7PaLHRJmLLVfSoesxztjPjE9o6R+qtVJij90ltg1joIovN9GKrRW3t1PzhDDG3UMEMFfZ+1wA==", + "version": "3.0.16", + "resolved": "https://registry.npmjs.org/pbkdf2/-/pbkdf2-3.0.16.tgz", + "integrity": "sha512-y4CXP3thSxqf7c0qmOF+9UeOTrifiVTIM+u7NWlq+PRsHbr7r7dpCmvzrZxa96JJUNi0Y5w9VqG5ZNeCVMoDcA==", "requires": { "create-hash": "1.2.0", "create-hmac": "1.1.7", - "ripemd160": "2.0.1", + "ripemd160": "2.0.2", "safe-buffer": "5.1.1", "sha.js": "2.4.11" } @@ -9468,7 +9487,7 @@ "caniuse-api": "1.6.1", "postcss": "5.2.18", "postcss-selector-parser": "2.2.3", - "vendors": "1.0.1" + "vendors": "1.0.2" } }, "postcss-message-helpers": { @@ -9874,9 +9893,9 @@ } }, "posthtml-rename-id": { - "version": "1.0.4", - "resolved": "https://registry.npmjs.org/posthtml-rename-id/-/posthtml-rename-id-1.0.4.tgz", - "integrity": "sha512-bxsGN02JGqcihc9eztWu8Qlj2P/If9sY0ckYmEL+6hqrWRvwJw4RvnXSnlKmjS4yDBcT4cSpJdMy+xsSuHDvZw==", + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/posthtml-rename-id/-/posthtml-rename-id-1.0.5.tgz", + "integrity": "sha512-s1Zc2r8nxke55DCUbBE0EOUWqnt42BHIOl80GW20iNMCRZbkaPWRHTbSEVOmfng+Zl1F9wZL4LXVYDCzl6nYhw==", "requires": { "escape-string-regexp": "1.0.5" } @@ -10916,22 +10935,12 @@ } }, "ripemd160": { - "version": "2.0.1", - "resolved": "https://registry.npmjs.org/ripemd160/-/ripemd160-2.0.1.tgz", - "integrity": "sha1-D0WEKVxTo2KK9+bXmsohzlfRxuc=", + "version": "2.0.2", + "resolved": "https://registry.npmjs.org/ripemd160/-/ripemd160-2.0.2.tgz", + "integrity": "sha512-ii4iagi25WusVoiC4B4lq7pbXfAp3D9v5CwfkY33vffw2+pkDjY1D8GaN7spsxvCSx8dkPqOZCEZyfxcmJG2IA==", "requires": { - "hash-base": "2.0.2", + "hash-base": "3.0.4", "inherits": "2.0.3" - }, - "dependencies": { - "hash-base": { - "version": "2.0.2", - "resolved": "https://registry.npmjs.org/hash-base/-/hash-base-2.0.2.tgz", - "integrity": "sha1-ZuodhW206KVHDK32/OI65SRO8uE=", - "requires": { - "inherits": "2.0.3" - } - } } }, "roboto-font": { @@ -11765,7 +11774,7 @@ "faye-websocket": "0.11.1", "inherits": "2.0.3", "json3": "3.3.2", - "url-parse": "1.3.0" + "url-parse": "1.4.0" }, "dependencies": { "faye-websocket": { @@ -12202,7 +12211,7 @@ "micromatch": "3.1.0", "postcss": "5.2.18", "postcss-prefix-selector": "1.6.0", - "posthtml-rename-id": "1.0.4", + "posthtml-rename-id": "1.0.5", "posthtml-svg-mode": "1.0.2", "query-string": "4.3.4", "traverse": "0.6.6" @@ -12533,9 +12542,9 @@ "integrity": "sha1-8y6srFoXW+ol1/q1Zas+2HQe9W8=" }, "timers-browserify": { - "version": "2.0.9", - "resolved": "https://registry.npmjs.org/timers-browserify/-/timers-browserify-2.0.9.tgz", - "integrity": "sha512-2DhyvVpCWwY7gk8UmKhYvgHQl9XTlO0Dg0/2UZcLgPnpulhdm2aGIlFy5rU5igmOCA51w6jPHqLRA4UH1YmhcA==", + "version": "2.0.10", + "resolved": "https://registry.npmjs.org/timers-browserify/-/timers-browserify-2.0.10.tgz", + "integrity": "sha512-YvC1SV1XdOUaL6gx5CoGroT3Gu49pK9+TZ38ErPldOWW4j49GI1HKs9DV+KGq/w6y+LZ72W1c8cKz2vzY+qpzg==", "requires": { "setimmediate": "1.0.5" } @@ -12998,18 +13007,18 @@ "integrity": "sha1-TTNA6AfTdzvamZH4MFrNzCpmXSo=" }, "url-parse": { - "version": "1.3.0", - "resolved": "https://registry.npmjs.org/url-parse/-/url-parse-1.3.0.tgz", - "integrity": "sha512-zPvPA3T7P6M+0iNsgX+iAcAz4GshKrowtQBHHc/28tVsBc8jK7VRCNX+2GEcoE6zDB6XqXhcyiUWPVZY6C70Cg==", + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/url-parse/-/url-parse-1.4.0.tgz", + "integrity": "sha512-ERuGxDiQ6Xw/agN4tuoCRbmwRuZP0cJ1lJxJubXr5Q/5cDa78+Dc4wfvtxzhzhkm5VvmW6Mf8EVj9SPGN4l8Lg==", "requires": { - "querystringify": "1.0.0", + "querystringify": "2.0.0", "requires-port": "1.0.0" }, "dependencies": { "querystringify": { - "version": "1.0.0", - "resolved": "https://registry.npmjs.org/querystringify/-/querystringify-1.0.0.tgz", - "integrity": "sha1-YoYkIRLFtxL6ZU5SZlK/ahP/Bcs=" + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/querystringify/-/querystringify-2.0.0.tgz", + "integrity": "sha512-eTPo5t/4bgaMNZxyjWx6N2a6AuE0mq51KWvpc7nU/MAqixcI6v6KrGUKES0HaomdnolQBBXU/++X6/QQ9KL4tw==" } } }, @@ -13130,9 +13139,9 @@ "integrity": "sha1-IpnwLG3tMNSllhsLn3RSShj2NPw=" }, "vendors": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/vendors/-/vendors-1.0.1.tgz", - "integrity": "sha1-N61zyO5Bf7PVgOeFMSMH0nSEfyI=" + "version": "1.0.2", + "resolved": "https://registry.npmjs.org/vendors/-/vendors-1.0.2.tgz", + "integrity": "sha512-w/hry/368nO21AN9QljsaIhb9ZiZtZARoVH5f3CsFbawdLdayCgKRPup7CggujvySMxx0I91NOyxdVENohprLQ==" }, "verror": { "version": "1.10.0", @@ -13663,9 +13672,9 @@ } }, "cliui": { - "version": "4.0.0", - "resolved": "https://registry.npmjs.org/cliui/-/cliui-4.0.0.tgz", - "integrity": "sha512-nY3W5Gu2racvdDk//ELReY+dHjb9PlIcVDFXP72nVIhq2Gy3LuVXYwJoPVudwQnv1shtohpgkdCKT2YaKY0CKw==", + "version": "4.1.0", + "resolved": "https://registry.npmjs.org/cliui/-/cliui-4.1.0.tgz", + "integrity": "sha512-4FG+RSG9DL7uEwRUZXZn3SS34DiDPfzP0VOiEwtUWlE+AR2EIg+hSyvrIgUUfhdgR/UkAeW2QHgeP+hWrXs7jQ==", "requires": { "string-width": "2.1.1", "strip-ansi": "4.0.0", @@ -13763,7 +13772,7 @@ "resolved": "https://registry.npmjs.org/yargs/-/yargs-11.1.0.tgz", "integrity": "sha512-NwW69J42EsCSanF8kyn5upxvjp5ds+t3+udGBeTbFnERA+lF541DDpMawzo4z6W/QrzNM18D+BPMiOBibnFV5A==", "requires": { - "cliui": "4.0.0", + "cliui": "4.1.0", "decamelize": "1.2.0", "find-up": "2.1.0", "get-caller-file": "1.0.2", diff --git a/modules/web-console/frontend/public/images/icons/eyeClosed.svg b/modules/web-console/frontend/public/images/icons/eyeClosed.svg new file mode 100644 index 0000000000000..32bcba84e9bed --- /dev/null +++ b/modules/web-console/frontend/public/images/icons/eyeClosed.svg @@ -0,0 +1,6 @@ + + + + + + diff --git a/modules/web-console/frontend/public/images/icons/eyeOpened.svg b/modules/web-console/frontend/public/images/icons/eyeOpened.svg new file mode 100644 index 0000000000000..74c3f7845d08a --- /dev/null +++ b/modules/web-console/frontend/public/images/icons/eyeOpened.svg @@ -0,0 +1,7 @@ + + + + + + + diff --git a/modules/web-console/frontend/public/images/icons/index.js b/modules/web-console/frontend/public/images/icons/index.js index da7c3abd282e1..62b3ab842e492 100644 --- a/modules/web-console/frontend/public/images/icons/index.js +++ b/modules/web-console/frontend/public/images/icons/index.js @@ -37,4 +37,5 @@ export collapse from './collapse.svg'; export expand from './expand.svg'; export home from './home.svg'; export refresh from './refresh.svg'; - +export {default as eyeOpened} from './eyeOpened.svg'; +export {default as eyeClosed} from './eyeClosed.svg'; From 9f6c9d0d96f297e8b89810147fb358ff9531a104 Mon Sep 17 00:00:00 2001 From: Dmitriy Shabalin Date: Mon, 23 Apr 2018 16:19:32 +0700 Subject: [PATCH 0159/1463] IGNITE-8298 Web Console: Fixed tables UI issues. (cherry picked from commit f5a6525) --- .../frontend/app/primitives/ui-grid-header/index.scss | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/modules/web-console/frontend/app/primitives/ui-grid-header/index.scss b/modules/web-console/frontend/app/primitives/ui-grid-header/index.scss index 1ff27b29c05dd..4d5d455547ac4 100644 --- a/modules/web-console/frontend/app/primitives/ui-grid-header/index.scss +++ b/modules/web-console/frontend/app/primitives/ui-grid-header/index.scss @@ -115,7 +115,8 @@ } } -.ui-grid[ui-grid-selection][ui-grid-grouping] { +.ui-grid[ui-grid-selection][ui-grid-grouping], +.ui-grid[ui-grid-selection][ui-grid-tree-view] { .ui-grid-pinned-container-left { .ui-grid-header--subcategories { .ui-grid-header-span { From bf80eebd83325588a43e3f1987f25060d69c5f68 Mon Sep 17 00:00:00 2001 From: Ilya Borisov Date: Wed, 25 Apr 2018 13:22:05 +0700 Subject: [PATCH 0160/1463] IGNITE-8314 Web Console: Added progress-line component. (cherry picked from commit 8a0ed0e) --- modules/web-console/frontend/app/app.js | 2 + .../panel-collapsible/index.spec.js | 1 - .../app/components/progress-line/component.js | 28 +++++++ .../components/progress-line/controller.js | 60 ++++++++++++++ .../app/components/progress-line/index.js | 23 ++++++ .../components/progress-line/index.spec.js | 69 ++++++++++++++++ .../app/components/progress-line/style.scss | 82 +++++++++++++++++++ .../app/components/progress-line/template.pug | 18 ++++ .../frontend/test/karma.conf.babel.js | 2 + 9 files changed, 284 insertions(+), 1 deletion(-) create mode 100644 modules/web-console/frontend/app/components/progress-line/component.js create mode 100644 modules/web-console/frontend/app/components/progress-line/controller.js create mode 100644 modules/web-console/frontend/app/components/progress-line/index.js create mode 100644 modules/web-console/frontend/app/components/progress-line/index.spec.js create mode 100644 modules/web-console/frontend/app/components/progress-line/style.scss create mode 100644 modules/web-console/frontend/app/components/progress-line/template.pug diff --git a/modules/web-console/frontend/app/app.js b/modules/web-console/frontend/app/app.js index c6720bc7c54c6..81a5bc7fbacc1 100644 --- a/modules/web-console/frontend/app/app.js +++ b/modules/web-console/frontend/app/app.js @@ -139,6 +139,7 @@ import connectedClusters from './components/connected-clusters'; import pageSignIn from './components/page-signin'; import pageLanding from './components/page-landing'; import passwordVisibility from './components/password-visibility'; +import progressLine from './components/progress-line'; import pageProfile from './components/page-profile'; import pagePasswordChanged from './components/page-password-changed'; @@ -241,6 +242,7 @@ angular.module('ignite-console', [ uiAceSpring.name, breadcrumbs.name, passwordVisibility.name, + progressLine.name, // Ignite modules. IgniteModules.name ]) diff --git a/modules/web-console/frontend/app/components/panel-collapsible/index.spec.js b/modules/web-console/frontend/app/components/panel-collapsible/index.spec.js index 79722464c1a07..b5c00cd5a3933 100644 --- a/modules/web-console/frontend/app/components/panel-collapsible/index.spec.js +++ b/modules/web-console/frontend/app/components/panel-collapsible/index.spec.js @@ -18,7 +18,6 @@ import 'mocha'; import {assert} from 'chai'; import angular from 'angular'; -import 'angular-mocks'; import {spy} from 'sinon'; import componentModule from './index.js'; diff --git a/modules/web-console/frontend/app/components/progress-line/component.js b/modules/web-console/frontend/app/components/progress-line/component.js new file mode 100644 index 0000000000000..f998ec809f023 --- /dev/null +++ b/modules/web-console/frontend/app/components/progress-line/component.js @@ -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. + */ + +import './style.scss'; +import controller from './controller'; +import template from './template.pug'; + +export default { + controller, + template, + bindings: { + value: '}} changes + */ + $onChanges(changes) { + if (changes.value.currentValue === -1) { + this.$element[0].classList.remove(COMPLETE_CLASS); + this.$element[0].classList.add(INDETERMINATE_CLASS); + return; + } + if (typeof changes.value.currentValue === 'number') { + if (changes.value.currentValue === 1) this.$element[0].classList.add(COMPLETE_CLASS); + this.$element[0].classList.remove(INDETERMINATE_CLASS); + } + } +} diff --git a/modules/web-console/frontend/app/components/progress-line/index.js b/modules/web-console/frontend/app/components/progress-line/index.js new file mode 100644 index 0000000000000..a91b97bb259f3 --- /dev/null +++ b/modules/web-console/frontend/app/components/progress-line/index.js @@ -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. + */ + +import angular from 'angular'; +import component from './component'; + +export default angular + .module('ignite-console.progress-line', []) + .component('progressLine', component); diff --git a/modules/web-console/frontend/app/components/progress-line/index.spec.js b/modules/web-console/frontend/app/components/progress-line/index.spec.js new file mode 100644 index 0000000000000..f5725e938a3b5 --- /dev/null +++ b/modules/web-console/frontend/app/components/progress-line/index.spec.js @@ -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. + */ + +import 'mocha'; +import {assert} from 'chai'; +import angular from 'angular'; +import module from './index'; + +const INDETERMINATE_CLASS = 'progress-line__indeterminate'; +const COMPLETE_CLASS = 'progress-line__complete'; + +suite('progress-line', () => { + let $scope; + let $compile; + + setup(() => { + angular.module('test', [module.name]); + angular.mock.module('test'); + angular.mock.inject((_$rootScope_, _$compile_) => { + $compile = _$compile_; + $scope = _$rootScope_.$new(); + }); + }); + + test('Progress states', () => { + $scope.progress = -1; + const el = angular.element(``); + + $compile(el)($scope); + $scope.$digest(); + + assert.isTrue( + el[0].classList.contains(INDETERMINATE_CLASS), + 'Adds indeterminate class for indeterminate state' + ); + + assert.isFalse( + el[0].classList.contains(COMPLETE_CLASS), + 'Does not have complete class when in indeterminate state' + ); + + $scope.progress = 1; + $scope.$digest(); + + assert.isFalse( + el[0].classList.contains(INDETERMINATE_CLASS), + 'Does not has indeterminate class when in finished state' + ); + + assert.isTrue( + el[0].classList.contains(COMPLETE_CLASS), + 'Adds complete class when in finished state' + ); + }); +}); diff --git a/modules/web-console/frontend/app/components/progress-line/style.scss b/modules/web-console/frontend/app/components/progress-line/style.scss new file mode 100644 index 0000000000000..3b7f4efe70b2a --- /dev/null +++ b/modules/web-console/frontend/app/components/progress-line/style.scss @@ -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. + */ + +progress-line { + @import 'public/stylesheets/variables'; + + --background-color: transparent; + --foreground-color: #{$ignite-brand-primary}; + + height: 1px; + position: relative; + display: block; + overflow: hidden; + + @keyframes progress-line-indeterminate { + 0% { + left: -33%; + width: 33%; + } + 100% { + left: 100%; + width: 33%; + } + } + + @keyframes progress-line-indeterminate-to-complete { + 0% { + opacity: 0; + } + 100% { + opacity: 1; + } + } + + .progress-line__background { + position: absolute; + top: 0; + left: 0; + right: 0; + bottom: 0; + display: block; + background: var(--background-color); + } + + .progress-line__foreground { + position: absolute; + top: 0; + bottom: 0; + content: ""; + display: block; + background: var(--foreground-color); + } + + &.progress-line__complete .progress-line__foreground { + animation-name: progress-line-indeterminate-to-complete; + animation-iteration-count: 1; + animation-duration: 0.2s; + left: 0; + right: 0; + width: 100%; + } + + &.progress-line__indeterminate .progress-line__foreground { + animation-name: progress-line-indeterminate; + animation-iteration-count: infinite; + animation-duration: 2s; + } +} diff --git a/modules/web-console/frontend/app/components/progress-line/template.pug b/modules/web-console/frontend/app/components/progress-line/template.pug new file mode 100644 index 0000000000000..b48beae29bf8c --- /dev/null +++ b/modules/web-console/frontend/app/components/progress-line/template.pug @@ -0,0 +1,18 @@ +//- + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + +.progress-line__background +.progress-line__foreground diff --git a/modules/web-console/frontend/test/karma.conf.babel.js b/modules/web-console/frontend/test/karma.conf.babel.js index b19259421e2bc..9bf92681153ed 100644 --- a/modules/web-console/frontend/test/karma.conf.babel.js +++ b/modules/web-console/frontend/test/karma.conf.babel.js @@ -30,6 +30,8 @@ export default (config) => { // List of files / patterns to load in the browser. files: [ 'node_modules/babel-polyfill/dist/polyfill.js', + 'node_modules/angular/angular.js', + 'node_modules/angular-mocks/angular-mocks.js', 'test/**/*.test.js', 'app/**/*.spec.js' ], From e3a7943434759ae16877c2fad7d08ebd4eaf53ea Mon Sep 17 00:00:00 2001 From: Ilya Borisov Date: Wed, 25 Apr 2018 16:13:19 +0700 Subject: [PATCH 0161/1463] IGNITE-8370 Web Console: Split "Sign In" page to three separate pages. (cherry picked from commit 2f8df26) --- .../testcafe/fixtures/auth/forgot-password.js | 22 +- .../e2e/testcafe/fixtures/auth/logout.js | 5 +- .../e2e/testcafe/fixtures/auth/signup.js | 14 +- .../e2e/testcafe/page-models/PageSignIn.js | 77 ------- .../page-models/pageForgotPassword.js | 24 +++ .../e2e/testcafe/page-models/pageSignin1.js | 32 +++ .../e2e/testcafe/page-models/pageSignup.js | 48 +++++ modules/web-console/e2e/testcafe/roles.js | 4 +- modules/web-console/frontend/app/app.js | 8 +- .../page-forgot-password/component.js | 30 +++ .../page-forgot-password/controller.js | 57 +++++ .../components/page-forgot-password/index.js | 28 +++ .../components/page-forgot-password/run.js | 48 +++++ .../page-forgot-password/style.scss | 54 +++++ .../page-forgot-password/template.pug | 50 +++++ .../components/page-forgot-password/types.ts | 24 +++ .../app/components/page-landing/template.pug | 4 +- .../app/components/page-signin/component.js | 26 +++ .../app/components/page-signin/controller.js | 107 ++-------- .../app/components/page-signin/index.js | 38 +--- .../app/components/page-signin/run.js | 53 +++++ .../app/components/page-signin/style.scss | 49 +++-- .../app/components/page-signin/template.pug | 195 ++++-------------- .../app/components/page-signin/types.ts | 26 +++ .../app/components/page-signup/component.js | 28 +++ .../app/components/page-signup/controller.js | 60 ++++++ .../app/components/page-signup/index.js | 28 +++ .../app/components/page-signup/run.js | 35 ++++ .../app/components/page-signup/style.scss | 61 ++++++ .../app/components/page-signup/template.pug | 122 +++++++++++ .../app/components/page-signup/types.ts | 36 ++++ .../app/primitives/form-field/dropdown.pug | 1 + .../web-console/frontend/app/types/index.ts | 33 +++ .../frontend/public/stylesheets/style.scss | 23 --- 34 files changed, 1020 insertions(+), 430 deletions(-) delete mode 100644 modules/web-console/e2e/testcafe/page-models/PageSignIn.js create mode 100644 modules/web-console/e2e/testcafe/page-models/pageForgotPassword.js create mode 100644 modules/web-console/e2e/testcafe/page-models/pageSignin1.js create mode 100644 modules/web-console/e2e/testcafe/page-models/pageSignup.js create mode 100644 modules/web-console/frontend/app/components/page-forgot-password/component.js create mode 100644 modules/web-console/frontend/app/components/page-forgot-password/controller.js create mode 100644 modules/web-console/frontend/app/components/page-forgot-password/index.js create mode 100644 modules/web-console/frontend/app/components/page-forgot-password/run.js create mode 100644 modules/web-console/frontend/app/components/page-forgot-password/style.scss create mode 100644 modules/web-console/frontend/app/components/page-forgot-password/template.pug create mode 100644 modules/web-console/frontend/app/components/page-forgot-password/types.ts create mode 100644 modules/web-console/frontend/app/components/page-signin/component.js create mode 100644 modules/web-console/frontend/app/components/page-signin/run.js create mode 100644 modules/web-console/frontend/app/components/page-signin/types.ts create mode 100644 modules/web-console/frontend/app/components/page-signup/component.js create mode 100644 modules/web-console/frontend/app/components/page-signup/controller.js create mode 100644 modules/web-console/frontend/app/components/page-signup/index.js create mode 100644 modules/web-console/frontend/app/components/page-signup/run.js create mode 100644 modules/web-console/frontend/app/components/page-signup/style.scss create mode 100644 modules/web-console/frontend/app/components/page-signup/template.pug create mode 100644 modules/web-console/frontend/app/components/page-signup/types.ts create mode 100644 modules/web-console/frontend/app/types/index.ts diff --git a/modules/web-console/e2e/testcafe/fixtures/auth/forgot-password.js b/modules/web-console/e2e/testcafe/fixtures/auth/forgot-password.js index 6b151edae2487..27ac2db155345 100644 --- a/modules/web-console/e2e/testcafe/fixtures/auth/forgot-password.js +++ b/modules/web-console/e2e/testcafe/fixtures/auth/forgot-password.js @@ -16,13 +16,12 @@ */ import { dropTestDB, resolveUrl, insertTestUser } from '../../envtools'; -import {PageSignIn} from '../../page-models/PageSignIn'; +import {PageSignIn} from '../../page-models/pageSignin1'; import {errorNotification} from '../../components/notifications'; - -const page = new PageSignIn(); +import {pageForgotPassword as page} from '../../page-models/pageForgotPassword'; fixture('Password reset') - .page(resolveUrl('/signin')) + .page(resolveUrl('/forgot-password')) .before(async() => { await dropTestDB(); await insertTestUser(); @@ -33,26 +32,23 @@ fixture('Password reset') test('Incorrect email', async(t) => { await t - .click(page.showForgotPasswordButton) - .typeText(page.forgotPassword.email.control, 'aa') - .expect(page.forgotPassword.email.getError('email').exists).ok('Marks field as invalid') + .typeText(page.email.control, 'aa') + .expect(page.email.getError('email').exists).ok('Marks field as invalid') .expect(page.remindPasswordButton.getAttribute('disabled')).ok('Disables submit button'); }); test('Unknown email', async(t) => { await t - .click(page.showForgotPasswordButton) - .typeText(page.forgotPassword.email.control, 'nonexisting@mail.com', {replace: true}) + .typeText(page.email.control, 'nonexisting@mail.com', {replace: true}) .click(page.remindPasswordButton) .expect(errorNotification.withText('Account with that email address does not exists!').exists).ok('Shows global error notification') - .expect(page.forgotPassword.email.getError('server').exists).ok('Marks input as server-invalid'); + .expect(page.email.getError('server').exists).ok('Marks input as server-invalid'); }); // TODO: IGNITE-8028 Implement this test as unit test. test.skip('Successful reset', async(t) => { await t - .click(page.showForgotPasswordButton) - .typeText(page.forgotPassword.email.control, 'a@a', {replace: true}) + .typeText(page.email.control, 'a@a', {replace: true}) .click(page.remindPasswordButton) - .expect(page.forgotPassword.email.getError('server').exists).notOk('No errors happen'); + .expect(page.email.getError('server').exists).notOk('No errors happen'); }); diff --git a/modules/web-console/e2e/testcafe/fixtures/auth/logout.js b/modules/web-console/e2e/testcafe/fixtures/auth/logout.js index 196c03f017ac0..5cad60fe17e32 100644 --- a/modules/web-console/e2e/testcafe/fixtures/auth/logout.js +++ b/modules/web-console/e2e/testcafe/fixtures/auth/logout.js @@ -18,10 +18,9 @@ import {dropTestDB, resolveUrl, insertTestUser} from 'envtools'; import {createRegularUser} from '../../roles'; import {userMenu} from '../../components/userMenu'; -import {PageSignIn} from '../../page-models/PageSignIn'; +import {pageSignin} from '../../page-models/pageSignin1'; const user = createRegularUser(); -const pageSignIn = new PageSignIn(); fixture('Logout') .before(async() => { @@ -35,5 +34,5 @@ fixture('Logout') test('Successful logout', async(t) => { await t.useRole(user).navigateTo(resolveUrl('/settings/profile')); await userMenu.clickOption('Log out'); - await t.expect(pageSignIn._selector.exists).ok('Goes to sign in page after logout'); + await t.expect(pageSignin.selector.exists).ok('Goes to sign in page after logout'); }); diff --git a/modules/web-console/e2e/testcafe/fixtures/auth/signup.js b/modules/web-console/e2e/testcafe/fixtures/auth/signup.js index 065bff96b2d84..f0951d93824e8 100644 --- a/modules/web-console/e2e/testcafe/fixtures/auth/signup.js +++ b/modules/web-console/e2e/testcafe/fixtures/auth/signup.js @@ -16,14 +16,12 @@ */ import {dropTestDB, resolveUrl, insertTestUser} from '../../envtools'; -import {PageSignIn} from '../../page-models/PageSignIn'; +import {pageSignup as page} from '../../page-models/pageSignup'; import {errorNotification} from '../../components/notifications'; import {userMenu} from '../../components/userMenu'; -const page = new PageSignIn(); - fixture('Signup') - .page(resolveUrl('/signin')) + .page(resolveUrl('/signup')) .before(async() => { await dropTestDB(); await insertTestUser(); @@ -45,9 +43,9 @@ test('Local validation', async(t) => { country: 'Brazil' }); await t - .expect(page.signup.email.getError('email').exists).ok() - .expect(page.signup.passwordConfirm.getError('mismatch').exists).ok() - .expect(page.signup.firstName.getError('required').exists).ok() + .expect(page.email.getError('email').exists).ok() + .expect(page.passwordConfirm.getError('mismatch').exists).ok() + .expect(page.firstName.getError('required').exists).ok() .expect(isButtonDisabled).ok('Button disabled with invalid fields'); await page.fillSignupForm({ email: 'foobar@bar.baz', @@ -73,7 +71,7 @@ test('Server validation', async(t) => { await t .click(page.signupButton) .expect(errorNotification.withText('A user with the given username is already registered').exists).ok('Shows global error') - .expect(page.signup.email.getError('server').exists).ok('Marks email input as server-invalid'); + .expect(page.email.getError('server').exists).ok('Marks email input as server-invalid'); }); test('Successful signup', async(t) => { await page.fillSignupForm({ diff --git a/modules/web-console/e2e/testcafe/page-models/PageSignIn.js b/modules/web-console/e2e/testcafe/page-models/PageSignIn.js deleted file mode 100644 index 1712cd69f7703..0000000000000 --- a/modules/web-console/e2e/testcafe/page-models/PageSignIn.js +++ /dev/null @@ -1,77 +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. - */ - -import {Selector, t} from 'testcafe'; -import { resolveUrl } from '../envtools'; -import {AngularJSSelector} from 'testcafe-angular-selectors'; -import {CustomFormField} from '../components/FormField'; - -export class PageSignIn { - constructor() { - this._selector = Selector('page-sign-in'); - this.signin = { - email: new CustomFormField({model: '$ctrl.data.signin.email'}), - password: new CustomFormField({model: '$ctrl.data.signin.password'}) - }; - this.forgotPassword = { - email: new CustomFormField({model: '$ctrl.data.remindPassword.email'}) - }; - this.signup = { - email: new CustomFormField({model: '$ctrl.data.signup.email'}), - password: new CustomFormField({model: '$ctrl.data.signup.password'}), - passwordConfirm: new CustomFormField({model: 'confirm'}), - firstName: new CustomFormField({model: '$ctrl.data.signup.firstName'}), - lastName: new CustomFormField({model: '$ctrl.data.signup.lastName'}), - company: new CustomFormField({model: '$ctrl.data.signup.company'}), - country: new CustomFormField({model: '$ctrl.data.signup.country'}) - }; - this.signinButton = Selector('#signin_submit'); - this.signupButton = Selector('#signup_submit'); - this.showForgotPasswordButton = Selector('#forgot_show'); - this.remindPasswordButton = Selector('#forgot_submit'); - } - - async open() { - await t.navigateTo(resolveUrl('/signin')); - } - - async login(email, password) { - return await t - .typeText(this.signin.email.control, email) - .typeText(this.signin.password.control, password) - .click(this.signinButton); - } - - async fillSignupForm({ - email, - password, - passwordConfirm, - firstName, - lastName, - company, - country - }) { - await t - .typeText(this.signup.email.control, email, {replace: true}) - .typeText(this.signup.password.control, password, {replace: true}) - .typeText(this.signup.passwordConfirm.control, passwordConfirm, {replace: true}) - .typeText(this.signup.firstName.control, firstName, {replace: true}) - .typeText(this.signup.lastName.control, lastName, {replace: true}) - .typeText(this.signup.company.control, company, {replace: true}); - await this.signup.country.selectOption(country); - } -} diff --git a/modules/web-console/e2e/testcafe/page-models/pageForgotPassword.js b/modules/web-console/e2e/testcafe/page-models/pageForgotPassword.js new file mode 100644 index 0000000000000..3b4160a24ef2c --- /dev/null +++ b/modules/web-console/e2e/testcafe/page-models/pageForgotPassword.js @@ -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. + */ + +import {Selector} from 'testcafe'; +import {CustomFormField} from '../components/FormField'; + +export const pageForgotPassword = { + email: new CustomFormField({model: '$ctrl.data.email'}), + remindPasswordButton: Selector('button').withText('Send it to me') +}; diff --git a/modules/web-console/e2e/testcafe/page-models/pageSignin1.js b/modules/web-console/e2e/testcafe/page-models/pageSignin1.js new file mode 100644 index 0000000000000..57031b6f3a0d0 --- /dev/null +++ b/modules/web-console/e2e/testcafe/page-models/pageSignin1.js @@ -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. + */ + +import {Selector, t} from 'testcafe'; +import {CustomFormField} from '../components/FormField'; + +export const pageSignin = { + email: new CustomFormField({model: '$ctrl.data.email'}), + password: new CustomFormField({model: '$ctrl.data.password'}), + signinButton: Selector('button').withText('Sign In'), + selector: Selector('page-signin'), + async login(email, password) { + return await t + .typeText(this.email.control, email) + .typeText(this.password.control, password) + .click(this.signinButton); + } +}; diff --git a/modules/web-console/e2e/testcafe/page-models/pageSignup.js b/modules/web-console/e2e/testcafe/page-models/pageSignup.js new file mode 100644 index 0000000000000..c380abed41907 --- /dev/null +++ b/modules/web-console/e2e/testcafe/page-models/pageSignup.js @@ -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. + */ + +import {Selector, t} from 'testcafe'; +import {CustomFormField} from '../components/FormField'; + +export const pageSignup = { + email: new CustomFormField({model: '$ctrl.data.email'}), + password: new CustomFormField({model: '$ctrl.data.password'}), + passwordConfirm: new CustomFormField({model: 'confirm'}), + firstName: new CustomFormField({model: '$ctrl.data.firstName'}), + lastName: new CustomFormField({model: '$ctrl.data.lastName'}), + company: new CustomFormField({model: '$ctrl.data.company'}), + country: new CustomFormField({model: '$ctrl.data.country'}), + signupButton: Selector('button').withText('Sign Up'), + async fillSignupForm({ + email, + password, + passwordConfirm, + firstName, + lastName, + company, + country + }) { + await t + .typeText(this.email.control, email, {replace: true}) + .typeText(this.password.control, password, {replace: true}) + .typeText(this.passwordConfirm.control, passwordConfirm, {replace: true}) + .typeText(this.firstName.control, firstName, {replace: true}) + .typeText(this.lastName.control, lastName, {replace: true}) + .typeText(this.company.control, company, {replace: true}); + await this.country.selectOption(country); + } +}; diff --git a/modules/web-console/e2e/testcafe/roles.js b/modules/web-console/e2e/testcafe/roles.js index 5f584b234aad1..5d9bfcc434c1f 100644 --- a/modules/web-console/e2e/testcafe/roles.js +++ b/modules/web-console/e2e/testcafe/roles.js @@ -17,7 +17,7 @@ const { Role, t } = require('testcafe'); import { resolveUrl } from './envtools'; -const { PageSignIn } = require('./page-models/PageSignIn'); +import {pageSignin as page} from './page-models/pageSignin1'; export const createRegularUser = () => { return new Role(resolveUrl('/signin'), async() => { @@ -25,8 +25,6 @@ export const createRegularUser = () => { // Disable "Getting started" modal. await t.eval(() => window.localStorage.showGettingStarted = 'false'); - - const page = new PageSignIn(); await page.login('a@a', 'a'); }); }; diff --git a/modules/web-console/frontend/app/app.js b/modules/web-console/frontend/app/app.js index 81a5bc7fbacc1..350005feb5b01 100644 --- a/modules/web-console/frontend/app/app.js +++ b/modules/web-console/frontend/app/app.js @@ -136,7 +136,6 @@ import breadcrumbs from './components/breadcrumbs'; import panelCollapsible from './components/panel-collapsible'; import clusterSelector from './components/cluster-selector'; import connectedClusters from './components/connected-clusters'; -import pageSignIn from './components/page-signin'; import pageLanding from './components/page-landing'; import passwordVisibility from './components/password-visibility'; import progressLine from './components/progress-line'; @@ -144,6 +143,9 @@ import progressLine from './components/progress-line'; import pageProfile from './components/page-profile'; import pagePasswordChanged from './components/page-password-changed'; import pagePasswordReset from './components/page-password-reset'; +import pageSignup from './components/page-signup'; +import pageSignin from './components/page-signin'; +import pageForgotPassword from './components/page-forgot-password'; import igniteServices from './services'; @@ -234,10 +236,12 @@ angular.module('ignite-console', [ igniteListOfRegisteredUsers.name, pageProfile.name, exposeInput.name, - pageSignIn.name, pageLanding.name, pagePasswordChanged.name, pagePasswordReset.name, + pageSignup.name, + pageSignin.name, + pageForgotPassword.name, uiAceJava.name, uiAceSpring.name, breadcrumbs.name, diff --git a/modules/web-console/frontend/app/components/page-forgot-password/component.js b/modules/web-console/frontend/app/components/page-forgot-password/component.js new file mode 100644 index 0000000000000..8c70e73786a53 --- /dev/null +++ b/modules/web-console/frontend/app/components/page-forgot-password/component.js @@ -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. + */ + +import template from './template.pug'; +import controller from './controller'; + +import './style.scss'; + +/** @type {ng.IComponentOptions} */ +export default { + controller, + template, + bindings: { + email: ' !this.serverError; + } + /** + * @param {{email: ng.IChangesObject}} changes + */ + $onChanges(changes) { + if ('email' in changes) this.data.email = changes.email.currentValue; + } + remindPassword() { + return this.Auth.remindPassword(this.data.email).catch((res) => { + this.IgniteMessages.showError(null, res.data); + this.serverError = res.data; + this.form.email.$validate(); + }); + } +} diff --git a/modules/web-console/frontend/app/components/page-forgot-password/index.js b/modules/web-console/frontend/app/components/page-forgot-password/index.js new file mode 100644 index 0000000000000..dec8fd9f82cad --- /dev/null +++ b/modules/web-console/frontend/app/components/page-forgot-password/index.js @@ -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. + */ + +import angular from 'angular'; +import component from './component'; +import {registerState} from './run'; + +export default angular + .module('ignite-console.page-forgot-password', [ + 'ui.router', + 'ignite-console.user' + ]) + .component('pageForgotPassword', component) + .run(registerState); diff --git a/modules/web-console/frontend/app/components/page-forgot-password/run.js b/modules/web-console/frontend/app/components/page-forgot-password/run.js new file mode 100644 index 0000000000000..c7ee6a64a1d3f --- /dev/null +++ b/modules/web-console/frontend/app/components/page-forgot-password/run.js @@ -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. + */ + +/** + * @param {import("@uirouter/angularjs").UIRouter} $uiRouter + */ +export function registerState($uiRouter) { + /** @type {import("app/types").IIgniteNg1StateDeclaration} */ + const state = { + name: 'forgotPassword', + url: '/forgot-password', + component: 'pageForgotPassword', + unsaved: true, + tfMetaTags: { + title: 'Forgot Password' + }, + resolve: [ + { + token: 'email', + deps: ['$uiRouter'], + /** + * @param {import('@uirouter/angularjs').UIRouter} $uiRouter + */ + resolveFn($uiRouter) { + return $uiRouter.stateService.transition.targetState().params().email; + } + } + ] + }; + + $uiRouter.stateRegistry.register(state); +} + +registerState.$inject = ['$uiRouter']; diff --git a/modules/web-console/frontend/app/components/page-forgot-password/style.scss b/modules/web-console/frontend/app/components/page-forgot-password/style.scss new file mode 100644 index 0000000000000..8122c348cc380 --- /dev/null +++ b/modules/web-console/frontend/app/components/page-forgot-password/style.scss @@ -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. + */ + +page-forgot-password { + display: flex; + flex-direction: column; + flex: 1 0 auto; + + section { + margin-left: auto; + margin-right: auto; + width: 530px; + + h3 { + font-size: 38px; + font-weight: 300; + margin: 30px 0 30px; + } + + p { + margin-bottom: 20px; + } + + .form-field { + margin: 10px 0; + } + + .form-footer { + padding: 15px 0; + text-align: right; + border-top: 1px solid #e5e5e5; + display: flex; + align-items: center; + + .btn-ignite { + margin-left: auto; + } + } + } +} diff --git a/modules/web-console/frontend/app/components/page-forgot-password/template.pug b/modules/web-console/frontend/app/components/page-forgot-password/template.pug new file mode 100644 index 0000000000000..6a11baff9c863 --- /dev/null +++ b/modules/web-console/frontend/app/components/page-forgot-password/template.pug @@ -0,0 +1,50 @@ +//- + Licensed to the Apache Software Foundation (ASF) under one or more + contributor license agreements. See the NOTICE file distributed with + this work for additional information regarding copyright ownership. + The ASF licenses this file to You under the Apache License, Version 2.0 + (the "License"); you may not use this file except in compliance with + the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + +include /app/helpers/jade/mixins + +web-console-header + web-console-header-left + ignite-header-title + +.container.body-container + section + - const form = '$ctrl.form' + h3 Forgot password? + p That's ok! Simply enter your email below and a reset password link will be sent to you via email. You can then follow that link and select a new password. + form(name=form novalidate ng-submit='$ctrl.remindPassword()') + +form-field__email({ + label: 'Email:', + model: '$ctrl.data.email', + name: '"email"', + placeholder: 'Input email', + required: true + })( + ng-model-options='{allowInvalid: true}' + autocomplete='email' + ignite-auto-focus + tabindex='0' + ) + +form-field__error({error: 'server', message: `{{$ctrl.serverError}}`}) + footer.form-footer + a(ui-sref='signin') Back to sign in + button.btn-ignite.btn-ignite--primary( + tabindex='1' + type='submit' + ng-disabled=`!$ctrl.canSubmitForm(${form})` + ) Send it to me + +web-console-footer diff --git a/modules/web-console/frontend/app/components/page-forgot-password/types.ts b/modules/web-console/frontend/app/components/page-forgot-password/types.ts new file mode 100644 index 0000000000000..cbcff81109b20 --- /dev/null +++ b/modules/web-console/frontend/app/components/page-forgot-password/types.ts @@ -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. + */ + +export interface IForgotPasswordData { + email: string +} + +export interface IForgotPasswordFormController extends ng.IFormController { + email: ng.INgModelController +} diff --git a/modules/web-console/frontend/app/components/page-landing/template.pug b/modules/web-console/frontend/app/components/page-landing/template.pug index a1ce4f71d614b..7b5b1ccfdcba2 100644 --- a/modules/web-console/frontend/app/components/page-landing/template.pug +++ b/modules/web-console/frontend/app/components/page-landing/template.pug @@ -26,7 +26,7 @@ section.intro-container-wrapper h1 Web Console h2 An Interactive Configuration Wizard and Management Tool for Apache™ Ignite® p It provides an interactive configuration wizard which helps you create and download configuration files and code snippets for your Apache Ignite projects. Additionally, the tool allows you to automatically load SQL metadata from any RDBMS, run SQL queries on your in-memory cache as well as view execution plans, in-memory schema, and streaming charts. - a#signup_show.btn.btn-lg.btn-primary.btn-custom(ui-sref='signin') Try Now + a#signup_show.btn.btn-lg.btn-primary.btn-custom(ui-sref='signup') Sign Up .col-lg-6.col-md-6.col-sm-6.col-xs-12 ui-carousel(slides='$ctrl.images' autoplay='true' slides-to-show='1' arrows='false') carousel-item @@ -61,6 +61,6 @@ section.features-container-wrapper h3 Manage The Web Console users p With The Web Console you can have accounts with different roles. .align-center.text-center - a.btn.btn-lg.btn-primary.btn-custom(ui-sref='signin') Get Started + a.btn.btn-lg.btn-primary.btn-custom(ui-sref='signup') Get Started web-console-footer diff --git a/modules/web-console/frontend/app/components/page-signin/component.js b/modules/web-console/frontend/app/components/page-signin/component.js new file mode 100644 index 0000000000000..968ff396e298d --- /dev/null +++ b/modules/web-console/frontend/app/components/page-signin/component.js @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import template from './template.pug'; +import controller from './controller'; +import './style.scss'; + +/** @type {ng.IComponentOptions} */ +export default { + controller, + template +}; diff --git a/modules/web-console/frontend/app/components/page-signin/controller.js b/modules/web-console/frontend/app/components/page-signin/controller.js index fb576203b4255..b13cf4b2efb25 100644 --- a/modules/web-console/frontend/app/components/page-signin/controller.js +++ b/modules/web-console/frontend/app/components/page-signin/controller.js @@ -15,107 +15,42 @@ * limitations under the License. */ -// eslint-disable-next-line -import {default as AuthService} from 'app/modules/user/Auth.service'; - export default class { - /** @type {ng.IFormController} */ - form_signup; - /** @type {ng.IFormController} */ - form_signin; - /** @type {ng.IFormController} */ - form_forgot; + /** @type {import('./types').ISiginData} */ + data = { + email: null, + password: null + }; + /** @type {import('./types').ISigninFormController} */ + form; + /** @type {string} */ + serverError = null; - static $inject = ['IgniteFocus', 'IgniteCountries', 'Auth', 'IgniteMessages']; + static $inject = ['Auth', 'IgniteMessages']; /** - * @param {AuthService} Auth + * @param {import('app/modules/user/Auth.service').default} Auth */ - constructor(Focus, Countries, Auth, IgniteMessages) { + constructor(Auth, IgniteMessages) { this.Auth = Auth; this.IgniteMessages = IgniteMessages; - this.countries = Countries.getAll(); - - Focus.move('user_email'); - } - $onInit() { - const data = this.data = { - signup: { - email: null, - password: null, - firstName: null, - lastName: null, - company: null, - country: null - }, - signin: { - /** @type {string} */ - email: null, - /** @type {string} */ - password: null - }, - remindPassword: { - /** - * Explicitly mirrors signin email so user won't have to type it twice - * @type {string} - */ - get email() { - return data.signin.email; - }, - set email(value) { - data.signin.email = value; - } - } - }; - /** @type {('signin'|'remindPassword')} */ - this.activeForm = 'signin'; - /** @type {{signup: string, signin: string, remindPassword: string}} */ - this.serverErrors = { - signup: null, - signin: null, - remindPassword: null - }; - } - get isSigninOpened() { - return this.activeForm === 'signin'; } - get isRemindPasswordOpened() { - return this.activeForm === 'remindPassword'; - } - /** Toggles between signin and remind password forms */ - toggleActiveForm() { - this.activeForm = this.activeForm === 'signin' ? 'remindPassword' : 'signin'; - } - /** @param {ng.IFormController} form */ + + /** @param {import('./types').ISigninFormController} form */ canSubmitForm(form) { return form.$error.server ? true : !form.$invalid; } + $postLink() { - this.form_signup.signupEmail.$validators.server = () => !this.serverErrors.signup; - this.form_signin.signinEmail.$validators.server = () => !this.serverErrors.signin; - this.form_signin.signinPassword.$validators.server = () => !this.serverErrors.signin; - this.form_forgot.forgotEmail.$validators.server = () => !this.serverErrors.remindPassword; - } - signup() { - return this.Auth.signnup(this.data.signup).catch((res) => { - this.IgniteMessages.showError(null, res.data); - this.serverErrors.signup = res.data; - this.form_signup.signupEmail.$validate(); - }); + this.form.password.$validators.server = () => !this.serverError; } + signin() { - return this.Auth.signin(this.data.signin.email, this.data.signin.password).catch((res) => { - this.IgniteMessages.showError(null, res.data); - this.serverErrors.signin = res.data; - this.form_signin.signinEmail.$validate(); - this.form_signin.signinPassword.$validate(); - }); - } - remindPassword() { - return this.Auth.remindPassword(this.data.remindPassword.email).catch((res) => { + return this.Auth.signin(this.data.email, this.data.password).catch((res) => { this.IgniteMessages.showError(null, res.data); - this.serverErrors.remindPassword = res.data; - this.form_forgot.forgotEmail.$validate(); + this.serverError = res.data; + this.form.email.$validate(); + this.form.password.$validate(); }); } } diff --git a/modules/web-console/frontend/app/components/page-signin/index.js b/modules/web-console/frontend/app/components/page-signin/index.js index 6be374f27b3cc..ee027a69ddd99 100644 --- a/modules/web-console/frontend/app/components/page-signin/index.js +++ b/modules/web-console/frontend/app/components/page-signin/index.js @@ -16,41 +16,13 @@ */ import angular from 'angular'; - -import template from './template.pug'; -import controller from './controller'; -import './style.scss'; +import component from './component'; +import {registerState} from './run'; export default angular - .module('ignite-console.sign-in', [ + .module('ignite-console.page-signin', [ 'ui.router', 'ignite-console.user' ]) - .component('pageSignIn', { - controller, - template - }) - .config(['$stateProvider', function($stateProvider) { - // set up the states - $stateProvider - .state('signin', { - url: '/signin', - template: '', - redirectTo: (trans) => { - return trans.injector().get('User').read() - .then(() => { - try { - const {name, params} = JSON.parse(localStorage.getItem('lastStateChangeSuccess')); - - const restored = trans.router.stateService.target(name, params); - - return restored.valid() ? restored : 'default-state'; - } catch (ignored) { - return 'default-state'; - } - }) - .catch(() => true); - }, - unsaved: true - }); - }]); + .component('pageSignin', component) + .run(registerState); diff --git a/modules/web-console/frontend/app/components/page-signin/run.js b/modules/web-console/frontend/app/components/page-signin/run.js new file mode 100644 index 0000000000000..7ac3bf811483b --- /dev/null +++ b/modules/web-console/frontend/app/components/page-signin/run.js @@ -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. + */ + +/** + * @param {import("@uirouter/angularjs").UIRouter} $uiRouter + */ +export function registerState($uiRouter) { + /** @type {import("app/types").IIgniteNg1StateDeclaration} */ + const state = { + url: '/signin', + name: 'signin', + component: 'pageSignin', + unsaved: true, + redirectTo: (trans) => { + const skipStates = new Set(['signup', 'forgotPassword', 'landing']); + if (skipStates.has(trans.from().name)) return; + return trans.injector().get('User').read() + .then(() => { + try { + const {name, params} = JSON.parse(localStorage.getItem('lastStateChangeSuccess')); + + const restored = trans.router.stateService.target(name, params); + + return restored.valid() ? restored : 'default-state'; + } catch (ignored) { + return 'default-state'; + } + }) + .catch(() => true); + }, + tfMetaTags: { + title: 'Sign In' + } + }; + + $uiRouter.stateRegistry.register(state); +} + +registerState.$inject = ['$uiRouter']; diff --git a/modules/web-console/frontend/app/components/page-signin/style.scss b/modules/web-console/frontend/app/components/page-signin/style.scss index 8ea143af440cd..c4dee987179c1 100644 --- a/modules/web-console/frontend/app/components/page-signin/style.scss +++ b/modules/web-console/frontend/app/components/page-signin/style.scss @@ -15,34 +15,41 @@ * limitations under the License. */ -page-sign-in { +page-signin { display: flex; flex-direction: column; flex: 1 0 auto; - font-family: Roboto; - - h3 { - font-size: 38px; - font-weight: 300; - margin: 30px 0 60px; - } - section { - flex-grow: 1; - padding: 25px 0 60px; + margin-left: auto; + margin-right: auto; + width: 530px; - background-color: #ffffff; - color: #444444; - } + h3 { + font-size: 38px; + font-weight: 300; + margin: 30px 0 30px; + } + + .form-field { + margin: 10px 0; + } - .ps-grid { - display: grid; - grid-gap: 10px; - grid-template-columns: 1fr 1fr; + .form-footer { + padding: 15px 0; + text-align: right; + border-top: 1px solid #e5e5e5; + display: flex; + align-items: center; - .ps-grid-full-width { - grid-column: 1 / 3; + .btn-ignite { + margin-left: auto; + } } } -} \ No newline at end of file + + .page-signin__no-account-message { + text-align: center; + margin: 20px 0; + } +} diff --git a/modules/web-console/frontend/app/components/page-signin/template.pug b/modules/web-console/frontend/app/components/page-signin/template.pug index 58d85715dc0a5..fd5e95fb81b27 100644 --- a/modules/web-console/frontend/app/components/page-signin/template.pug +++ b/modules/web-console/frontend/app/components/page-signin/template.pug @@ -20,164 +20,41 @@ web-console-header web-console-header-left ignite-header-title -section - .container - .row - .col-lg-6.col-md-6.col-sm-6.col-xs-12 - .row - .col-xs-12.col-md-11 - -var form = '$ctrl.form_signup' - h3 Don't Have An Account? - form.ps-grid(name=form novalidate) - .ps-grid-full-width - +form-field__email({ - label: 'Email:', - model: '$ctrl.data.signup.email', - name: '"signupEmail"', - placeholder: 'Input email', - required: true - })( - ignite-on-enter-focus-move='passwordInput' - ng-model-options='{allowInvalid: true}' - ) - +form-field__error({error: 'server', message: `{{$ctrl.serverErrors.signup}}`}) - div - +form-field__password({ - label: 'Password:', - model: '$ctrl.data.signup.password', - name: '"password"', - placeholder: 'Input password', - required: true - })( - ignite-on-enter-focus-move='confirmInput' - ) - div - +form-field__password({ - label: 'Confirm:', - model: 'confirm', - name: '"confirm"', - placeholder: 'Confirm password', - required: true - })( - ignite-on-enter-focus-move='firstNameInput' - ignite-match='$ctrl.data.signup.password' - ) - div - +form-field__text({ - label: 'First name:', - model: '$ctrl.data.signup.firstName', - name: '"firstName"', - placeholder: 'Input first name', - required: true - })( - ignite-on-enter-focus-move='lastNameInput' - ) - div - +form-field__text({ - label: 'Last name:', - model: '$ctrl.data.signup.lastName', - name: '"lastName"', - placeholder: 'Input last name', - required: true - })( - ignite-on-enter-focus-move='companyInput' - ) - .ps-grid-full-width - +form-field__dropdown({ - label: 'Country:', - model: '$ctrl.data.signup.country', - name: '"country"', - required: true, - placeholder: 'Choose your country', - options: '$ctrl.countries' - })( - ignite-on-enter-focus-move='signup_submit' - ) - .ps-grid-full-width - +form-field__text({ - label: 'Company:', - model: '$ctrl.data.signup.company', - name: '"company"', - placeholder: 'Input company name', - required: true - })( - ignite-on-enter-focus-move='countryInput' - ) - .login-footer.ps-grid-full-width - button#signup_submit.btn-ignite.btn-ignite--primary( - ng-click='$ctrl.signup()' - ng-disabled=`!$ctrl.canSubmitForm(${form})` - ) Sign Up - - .col-lg-6.col-md-6.col-sm-6.col-xs-12 - .row - .col-xs-12.col-md-11 - -var form = '$ctrl.form_signin' - form.row(ng-show='$ctrl.isSigninOpened' name=form novalidate) - .settings-row - h3 Sign In - .settings-row - +form-field__email({ - label: 'Email:', - model: '$ctrl.data.signin.email', - name: '"signinEmail"', - placeholder: 'Input email', - required: true - })( - ignite-auto-focus - ignite-on-enter-focus-move='singinPasswordInput' - ng-model-options='{allowInvalid: true}' - ) - +form-field__error({error: 'server', message: `{{$ctrl.serverErrors.signin}}`}) - .settings-row - +form-field__password({ - label: 'Password:', - model: '$ctrl.data.signin.password', - name: '"signinPassword"', - placeholder: 'Input password', - required: true - })( - ignite-on-enter='$ctrl.signin($ctrl.data.signin.email, $ctrl.data.signin.password)' - ng-model-options='{allowInvalid: true}' - ) - +form-field__error({error: 'server', message: `{{$ctrl.serverErrors.signin}}`}) - .login-footer - a.labelField#forgot_show( - ng-click='$ctrl.toggleActiveForm()' - ignite-on-click-focus='forgot_email' - ) Forgot password? - button#signin_submit.btn-ignite.btn-ignite--primary( - ng-click='$ctrl.signin()' - ng-disabled=`!$ctrl.canSubmitForm(${form})` - ) Sign In - - - var form = '$ctrl.form_forgot' - form.row(ng-show='$ctrl.isRemindPasswordOpened' name=form novalidate) - .settings-row - h3 Forgot password? - .settings-row - p.col-xs-12.col-md-11 That's ok! Simply enter your email below and a reset password link will be sent to you via email. You can then follow that link and select a new password. - .settings-row - +form-field__email({ - label: 'Email:', - model: '$ctrl.data.remindPassword.email', - name: '"forgotEmail"', - placeholder: 'Input email', - required: true - })( - ignite-auto-focus - ng-model-options='{allowInvalid: true}' - ignite-on-enter='$ctrl.remindPassword()' - ) - +form-field__error({error: 'server', message: `{{$ctrl.serverErrors.remindPassword}}`}) - .login-footer - a.labelField#forgot_signin( - ng-click='$ctrl.toggleActiveForm()' - ignite-on-click-focus='signin_email' - ) Sign In - button#forgot_submit.btn-ignite.btn-ignite--primary( - ng-click='$ctrl.remindPassword()' - ng-disabled=`!$ctrl.canSubmitForm(${form})` - ) Send it to me +.container.body-container + section + -var form = '$ctrl.form' + h3 Sign In + form(name=form novalidate ng-submit='$ctrl.signin()') + +form-field__email({ + label: 'Email:', + model: '$ctrl.data.email', + name: '"email"', + placeholder: 'Input email', + required: true + })( + ng-model-options='{allowInvalid: true}' + autocomplete='email' + ignite-auto-focus + ) + +form-field__error({error: 'server', message: `{{$ctrl.serverErrors.}`}) + +form-field__password({ + label: 'Password:', + model: '$ctrl.data.password', + name: '"password"', + placeholder: 'Input password', + required: true + })( + ng-model-options='{allowInvalid: true}' + autocomplete='current-password' + ) + +form-field__error({error: 'server', message: `{{$ctrl.serverErrors}}`}) + footer.form-footer + a(ui-sref='forgotPassword({email: $ctrl.data.email})') Forgot password? + button.btn-ignite.btn-ignite--primary( + type='submit' + ng-disabled=`!$ctrl.canSubmitForm(${form})` + ) Sign In + footer.page-signin__no-account-message + | Don't have an account? #[a(ui-sref='signup') Get started] web-console-footer diff --git a/modules/web-console/frontend/app/components/page-signin/types.ts b/modules/web-console/frontend/app/components/page-signin/types.ts new file mode 100644 index 0000000000000..96cca75a5e41d --- /dev/null +++ b/modules/web-console/frontend/app/components/page-signin/types.ts @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +export interface ISiginData { + email: string, + password: string +} + +export interface ISigninFormController extends ng.IFormController { + email: ng.INgModelController, + password: ng.INgModelController +} diff --git a/modules/web-console/frontend/app/components/page-signup/component.js b/modules/web-console/frontend/app/components/page-signup/component.js new file mode 100644 index 0000000000000..789a68187e828 --- /dev/null +++ b/modules/web-console/frontend/app/components/page-signup/component.js @@ -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. + */ + +import angular from 'angular'; + +import template from './template.pug'; +import controller from './controller'; +import './style.scss'; + +/** @type {ng.IComponentOptions} */ +export default { + controller, + template +}; diff --git a/modules/web-console/frontend/app/components/page-signup/controller.js b/modules/web-console/frontend/app/components/page-signup/controller.js new file mode 100644 index 0000000000000..ffc534a72e02c --- /dev/null +++ b/modules/web-console/frontend/app/components/page-signup/controller.js @@ -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. + */ + +export default class PageSignup { + /** @type {import('./types').ISignupFormController} */ + form; + /** @type {import('./types').ISignupData} */ + data = { + email: null, + password: null, + firstName: null, + lastName: null, + company: null, + country: null + }; + /** @type {string} */ + serverError = null; + + static $inject = ['IgniteCountries', 'Auth', 'IgniteMessages']; + + /** + * @param {import('app/modules/user/Auth.service').default} Auth + */ + constructor(Countries, Auth, IgniteMessages) { + this.Auth = Auth; + this.IgniteMessages = IgniteMessages; + this.countries = Countries.getAll(); + } + + /** @param {import('./types').ISignupFormController} form */ + canSubmitForm(form) { + return form.$error.server ? true : !form.$invalid; + } + + $postLink() { + this.form.email.$validators.server = () => !this.serverError; + } + + signup() { + return this.Auth.signnup(this.data).catch((res) => { + this.IgniteMessages.showError(null, res.data); + this.serverError = res.data; + this.form.email.$validate(); + }); + } +} diff --git a/modules/web-console/frontend/app/components/page-signup/index.js b/modules/web-console/frontend/app/components/page-signup/index.js new file mode 100644 index 0000000000000..4efadb5f47af5 --- /dev/null +++ b/modules/web-console/frontend/app/components/page-signup/index.js @@ -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. + */ + +import angular from 'angular'; +import component from './component'; +import {registerState} from './run'; + +export default angular + .module('ignite-console.page-signup', [ + 'ui.router', + 'ignite-console.user' + ]) + .component('pageSignup', component) + .run(registerState); diff --git a/modules/web-console/frontend/app/components/page-signup/run.js b/modules/web-console/frontend/app/components/page-signup/run.js new file mode 100644 index 0000000000000..1d04fa257714f --- /dev/null +++ b/modules/web-console/frontend/app/components/page-signup/run.js @@ -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. + */ + +/** + * @param {import("@uirouter/angularjs").UIRouter} $uiRouter + */ +export function registerState($uiRouter) { + /** @type {import("app/types").IIgniteNg1StateDeclaration} */ + const state = { + name: 'signup', + url: '/signup', + component: 'pageSignup', + unsaved: true, + tfMetaTags: { + title: 'Sign Up' + } + }; + $uiRouter.stateRegistry.register(state); +} + +registerState.$inject = ['$uiRouter']; diff --git a/modules/web-console/frontend/app/components/page-signup/style.scss b/modules/web-console/frontend/app/components/page-signup/style.scss new file mode 100644 index 0000000000000..54ae14e947bc8 --- /dev/null +++ b/modules/web-console/frontend/app/components/page-signup/style.scss @@ -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. + */ + +page-signup { + display: flex; + flex-direction: column; + flex: 1 0 auto; + + section { + h3 { + font-size: 38px; + font-weight: 300; + margin: 30px 0 30px; + } + + margin-left: auto; + margin-right: auto; + width: 530px; + + form footer { + padding: 15px 0; + text-align: right; + border-top: 1px solid #e5e5e5; + display: flex; + align-items: center; + + .btn-ignite { + margin-left: auto; + } + } + + form { + display: grid; + grid-gap: 10px; + grid-template-columns: 1fr 1fr; + + .full-width { + grid-column: 1 / 3; + } + } + } + + .page-signup__has-account-message { + text-align: center; + margin: 20px 0; + } +} diff --git a/modules/web-console/frontend/app/components/page-signup/template.pug b/modules/web-console/frontend/app/components/page-signup/template.pug new file mode 100644 index 0000000000000..8cc7b145302ab --- /dev/null +++ b/modules/web-console/frontend/app/components/page-signup/template.pug @@ -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. + +include /app/helpers/jade/mixins + +web-console-header + web-console-header-left + ignite-header-title + +.container.body-container + section + -var form = '$ctrl.form' + h3 Don't Have An Account? + form(name=form novalidate ng-submit='$ctrl.signup()') + .full-width + +form-field__email({ + label: 'Email:', + model: '$ctrl.data.email', + name: '"email"', + placeholder: 'Input email', + required: true + })( + ng-model-options='{allowInvalid: true}' + autocomplete='email' + ignite-auto-focus + ) + +form-field__error({error: 'server', message: `{{$ctrl.serverError}}`}) + div + +form-field__password({ + label: 'Password:', + model: '$ctrl.data.password', + name: '"password"', + placeholder: 'Input password', + required: true + })( + autocomplete='new-password' + ) + div + +form-field__password({ + label: 'Confirm:', + model: 'confirm', + name: '"confirm"', + placeholder: 'Confirm password', + required: true + })( + ignite-match='$ctrl.data.password' + autocomplete='off' + ) + div + +form-field__text({ + label: 'First name:', + model: '$ctrl.data.firstName', + name: '"firstName"', + placeholder: 'Input first name', + required: true + })( + autocomplete='given-name' + ) + div + +form-field__text({ + label: 'Last name:', + model: '$ctrl.data.lastName', + name: '"lastName"', + placeholder: 'Input last name', + required: true + })( + autocomplete='family-name' + ) + div + +form-field__phone({ + label: 'Phone:', + model: '$ctrl.data.phone', + name: '"phone"', + placeholder: 'Input phone (ex.: +15417543010)', + optional: true + })( + autocomplete='tel' + ) + div + +form-field__dropdown({ + label: 'Country:', + model: '$ctrl.data.country', + name: '"country"', + required: true, + placeholder: 'Choose your country', + options: '$ctrl.countries' + })( + autocomplete='country' + ) + .full-width + +form-field__text({ + label: 'Company:', + model: '$ctrl.data.company', + name: '"company"', + placeholder: 'Input company name', + required: true + })( + ignite-on-enter-focus-move='countryInput' + autocomplete='organization' + ) + footer.full-width.form-footer + button.btn-ignite.btn-ignite--primary( + type='submit' + ng-disabled=`!$ctrl.canSubmitForm(${form})` + ) Sign Up + footer.page-signup__has-account-message + | Already have an account? #[a(ui-sref='signin') Sign in here] + +web-console-footer diff --git a/modules/web-console/frontend/app/components/page-signup/types.ts b/modules/web-console/frontend/app/components/page-signup/types.ts new file mode 100644 index 0000000000000..49e8d5d15072b --- /dev/null +++ b/modules/web-console/frontend/app/components/page-signup/types.ts @@ -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. + */ + +export interface ISignupData { + email: string, + password: string, + firstName: string, + lastName: string, + phone?: string, + company: string, + country: string +} + +export interface ISignupFormController extends ng.IFormController { + email: ng.INgModelController, + password: ng.INgModelController, + firstName: ng.INgModelController, + lastName: ng.INgModelController, + phone: ng.INgModelController, + company: ng.INgModelController, + country: ng.INgModelController +} diff --git a/modules/web-console/frontend/app/primitives/form-field/dropdown.pug b/modules/web-console/frontend/app/primitives/form-field/dropdown.pug index cb058295c0077..c41825cef1413 100644 --- a/modules/web-console/frontend/app/primitives/form-field/dropdown.pug +++ b/modules/web-console/frontend/app/primitives/form-field/dropdown.pug @@ -17,6 +17,7 @@ mixin form-field__dropdown({ label, model, name, disabled, required, multiple, placeholder, placeholderEmpty, options, tip }) mixin __form-field__input() button.select-toggle( + type='button' id=`{{ ${name} }}Input` name=`{{ ${name} }}` diff --git a/modules/web-console/frontend/app/types/index.ts b/modules/web-console/frontend/app/types/index.ts new file mode 100644 index 0000000000000..333c00be875d2 --- /dev/null +++ b/modules/web-console/frontend/app/types/index.ts @@ -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. + */ + +import {Ng1StateDeclaration} from '@uirouter/angularjs'; + +interface ITfMetatagsConfig { + title: string +} + +export interface IIgniteNg1StateDeclaration extends Ng1StateDeclaration { + /** + * Whether to store state as last visited in local storage or not. + * true - will be saved + * false (default) - won't be saved + * @type {boolean} + */ + unsaved?: boolean, + tfMetaTags: ITfMetatagsConfig +} diff --git a/modules/web-console/frontend/public/stylesheets/style.scss b/modules/web-console/frontend/public/stylesheets/style.scss index 0c978e9ca43aa..a9702e38a8a24 100644 --- a/modules/web-console/frontend/public/stylesheets/style.scss +++ b/modules/web-console/frontend/public/stylesheets/style.scss @@ -241,29 +241,6 @@ h1, h2, h3, h4, h5, h6 { font-size: 2em; } -.login-footer { - padding: 15px 0; - text-align: right; - border-top: 1px solid #e5e5e5; - - label { - float: left; - margin: 0; - } - - .btn:last-child { - margin-right: 0; - } - - .checkbox { - margin: 0; - } - - .btn { - margin-right: 0; - } -} - body { overflow-x: hidden; display: flex; From 3d802681e418abe6d7df6fa463d4fdde3e8f0a15 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Wed, 25 Apr 2018 16:14:17 +0700 Subject: [PATCH 0162/1463] IGNITE-8370 Rename file. (cherry picked from commit 41699c3) --- .../web-console/e2e/testcafe/fixtures/auth/forgot-password.js | 2 +- modules/web-console/e2e/testcafe/fixtures/auth/logout.js | 2 +- .../e2e/testcafe/page-models/{pageSignin1.js => pageSignin.js} | 0 modules/web-console/e2e/testcafe/roles.js | 2 +- 4 files changed, 3 insertions(+), 3 deletions(-) rename modules/web-console/e2e/testcafe/page-models/{pageSignin1.js => pageSignin.js} (100%) diff --git a/modules/web-console/e2e/testcafe/fixtures/auth/forgot-password.js b/modules/web-console/e2e/testcafe/fixtures/auth/forgot-password.js index 27ac2db155345..1163595d70464 100644 --- a/modules/web-console/e2e/testcafe/fixtures/auth/forgot-password.js +++ b/modules/web-console/e2e/testcafe/fixtures/auth/forgot-password.js @@ -16,7 +16,7 @@ */ import { dropTestDB, resolveUrl, insertTestUser } from '../../envtools'; -import {PageSignIn} from '../../page-models/pageSignin1'; +import {PageSignIn} from '../../page-models/pageSignin'; import {errorNotification} from '../../components/notifications'; import {pageForgotPassword as page} from '../../page-models/pageForgotPassword'; diff --git a/modules/web-console/e2e/testcafe/fixtures/auth/logout.js b/modules/web-console/e2e/testcafe/fixtures/auth/logout.js index 5cad60fe17e32..d2e7e661d8fc7 100644 --- a/modules/web-console/e2e/testcafe/fixtures/auth/logout.js +++ b/modules/web-console/e2e/testcafe/fixtures/auth/logout.js @@ -18,7 +18,7 @@ import {dropTestDB, resolveUrl, insertTestUser} from 'envtools'; import {createRegularUser} from '../../roles'; import {userMenu} from '../../components/userMenu'; -import {pageSignin} from '../../page-models/pageSignin1'; +import {pageSignin} from '../../page-models/pageSignin'; const user = createRegularUser(); diff --git a/modules/web-console/e2e/testcafe/page-models/pageSignin1.js b/modules/web-console/e2e/testcafe/page-models/pageSignin.js similarity index 100% rename from modules/web-console/e2e/testcafe/page-models/pageSignin1.js rename to modules/web-console/e2e/testcafe/page-models/pageSignin.js diff --git a/modules/web-console/e2e/testcafe/roles.js b/modules/web-console/e2e/testcafe/roles.js index 5d9bfcc434c1f..3089e97e12236 100644 --- a/modules/web-console/e2e/testcafe/roles.js +++ b/modules/web-console/e2e/testcafe/roles.js @@ -17,7 +17,7 @@ const { Role, t } = require('testcafe'); import { resolveUrl } from './envtools'; -import {pageSignin as page} from './page-models/pageSignin1'; +import {pageSignin as page} from './page-models/pageSignin'; export const createRegularUser = () => { return new Role(resolveUrl('/signin'), async() => { From 9cea447539ca532a095fce1f074094cad7d6cb19 Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Thu, 26 Apr 2018 09:05:52 +0700 Subject: [PATCH 0163/1463] Minor fix for tests. (cherry picked from commit b3f912f) --- .../frontend/app/directives/match.directive.spec.js | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/modules/web-console/frontend/app/directives/match.directive.spec.js b/modules/web-console/frontend/app/directives/match.directive.spec.js index 2a00b03d0bb89..c274585ff6851 100644 --- a/modules/web-console/frontend/app/directives/match.directive.spec.js +++ b/modules/web-console/frontend/app/directives/match.directive.spec.js @@ -56,7 +56,10 @@ suite('ignite-match', () => { $compile(el)($scope); $scope.$digest(); - const [master, , slave] = el; + // const [master, , slave] = el; + // For some reason, this code not work after Babel, replaced with 'old' syntax. + const master = el[0]; + const slave = el[2]; setValue(slave, '123'); $scope.$digest(); From 6019004d26c502a908bd2821febbce784dd9746f Mon Sep 17 00:00:00 2001 From: Alexander Kalinin Date: Thu, 26 Apr 2018 15:37:07 +0700 Subject: [PATCH 0164/1463] IGNITE-8235 Web Console: Implement execution of selected part of SQL query. (cherry picked from commit 9d61a42) --- .../components/queries-notebook/controller.js | 13 ++++++++----- .../queries-notebook/template.tpl.pug | 2 +- .../frontend/app/modules/ace.module.js | 18 ++++++++++++++++-- 3 files changed, 25 insertions(+), 8 deletions(-) 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 fa7460f98859d..859d1f737db52 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 @@ -1477,8 +1477,9 @@ export class NotebookCtrl { $scope.queryAvailable(paragraph) && _chooseNode(paragraph.cacheName, local) .then((nid) => { - Notebook.save($scope.notebook) - .catch(Messages.showError); + // If we are executing only selected part of query then Notebook shouldn't be saved. + if (!paragraph.partialQuery) + Notebook.save($scope.notebook).catch(Messages.showError); paragraph.localQueryMode = local; paragraph.prevQuery = paragraph.queryArgs ? paragraph.queryArgs.query : paragraph.query; @@ -1487,10 +1488,12 @@ export class NotebookCtrl { return _closeOldQuery(paragraph) .then(() => { + const query = paragraph.partialQuery || paragraph.query; + const args = paragraph.queryArgs = { type: 'QUERY', cacheName: $scope.cacheNameForSql(paragraph), - query: paragraph.query, + query, pageSize: paragraph.pageSize, maxPages: paragraph.maxPages, nonCollocatedJoins, @@ -1499,10 +1502,10 @@ export class NotebookCtrl { lazy }; - const qry = args.maxPages ? addLimit(args.query, args.pageSize * args.maxPages) : paragraph.query; - ActivitiesData.post({ action: '/queries/execute' }); + const qry = args.maxPages ? addLimit(args.query, args.pageSize * args.maxPages) : query; + return agentMgr.querySql(nid, args.cacheName, qry, nonCollocatedJoins, enforceJoinOrder, false, local, args.pageSize, lazy); }) .then((res) => { diff --git a/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/template.tpl.pug b/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/template.tpl.pug index a163125be0ce7..96c704d8d731d 100644 --- a/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/template.tpl.pug +++ b/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/template.tpl.pug @@ -309,7 +309,7 @@ mixin paragraph-query .col-xs-8.col-sm-9(style='border-right: 1px solid #eee') .sql-editor(ignite-ace='{onLoad: aceInit(paragraph), theme: "chrome", mode: "sql", require: ["ace/ext/language_tools"],' + 'advanced: {enableSnippets: false, enableBasicAutocompletion: true, enableLiveAutocompletion: true}}' - ng-model='paragraph.query') + ng-model='paragraph.query' on-selection-change='paragraph.partialQuery = $event') .col-xs-4.col-sm-3 div(ng-show='caches.length > 0' style='padding: 5px 10px' st-table='displayedCaches' st-safe-src='caches') lable.labelField.labelFormField Caches: diff --git a/modules/web-console/frontend/app/modules/ace.module.js b/modules/web-console/frontend/app/modules/ace.module.js index 6a6e70a45f83e..44e51caf0a015 100644 --- a/modules/web-console/frontend/app/modules/ace.module.js +++ b/modules/web-console/frontend/app/modules/ace.module.js @@ -134,8 +134,12 @@ angular return { restrict: 'EA', - require: ['?ngModel', '?^form'], - link: (scope, elm, attrs, [ngModel, form]) => { + require: ['?ngModel', '?^form', 'igniteAce'], + bindToController: { + onSelectionChange: '&?' + }, + controller() {}, + link: (scope, elm, attrs, [ngModel, form, igniteAce]) => { /** * Corresponds the igniteAceConfig ACE configuration. * @@ -165,6 +169,8 @@ angular */ const session = acee.getSession(); + const selection = session.getSelection(); + /** * Reference to a change listener created by the listener factory. * @@ -223,6 +229,14 @@ angular ngModel.$render = () => session.setValue(ngModel.$viewValue); acee.on('change', () => ngModel.$setViewValue(acee.getValue())); + + selection.on('changeSelection', () => { + if (igniteAce.onSelectionChange) { + const aceSelection = selection.isEmpty() ? null : acee.session.getTextRange(acee.getSelectionRange()); + + igniteAce.onSelectionChange({$event: aceSelection}); + } + }); } // Listen for option updates. From effb310676b992f99f719af9b126087271b3c051 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Thu, 26 Apr 2018 15:52:57 +0700 Subject: [PATCH 0165/1463] IGNITE-8214 Web Console: Added validation on "Configuration" screen that swap can be set only if persistence is off. (cherry picked from commit d31aa12) --- .../templates/data-storage.pug | 122 +++++++++--------- .../generator/ConfigurationGenerator.js | 6 +- 2 files changed, 65 insertions(+), 63 deletions(-) diff --git a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/data-storage.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/data-storage.pug index c9fea569a4f5b..4ed30b6f7a406 100644 --- a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/data-storage.pug +++ b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/data-storage.pug @@ -59,25 +59,25 @@ mixin data-region-form({modelAt, namePlaceholder, dataRegionsAt}) min=`{{ $ctrl.Clusters.dataRegion.maxSize.min(${modelAt}) }}` ) - .pc-form-grid-col-60 - +text('Swap file path:', `${modelAt}.swapFilePath`, '"swapFilePath"', 'false', 'Input swap file path', 'An optional path to a memory mapped file for this data region') + .pc-form-grid-col-60(ng-if=`!${modelAt}.persistenceEnabled || ${modelAt}.swapPath`) + +text('Swap file path:', `${modelAt}.swapPath`, '"swapPath"', 'false', 'Input swap file path', 'An optional path to a memory mapped file for this data region') .pc-form-grid-col-60 +number('Checkpoint page buffer:', `${modelAt}.checkpointPageBufferSize`, '"checkpointPageBufferSize"', 'true', '0', '0', 'Amount of memory allocated for a checkpoint temporary buffer in bytes') .pc-form-grid-col-60 +dropdown('Eviction mode:', `${modelAt}.pageEvictionMode`, '"pageEvictionMode"', 'true', 'DISABLED', - '[\ - {value: "DISABLED", label: "DISABLED"},\ - {value: "RANDOM_LRU", label: "RANDOM_LRU"},\ - {value: "RANDOM_2_LRU", label: "RANDOM_2_LRU"}\ - ]', - `An algorithm for memory pages eviction -
      -
    • DISABLED - Eviction is disabled
    • -
    • RANDOM_LRU - Once a memory region defined by a data region is configured, an off-heap array is allocated to track last usage timestamp for every individual data page
    • -
    • RANDOM_2_LRU - Differs from Random - LRU only in a way that two latest access timestamps are stored for every data page
    • -
    `) + '[\ + {value: "DISABLED", label: "DISABLED"},\ + {value: "RANDOM_LRU", label: "RANDOM_LRU"},\ + {value: "RANDOM_2_LRU", label: "RANDOM_2_LRU"}\ + ]', + `An algorithm for memory pages eviction +
      +
    • DISABLED - Eviction is disabled
    • +
    • RANDOM_LRU - Once a memory region defined by a data region is configured, an off-heap array is allocated to track last usage timestamp for every individual data page
    • +
    • RANDOM_2_LRU - Differs from Random - LRU only in a way that two latest access timestamps are stored for every data page
    • +
    `) .pc-form-grid-col-30 +sane-ignite-form-field-number({ @@ -129,11 +129,11 @@ mixin data-region-form({modelAt, namePlaceholder, dataRegionsAt}) .pc-form-grid-col-60 +checkbox('Metrics enabled', `${modelAt}.metricsEnabled`, '"MemoryPolicyMetricsEnabled"', - 'Whether memory metrics are enabled by default on node startup') + 'Whether memory metrics are enabled by default on node startup') - .pc-form-grid-col-60 + .pc-form-grid-col-60(ng-if=`!${modelAt}.swapPath`) +checkbox('Persistence enabled', `${modelAt}.persistenceEnabled`, '"RegionPersistenceEnabled" + $index', - 'Enable Ignite Native Persistence') + 'Enable Ignite Native Persistence') panel-collapsible(ng-show='$ctrl.available("2.3.0")' ng-form=form on-open=`ui.loadPanel('${form}')`) panel-title Data storage configuration @@ -152,7 +152,7 @@ panel-collapsible(ng-show='$ctrl.available("2.3.0")' ng-form=form on-open=`ui.lo }) .pc-form-grid-col-30 +number('Concurrency level:', model + '.concurrencyLevel', '"DataStorageConfigurationConcurrencyLevel"', - 'true', 'availableProcessors', '2', 'The number of concurrent segments in Ignite internal page mapping tables') + 'true', 'availableProcessors', '2', 'The number of concurrent segments in Ignite internal page mapping tables') .pc-form-grid-col-60.pc-form-group__text-title span System region .pc-form-group.pc-form-grid-row @@ -206,38 +206,38 @@ panel-collapsible(ng-show='$ctrl.available("2.3.0")' ng-form=form on-open=`ui.lo .pc-form-grid-col-60 +text-enabled('Storage path:', `${model}.storagePath`, '"DataStoragePath"', 'true', 'false', 'db', - 'Directory where index and partition files are stored') + 'Directory where index and partition files are stored') .pc-form-grid-col-60 +number('Checkpoint frequency:', `${model}.checkpointFrequency`, '"DataStorageCheckpointFrequency"', 'true', '180000', '1', - 'Frequency which is a minimal interval when the dirty pages will be written to the Persistent Store') + 'Frequency which is a minimal interval when the dirty pages will be written to the Persistent Store') .pc-form-grid-col-20 +number('Checkpoint threads:', `${model}.checkpointThreads`, '"DataStorageCheckpointThreads"', 'true', '4', '1', 'A number of threads to use for the checkpoint purposes') .pc-form-grid-col-20 +dropdown('Checkpoint write order:', `${model}.checkpointWriteOrder`, '"DataStorageCheckpointWriteOrder"', 'true', 'SEQUENTIAL', - '[\ - {value: "RANDOM", label: "RANDOM"},\ - {value: "SEQUENTIAL", label: "SEQUENTIAL"}\ - ]', - 'Order of writing pages to disk storage during checkpoint.\ -
      \ -
    • RANDOM - Pages are written in order provided by checkpoint pages collection iterator
    • \ -
    • SEQUENTIAL - All checkpoint pages are collected into single list and sorted by page index
    • \ -
    ') + '[\ + {value: "RANDOM", label: "RANDOM"},\ + {value: "SEQUENTIAL", label: "SEQUENTIAL"}\ + ]', + 'Order of writing pages to disk storage during checkpoint.\ +
      \ +
    • RANDOM - Pages are written in order provided by checkpoint pages collection iterator
    • \ +
    • SEQUENTIAL - All checkpoint pages are collected into single list and sorted by page index
    • \ +
    ') .pc-form-grid-col-20 +dropdown('WAL mode:', `${model}.walMode`, '"DataStorageWalMode"', 'true', 'DEFAULT', - '[\ - {value: "DEFAULT", label: "DEFAULT"},\ - {value: "LOG_ONLY", label: "LOG_ONLY"},\ - {value: "BACKGROUND", label: "BACKGROUND"},\ - {value: "NONE", label: "NONE"}\ - ]', - 'Type define behavior wal fsync.\ -
      \ -
    • DEFAULT - full-sync disk writes
    • \ -
    • LOG_ONLY - flushes application buffers
    • \ -
    • BACKGROUND - does not force application's buffer flush
    • \ -
    • NONE - WAL is disabled
    • \ -
    ') + '[\ + {value: "DEFAULT", label: "DEFAULT"},\ + {value: "LOG_ONLY", label: "LOG_ONLY"},\ + {value: "BACKGROUND", label: "BACKGROUND"},\ + {value: "NONE", label: "NONE"}\ + ]', + 'Type define behavior wal fsync.\ +
      \ +
    • DEFAULT - full-sync disk writes
    • \ +
    • LOG_ONLY - flushes application buffers
    • \ +
    • BACKGROUND - does not force application's buffer flush
    • \ +
    • NONE - WAL is disabled
    • \ +
    ') .pc-form-grid-col-60 +text-enabled('WAL path:', `${model}.walPath`, '"DataStorageWalPath"', 'true', 'false', 'db/wal', 'A path to the directory where WAL is stored') .pc-form-grid-col-60 @@ -250,52 +250,52 @@ panel-collapsible(ng-show='$ctrl.available("2.3.0")' ng-form=form on-open=`ui.lo +number('WAL history size:', `${model}.walHistorySize`, '"DataStorageWalHistorySize"', 'true', '20', '1', 'A total number of checkpoints to keep in the WAL history') .pc-form-grid-col-60(ng-if='$ctrl.available("2.4.0")') +number('WAL buffer size:', `${model}.walBufferSize`, '"DataStorageWalBufferSize"', 'true', 'WAL segment size / 4', '1', - 'Size of WAL buffer') + 'Size of WAL buffer') .pc-form-grid-col-30 +number('WAL flush frequency:', `${model}.walFlushFrequency`, '"DataStorageWalFlushFrequency"', 'true', '2000', '1', - 'How often will be fsync, in milliseconds. In background mode, exist thread which do fsync by timeout') + 'How often will be fsync, in milliseconds. In background mode, exist thread which do fsync by timeout') .pc-form-grid-col-30 +number('WAL fsync delay:', `${model}.walFsyncDelayNanos`, '"DataStorageWalFsyncDelay"', 'true', '1000', '1', 'WAL fsync delay, in nanoseconds') .pc-form-grid-col-60 +number('WAL record iterator buffer size:', `${model}.walRecordIteratorBufferSize`, '"DataStorageWalRecordIteratorBufferSize"', 'true', '67108864', '1', - 'How many bytes iterator read from disk(for one reading), during go ahead WAL') + 'How many bytes iterator read from disk(for one reading), during go ahead WAL') .pc-form-grid-col-30 +number('Lock wait time:', `${model}.lockWaitTime`, '"DataStorageLockWaitTime"', 'true', '10000', '1', - 'Time out in milliseconds, while wait and try get file lock for start persist manager') + 'Time out in milliseconds, while wait and try get file lock for start persist manager') .pc-form-grid-col-30 +number('WAL thread local buffer size:', `${model}.walThreadLocalBufferSize`, '"DataStorageWalThreadLocalBufferSize"', 'true', '131072', '1', - 'Define size thread local buffer. Each thread which write to WAL have thread local buffer for serialize recode before write in WAL') + 'Define size thread local buffer. Each thread which write to WAL have thread local buffer for serialize recode before write in WAL') .pc-form-grid-col-30 +number('Metrics sub interval count:', `${model}.metricsSubIntervalCount`, '"DataStorageMetricsSubIntervalCount"', 'true', '5', '1', - 'Number of sub - intervals the whole rate time interval will be split into to calculate rate - based metrics') + 'Number of sub - intervals the whole rate time interval will be split into to calculate rate - based metrics') .pc-form-grid-col-30 +number('Metrics rate time interval:', `${model}.metricsRateTimeInterval`, '"DataStorageMetricsRateTimeInterval"', 'true', '60000', '1000', - 'The length of the time interval for rate - based metrics. This interval defines a window over which hits will be tracked') + 'The length of the time interval for rate - based metrics. This interval defines a window over which hits will be tracked') .pc-form-grid-col-30 +dropdown('File IO factory:', `${model}.fileIOFactory`, '"DataStorageFileIOFactory"', 'true', 'Default', - '[\ - {value: "RANDOM", label: "RANDOM"},\ - {value: "ASYNC", label: "ASYNC"},\ - {value: null, label: "Default"},\ - ]', - 'Order of writing pages to disk storage during checkpoint.\ -
      \ -
    • RANDOM - Pages are written in order provided by checkpoint pages collection iterator
    • \ -
    • SEQUENTIAL - All checkpoint pages are collected into single list and sorted by page index
    • \ -
    ') + '[\ + {value: "RANDOM", label: "RANDOM"},\ + {value: "ASYNC", label: "ASYNC"},\ + {value: null, label: "Default"},\ + ]', + 'Order of writing pages to disk storage during checkpoint.\ +
      \ +
    • RANDOM - Pages are written in order provided by checkpoint pages collection iterator
    • \ +
    • SEQUENTIAL - All checkpoint pages are collected into single list and sorted by page index
    • \ +
    ') .pc-form-grid-col-30 +number('WAL auto archive after inactivity:', `${model}.walAutoArchiveAfterInactivity`, '"DataStorageWalAutoArchiveAfterInactivity"', 'true', '-1', '-1', - 'Time in millis to run auto archiving segment after last record logging') + 'Time in millis to run auto archiving segment after last record logging') .pc-form-grid-col-60 +checkbox-enabled('Metrics enabled', `${model}.metricsEnabled`, '"DataStorageMetricsEnabled"', 'true', 'Flag indicating whether persistence metrics collection is enabled') .pc-form-grid-col-60 +checkbox-enabled('Always write full pages', `${model}.alwaysWriteFullPages`, '"DataStorageAlwaysWriteFullPages"', 'true', 'Flag indicating whether always write full pages') .pc-form-grid-col-60 +checkbox('Write throttling enabled', `${model}.writeThrottlingEnabled`, '"DataStorageWriteThrottlingEnabled"', - 'Throttle threads that generate dirty pages too fast during ongoing checkpoint') + 'Throttle threads that generate dirty pages too fast during ongoing checkpoint') .pc-form-grid-col-60(ng-if='$ctrl.available("2.4.0")') +checkbox('Enable WAL compaction', `${model}.walCompactionEnabled`, '"DataStorageWalCompactionEnabled"', - 'If true, system filters and compresses WAL archive in background') + 'If true, system filters and compresses WAL archive in background') .pca-form-column-6 +preview-xml-java(model, 'clusterDataStorageConfiguration') 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 c5f82d3b80841..95b7ce410d7bc 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js @@ -1423,8 +1423,10 @@ export default class IgniteConfigurationGenerator { .intProperty('metricsSubIntervalCount') .longProperty('metricsRateTimeInterval') .longProperty('checkpointPageBufferSize') - .boolProperty('metricsEnabled') - .boolProperty('persistenceEnabled'); + .boolProperty('metricsEnabled'); + + if (!plcBean.valueOf('swapPath')) + plcBean.boolProperty('persistenceEnabled'); return plcBean; } From c5252158cd8fedd2abe22bb6fba9febec6674c87 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Thu, 26 Apr 2018 22:10:54 +0700 Subject: [PATCH 0166/1463] IGNITE-8215 Web Console: Fixed missing "metricsSubIntervalCount" and "metricsRateTimeInterval" on "Configuration" screen. (cherry picked from commit 9f855fb) --- .../templates/data-storage.pug | 20 +++++++++---------- .../cluster-edit-form/templates/discovery.pug | 2 +- .../frontend/app/services/Clusters.js | 4 ++-- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/data-storage.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/data-storage.pug index 4ed30b6f7a406..06a5a4a64644a 100644 --- a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/data-storage.pug +++ b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/data-storage.pug @@ -105,25 +105,25 @@ mixin data-region-form({modelAt, namePlaceholder, dataRegionsAt}) .pc-form-grid-col-30 +sane-ignite-form-field-number({ label: 'Metrics sub interval count:', - model: `${modelAt}.subIntervals`, - name: '"subIntervals"', - placeholder: '{{ ::$ctrl.Clusters.dataRegion.subIntervals.default }}', - min: '{{ ::$ctrl.Clusters.dataRegion.subIntervals.min }}', - step: '{{ ::$ctrl.Clusters.dataRegion.subIntervals.step }}', + model: `${modelAt}.metricsSubIntervalCount`, + name: '"metricsSubIntervalCount"', + placeholder: '{{ ::$ctrl.Clusters.dataRegion.metricsSubIntervalCount.default }}', + min: '{{ ::$ctrl.Clusters.dataRegion.metricsSubIntervalCount.min }}', + step: '{{ ::$ctrl.Clusters.dataRegion.metricsSubIntervalCount.step }}', tip: 'A number of sub-intervals the whole rate time interval will be split into to calculate allocation and eviction rates' }) .pc-form-grid-col-30 pc-form-field-size( - ng-model=`${modelAt}.rateTimeInterval` + ng-model=`${modelAt}.metricsRateTimeInterval` ng-model-options='{allowInvalid: true}' - name='rateTimeInterval' + name='metricsRateTimeInterval' size-type='seconds' label='Metrics rate time interval:' - placeholder='{{ $ctrl.Clusters.dataRegion.rateTimeInterval.default / _rateTimeIntervalScale.value }}' - min=`{{ ::$ctrl.Clusters.dataRegion.rateTimeInterval.min }}` + placeholder='{{ $ctrl.Clusters.dataRegion.metricsRateTimeInterval.default / _metricsRateTimeIntervalScale.value }}' + min=`{{ ::$ctrl.Clusters.dataRegion.metricsRateTimeInterval.min }}` tip='Time interval for allocation rate and eviction rate monitoring purposes' - on-scale-change='_rateTimeIntervalScale = $event' + on-scale-change='_metricsRateTimeIntervalScale = $event' size-scale-label='s' ) diff --git a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/discovery.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/discovery.pug index 35bd5e81d5254..6008df31890c6 100644 --- a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/discovery.pug +++ b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/discovery.pug @@ -63,7 +63,7 @@ panel-collapsible(ng-form=form on-open=`ui.loadPanel('${form}')`) +number('Thread priority:', `${model}.threadPriority`, '"threadPriority"', 'true', '10', '1', 'Thread priority for all threads started by SPI') //- Removed in ignite 2.0 - .pc-form-grid-col-60(ng-if='$ctrl.available(["1.0.0", "2.0.0"])') + .pc-form-grid-col-60(ng-if-start='$ctrl.available(["1.0.0", "2.0.0"])') +number('Heartbeat frequency:', `${model}.heartbeatFrequency`, '"heartbeatFrequency"', 'true', '2000', '1', 'Heartbeat messages issuing frequency') .pc-form-grid-col-30 +number('Max heartbeats miss w/o init:', `${model}.maxMissedHeartbeats`, '"maxMissedHeartbeats"', 'true', '1', '1', diff --git a/modules/web-console/frontend/app/services/Clusters.js b/modules/web-console/frontend/app/services/Clusters.js index 4e057fcf8652a..d76f8628ce116 100644 --- a/modules/web-console/frontend/app/services/Clusters.js +++ b/modules/web-console/frontend/app/services/Clusters.js @@ -275,12 +275,12 @@ export default class Clusters { return maxPoolSize; } }, - subIntervals: { + metricsSubIntervalCount: { default: 5, min: 1, step: 1 }, - rateTimeInterval: { + metricsRateTimeInterval: { min: 1000, default: 60000, step: 1000 From cd842b4b18af02b456e6c4b133b0a4f845757188 Mon Sep 17 00:00:00 2001 From: Ilya Borisov Date: Thu, 26 Apr 2018 23:57:06 +0700 Subject: [PATCH 0167/1463] IGNITE-8267 Web console: Adjusted UI for connector configuration fields. (cherry picked from commit 56ba0a5) --- .../templates/client-connector.pug | 10 +++--- .../cluster-edit-form/templates/igfs.pug | 34 ------------------- .../cluster-edit-form/templates/odbc.pug | 14 ++++---- .../templates/sql-connector.pug | 10 +++--- 4 files changed, 17 insertions(+), 51 deletions(-) delete mode 100644 modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/igfs.pug diff --git a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/client-connector.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/client-connector.pug index d27fa14d6aacf..620137b9d923c 100644 --- a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/client-connector.pug +++ b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/client-connector.pug @@ -29,17 +29,17 @@ panel-collapsible(ng-show='$ctrl.available("2.3.0")' ng-form=form on-open=`ui.lo .pca-form-column-6.pc-form-grid-row .pc-form-grid-col-60 +checkbox('Enabled', connectionEnabled, '"ClientConnectorEnabled"', 'Flag indicating whether to configure client connector configuration') - .pc-form-grid-col-40 + .pc-form-grid-col-60 +text-enabled('Host:', `${connectionModel}.host`, '"ClientConnectorHost"', connectionEnabled, 'false', 'localhost') - .pc-form-grid-col-20 + .pc-form-grid-col-30 +number('Port:', `${connectionModel}.port`, '"ClientConnectorPort"', connectionEnabled, '10800', '1025') - .pc-form-grid-col-20 + .pc-form-grid-col-30 +number('Port range:', `${connectionModel}.portRange`, '"ClientConnectorPortRange"', connectionEnabled, '100', '0') - .pc-form-grid-col-20 + .pc-form-grid-col-30 +number('Socket send buffer size:', `${connectionModel}.socketSendBufferSize`, '"ClientConnectorSocketSendBufferSize"', connectionEnabled, '0', '0', 'Socket send buffer size
    \ When set to 0, operation system default will be used') - .pc-form-grid-col-20 + .pc-form-grid-col-30 +number('Socket receive buffer size:', `${connectionModel}.socketReceiveBufferSize`, '"ClientConnectorSocketReceiveBufferSize"', connectionEnabled, '0', '0', 'Socket receive buffer size
    \ When set to 0, operation system default will be used') diff --git a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/igfs.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/igfs.pug deleted file mode 100644 index c1216a27e7f1b..0000000000000 --- a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/igfs.pug +++ /dev/null @@ -1,34 +0,0 @@ -//- - Licensed to the Apache Software Foundation (ASF) under one or more - contributor license agreements. See the NOTICE file distributed with - this work for additional information regarding copyright ownership. - The ASF licenses this file to You under the Apache License, Version 2.0 - (the "License"); you may not use this file except in compliance with - the License. You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. - -include /app/helpers/jade/mixins - --var form = 'igfs' --var model = '$ctrl.clonedCluster' - -panel-collapsible(ng-form=form on-open=`ui.loadPanel('${form}')`) - panel-title IGFS - panel-description - | IGFS (Ignite In-Memory File System) configurations assigned to cluster. - | #[a.link-success(href="https://apacheignite-fs.readme.io/docs/in-memory-file-system" target="_blank") More info] - panel-content.pca-form-row(ng-if=`ui.isPanelLoaded('${form}')`) - .pca-form-column-6 - .settings-row - +dropdown-multiple('IGFS: (add)', - `${model}.igfss`, '"igfss"', true, 'Choose IGFS', 'No IGFS configured', 'igfss', - 'Select IGFS to start in cluster or add a new IGFS') - .pca-form-column-6 - +preview-xml-java(model, 'igfss', 'igfss') diff --git a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/odbc.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/odbc.pug index 74b1f02c316c2..481a9aa984c1d 100644 --- a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/odbc.pug +++ b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/odbc.pug @@ -30,8 +30,8 @@ panel-collapsible( | ODBC server configuration. | #[a.link-success(href="https://apacheignite.readme.io/docs/odbc-driver" target="_blank") More info] panel-content.pca-form-row(ng-if=`$ctrl.available(["1.0.0", "2.1.0"]) && ui.isPanelLoaded('${form}')`) - .pca-form-column-6 - .settings-row + .pca-form-column-6.pc-form-grid-row + .pc-form-grid-col-60 +sane-form-field-checkbox({ label: 'Enabled', model: enabled, @@ -44,7 +44,7 @@ panel-collapsible( ui-validate-watch='$ctrl.Clusters.odbc.odbcEnabled.correctMarshallerWatch("$ctrl.clonedCluster")' ) +form-field-feedback(null, 'correctMarshaller', 'ODBC can only be used with BinaryMarshaller') - .settings-row + .pc-form-grid-col-60 +text-ip-address-with-port-range('ODBC endpoint address:', `${model}.endpointAddress`, '"endpointAddress"', enabled, '0.0.0.0:10800..10810', 'ODBC endpoint address.
    \ The following address formats are permitted:\ @@ -53,17 +53,17 @@ panel-collapsible(
  • hostname:port - will use provided hostname and port
  • \
  • hostname:port_from..port_to - will use provided hostname and port range
  • \ ') - .settings-row + .pc-form-grid-col-30 +number('Send buffer size:', `${model}.socketSendBufferSize`, '"ODBCSocketSendBufferSize"', enabled, '0', '0', 'Socket send buffer size.
    \ When set to 0, operation system default will be used') - .settings-row + .pc-form-grid-col-30 +number('Socket receive buffer size:', `${model}.socketReceiveBufferSize`, '"ODBCSocketReceiveBufferSize"', enabled, '0', '0', 'Socket receive buffer size.
    \ When set to 0, operation system default will be used') - .settings-row + .pc-form-grid-col-30 +number('Maximum open cursors', `${model}.maxOpenCursors`, '"maxOpenCursors"', enabled, '128', '1', 'Maximum number of opened cursors per connection') - .settings-row + .pc-form-grid-col-30 +number('Pool size:', `${model}.threadPoolSize`, '"ODBCThreadPoolSize"', enabled, 'max(8, availableProcessors)', '1', 'Size of thread pool that is in charge of processing ODBC tasks') .pca-form-column-6 diff --git a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/sql-connector.pug b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/sql-connector.pug index 2e61fc2439cee..708aa0d8a3b17 100644 --- a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/sql-connector.pug +++ b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/sql-connector.pug @@ -32,17 +32,17 @@ panel-collapsible( .pca-form-column-6.pc-form-grid-row .pc-form-grid-col-60 +checkbox('Enabled', connectionEnabled, '"SqlConnectorEnabled"', 'Flag indicating whether to configure SQL connector configuration') - .pc-form-grid-col-40 + .pc-form-grid-col-60 +text-enabled('Host:', `${connectionModel}.host`, '"SqlConnectorHost"', connectionEnabled, 'false', 'localhost') - .pc-form-grid-col-20 + .pc-form-grid-col-30 +number('Port:', `${connectionModel}.port`, '"SqlConnectorPort"', connectionEnabled, '10800', '1025') - .pc-form-grid-col-20 + .pc-form-grid-col-30 +number('Port range:', `${connectionModel}.portRange`, '"SqlConnectorPortRange"', connectionEnabled, '100', '0') - .pc-form-grid-col-20 + .pc-form-grid-col-30 +number('Socket send buffer size:', `${connectionModel}.socketSendBufferSize`, '"SqlConnectorSocketSendBufferSize"', connectionEnabled, '0', '0', 'Socket send buffer size.
    \ When set to 0, operation system default will be used') - .pc-form-grid-col-20 + .pc-form-grid-col-30 +number('Socket receive buffer size:', `${connectionModel}.socketReceiveBufferSize`, '"SqlConnectorSocketReceiveBufferSize"', connectionEnabled, '0', '0', 'Socket receive buffer size.
    \ When set to 0, operation system default will be used') From 17a61759a638207d3ef105863bb5d4d64d1db053 Mon Sep 17 00:00:00 2001 From: Alexander Kalinin Date: Sat, 28 Apr 2018 17:58:27 +0700 Subject: [PATCH 0168/1463] IGNITE-8336 Web console: Fixed failing E2E notebooks tests. (cherry picked from commit eb6772f) --- .../components/queries-notebooks-list/controller.js | 2 ++ .../page-queries/components/queries-notebooks-list/style.scss | 4 ++++ .../components/queries-notebooks-list/template.tpl.pug | 2 +- .../frontend/app/components/password-visibility/index.spec.js | 2 +- 4 files changed, 8 insertions(+), 2 deletions(-) diff --git a/modules/web-console/frontend/app/components/page-queries/components/queries-notebooks-list/controller.js b/modules/web-console/frontend/app/components/page-queries/components/queries-notebooks-list/controller.js index 7c06f2a5c1fe6..2e4146ce798c3 100644 --- a/modules/web-console/frontend/app/components/page-queries/components/queries-notebooks-list/controller.js +++ b/modules/web-console/frontend/app/components/page-queries/components/queries-notebooks-list/controller.js @@ -103,6 +103,8 @@ export class NotebooksListCtrl { this.IgniteMessages.showError(err); } finally { + this.$scope.$applyAsync(); + await this.IgniteLoading.finish('notebooksLoading'); } } diff --git a/modules/web-console/frontend/app/components/page-queries/components/queries-notebooks-list/style.scss b/modules/web-console/frontend/app/components/page-queries/components/queries-notebooks-list/style.scss index 0b96b88d86e2a..6cdbd397689d7 100644 --- a/modules/web-console/frontend/app/components/page-queries/components/queries-notebooks-list/style.scss +++ b/modules/web-console/frontend/app/components/page-queries/components/queries-notebooks-list/style.scss @@ -27,6 +27,10 @@ background: white; } + .ui-grid-render-container-left:before { + display: none; + } + .notebook-name { a { color: #0067b9; diff --git a/modules/web-console/frontend/app/components/page-queries/components/queries-notebooks-list/template.tpl.pug b/modules/web-console/frontend/app/components/page-queries/components/queries-notebooks-list/template.tpl.pug index 75b5e99e3a4d6..c6268e03bdf9b 100644 --- a/modules/web-console/frontend/app/components/page-queries/components/queries-notebooks-list/template.tpl.pug +++ b/modules/web-console/frontend/app/components/page-queries/components/queries-notebooks-list/template.tpl.pug @@ -45,7 +45,7 @@ page-queries-slot(slot-name="'queriesButtons'" ng-if="!$root.IgniteDemoMode") .panel-collapse(ignite-loading='notebooksLoading' ignite-loading-text='Loading notebooks...') - .grid.ui-grid--ignite#queriesNotebooksList(ui-grid='$ctrl.gridOptions' ui-grid-resize-columns ui-grid-selection ui-grid-pinning ui-grid-hovering) + .grid.ui-grid--ignite#queriesNotebooksList(ui-grid='$ctrl.gridOptions' ui-grid-resize-columns ui-grid-selection ui-grid-hovering) grid-no-data(grid-api='$ctrl.gridApi') | You have no notebooks. a.link-success(ng-click='$ctrl.createNotebook()') Create one? diff --git a/modules/web-console/frontend/app/components/password-visibility/index.spec.js b/modules/web-console/frontend/app/components/password-visibility/index.spec.js index 236e497d55023..f887092b5fcb6 100644 --- a/modules/web-console/frontend/app/components/password-visibility/index.spec.js +++ b/modules/web-console/frontend/app/components/password-visibility/index.spec.js @@ -46,7 +46,7 @@ suite('password-visibility', () => { `); $compile(el)($scope); - const toggleButton = el.find('password-visibility-toggle-button button'); + const toggleButton = el.find('password-visibility-toggle-button').children()[0]; $scope.$digest(); assert.isFalse(el.hasClass(PASSWORD_VISIBLE_CLASS), 'Password is hidden by default'); From 6ba7d4883ca838c8feb79e2465f9396328012531 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Mon, 7 May 2018 16:59:01 +0700 Subject: [PATCH 0169/1463] WC-475 Web Console: Extract of meaningful message from exception. (cherry picked from commit 3da3fef) --- .../web-console/frontend/app/services/Messages.service.js | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/modules/web-console/frontend/app/services/Messages.service.js b/modules/web-console/frontend/app/services/Messages.service.js index 620d3728eb7fb..ab565b2a1f098 100644 --- a/modules/web-console/frontend/app/services/Messages.service.js +++ b/modules/web-console/frontend/app/services/Messages.service.js @@ -33,10 +33,9 @@ export default ['IgniteMessages', ['$alert', ($alert) => { if (err.hasOwnProperty('message')) { const msg = err.message; + const lastIdx = msg.lastIndexOf(' err='); - const errIndex = msg.indexOf(' err='); - - return prefix + (errIndex >= 0 ? msg.substring(errIndex + 5, msg.length - 1) : msg); + return prefix + (lastIdx >= 0 ? msg.substring(lastIdx + 5, msg.indexOf(']', lastIdx)) : msg); } if (nonEmpty(err.className)) { From cb1ce1d1f9fc0edd24e774e3d870461bb5d80df4 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Fri, 11 May 2018 17:55:22 +0300 Subject: [PATCH 0170/1463] IGNITE-7999: JDBC Thin Driver: added unordered streaming mode. This closes #3789. --- .../internal/jdbc2/JdbcStreamingSelfTest.java | 3 - .../jdbc/suite/IgniteJdbcDriverTestSuite.java | 6 +- ...bcThinConnectionMultipleAddressesTest.java | 7 +- ...=> JdbcThinStreamingAbstractSelfTest.java} | 59 ++-- .../JdbcThinStreamingNotOrderedSelfTest.java | 38 +++ .../JdbcThinStreamingOrderedSelfTest.java | 39 +++ .../jdbc/thin/JdbcThinConnection.java | 312 ++++++++++++++---- .../internal/jdbc/thin/JdbcThinTcpIo.java | 63 +++- .../odbc/ClientListenerNioListener.java | 23 +- .../odbc/jdbc/JdbcBatchExecuteRequest.java | 27 ++ .../odbc/jdbc/JdbcBatchExecuteResult.java | 26 +- .../odbc/jdbc/JdbcConnectionContext.java | 34 +- .../jdbc/JdbcOrderedBatchExecuteRequest.java | 85 +++++ .../jdbc/JdbcOrderedBatchExecuteResult.java | 75 +++++ .../processors/odbc/jdbc/JdbcRequest.java | 8 + .../odbc/jdbc/JdbcRequestHandler.java | 119 ++++++- .../odbc/jdbc/JdbcResponseSender.java | 31 ++ .../processors/odbc/jdbc/JdbcResult.java | 13 + .../processors/query/SqlClientContext.java | 147 +++++++-- .../ignite/internal/sql/SqlKeyword.java | 3 + .../sql/command/SqlSetStreamingCommand.java | 19 ++ .../sql/SqlParserSetStreamingSelfTest.java | 39 ++- .../processors/query/h2/IgniteH2Indexing.java | 2 +- .../config/ignite-localhost-config.xml | 2 - .../benchmark-jdbc-thin-streaming.properties | 132 ++++++++ .../yardstick/upload/StreamerParams.java | 7 +- .../upload/UploadBenchmarkArguments.java | 19 +- .../yardstick/upload/model/QueryFactory.java | 2 + 28 files changed, 1169 insertions(+), 171 deletions(-) rename modules/clients/src/test/java/org/apache/ignite/jdbc/thin/{JdbcThinStreamingSelfTest.java => JdbcThinStreamingAbstractSelfTest.java} (91%) create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingNotOrderedSelfTest.java create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingOrderedSelfTest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcOrderedBatchExecuteRequest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcOrderedBatchExecuteResult.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResponseSender.java create mode 100644 modules/yardstick/config/upload/benchmark-jdbc-thin-streaming.properties 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 10adedcf3ec66..e302529404d70 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 @@ -56,9 +56,6 @@ public class JdbcStreamingSelfTest extends GridCommonAbstractTest { private static final String STREAMING_URL = CFG_URL_PREFIX + "cache=person@modules/clients/src/test/config/jdbc-config.xml"; - /** */ - protected transient IgniteLogger log; - /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { return getConfiguration0(gridName); 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 a88ebe820e557..a18cb4501e207 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 @@ -70,7 +70,8 @@ import org.apache.ignite.jdbc.thin.JdbcThinSchemaCaseTest; import org.apache.ignite.jdbc.thin.JdbcThinSelectAfterAlterTable; import org.apache.ignite.jdbc.thin.JdbcThinStatementSelfTest; -import org.apache.ignite.jdbc.thin.JdbcThinStreamingSelfTest; +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.JdbcThinUpdateStatementSelfTest; import org.apache.ignite.jdbc.thin.JdbcThinUpdateStatementSkipReducerOnUpdateSelfTest; @@ -128,7 +129,8 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(JdbcBlobTest.class)); suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcStreamingSelfTest.class)); - suite.addTest(new TestSuite(JdbcThinStreamingSelfTest.class)); + suite.addTest(new TestSuite(JdbcThinStreamingNotOrderedSelfTest.class)); + suite.addTest(new TestSuite(JdbcThinStreamingOrderedSelfTest.class)); // DDL tests. suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcDynamicIndexAtomicPartitionedNearSelfTest.class)); diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionMultipleAddressesTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionMultipleAddressesTest.java index 2c2aba970a571..e1fb295c4d056 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionMultipleAddressesTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionMultipleAddressesTest.java @@ -372,9 +372,7 @@ private void checkReconnectOnStreaming(String url, boolean allNodes) throws Exce return null; } - }, SQLException.class, "Failed to communicate with Ignite cluster"); - - assertTrue(id[0] > 0); + }, SQLException.class, "Failed to communicate with Ignite cluster on JDBC streaming"); int minId = id[0]; @@ -382,6 +380,9 @@ private void checkReconnectOnStreaming(String url, boolean allNodes) throws Exce final Statement stmt1 = conn.createStatement(); + stmt1.execute("SET STREAMING 1 BATCH_SIZE 10 ALLOW_OVERWRITE 0 " + + " PER_NODE_BUFFER_SIZE 1000 FLUSH_FREQUENCY 1000"); + for (int i = 0; i < 10; ++i, id[0]++) stmt1.execute("INSERT INTO TEST(id, val) values (" + id[0] + ", " + id[0] + ")"); diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingAbstractSelfTest.java similarity index 91% rename from modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingSelfTest.java rename to modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingAbstractSelfTest.java index 3c36f54327df9..70046356b210b 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingAbstractSelfTest.java @@ -43,9 +43,9 @@ /** * Tests for streaming via thin driver. */ -public class JdbcThinStreamingSelfTest extends JdbcStreamingSelfTest { +public abstract class JdbcThinStreamingAbstractSelfTest extends JdbcStreamingSelfTest { /** */ - private int batchSize = 17; + protected int batchSize = 17; /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { @@ -56,6 +56,16 @@ public class JdbcThinStreamingSelfTest extends JdbcStreamingSelfTest { batchSize = 17; } + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + // Init IndexingWithContext.cliCtx + try (Connection c = createOrdinaryConnection()) { + execute(c, "SELECT 1"); + } + } + /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { try (Connection c = createOrdinaryConnection()) { @@ -67,17 +77,7 @@ public class JdbcThinStreamingSelfTest extends JdbcStreamingSelfTest { super.afterTest(); } - /** {@inheritDoc} */ - @Override protected Connection createStreamedConnection(boolean allowOverwrite, long flushFreq) throws Exception { - Connection c = JdbcThinAbstractSelfTest.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} */ + /** {@inheritDoc} */ @Override protected Connection createOrdinaryConnection() throws SQLException { return JdbcThinAbstractSelfTest.connect(grid(0), null); } @@ -170,7 +170,10 @@ public void testSimultaneousStreaming() throws Exception { SqlClientContext cliCtx = sqlClientContext(); - HashMap> streamers = U.field(cliCtx, "streamers"); + final HashMap> streamers = U.field(cliCtx, "streamers"); + + // Wait when node process requests (because client send batch requests async). + GridTestUtils.waitForCondition(() -> streamers.size() == 2, 1000); assertEquals(2, streamers.size()); @@ -254,7 +257,7 @@ public void testStreamingWithMixedStatementTypes() throws Exception { /** * @throws SQLException if failed. */ - public void testStreamingOffToOn() throws SQLException { + public void testStreamingOffToOn() throws Exception { try (Connection conn = createOrdinaryConnection()) { assertStreamingState(false); @@ -264,6 +267,19 @@ public void testStreamingOffToOn() throws SQLException { } } + /** + * @throws SQLException if failed. + */ + public void testStreamingOffToOff() throws Exception { + try (Connection conn = createOrdinaryConnection()) { + assertStreamingState(false); + + execute(conn, "SET STREAMING 0"); + + assertStreamingState(false); + } + } + /** * @throws SQLException if failed. */ @@ -330,7 +346,7 @@ public void testStreamingReEnabled() throws Exception { U.sleep(500); - assertEquals((Integer)111, U.field(conn, "streamBatchSize")); + assertEquals((Integer)111, U.field((Object)U.field(conn, "streamState"), "streamBatchSize")); SqlClientContext cliCtx = sqlClientContext(); @@ -418,7 +434,7 @@ public void testBatchingSetStreamingStatement() { /** * Check that there's nothing in cache. */ - private void assertCacheEmpty() { + protected void assertCacheEmpty() { assertEquals(0, cache().size(CachePeekMode.ALL)); } @@ -427,7 +443,7 @@ private void assertCacheEmpty() { * @param sql Statement. * @throws SQLException if failed. */ - private static void execute(Connection conn, String sql) throws SQLException { + protected static void execute(Connection conn, String sql) throws SQLException { try (Statement s = conn.createStatement()) { s.execute(sql); } @@ -444,11 +460,14 @@ private SqlClientContext sqlClientContext() { /** * Check that streaming state on target node is as expected. + * * @param on Expected streaming state. */ - private void assertStreamingState(boolean on) { + protected void assertStreamingState(boolean on) throws Exception { SqlClientContext cliCtx = sqlClientContext(); + GridTestUtils.waitForCondition(() -> cliCtx.isStream() == on, 1000); + assertEquals(on, cliCtx.isStream()); } @@ -462,7 +481,7 @@ private void assertStreamingState(boolean on) { /** * */ - private static final class IndexingWithContext extends IgniteH2Indexing { + static final class IndexingWithContext extends IgniteH2Indexing { /** Client context. */ static SqlClientContext cliCtx; 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 new file mode 100644 index 0000000000000..b91258f8a4eb9 --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingNotOrderedSelfTest.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.jdbc.thin; + +import java.sql.Connection; + +/** + * Tests for not ordered streaming via thin driver. + */ +public class JdbcThinStreamingNotOrderedSelfTest extends JdbcThinStreamingAbstractSelfTest { + /** {@inheritDoc} */ + @Override protected Connection createStreamedConnection(boolean allowOverwrite, long flushFreq) throws Exception { + Connection c = JdbcThinAbstractSelfTest.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; + } +} \ No newline at end of file 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 new file mode 100644 index 0000000000000..b615f8cd9614e --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStreamingOrderedSelfTest.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.jdbc.thin; + +import java.sql.Connection; + +/** + * Tests for ordered streaming via thin driver. + */ +public class JdbcThinStreamingOrderedSelfTest extends JdbcThinStreamingAbstractSelfTest { + /** {@inheritDoc} */ + @Override protected Connection createStreamedConnection(boolean allowOverwrite, long flushFreq) throws Exception { + Connection c = JdbcThinAbstractSelfTest.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 + + " ORDERED;" + ); + + return c; + } +} \ No newline at end of file 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 3478124ac1811..634579b7074e5 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 @@ -39,13 +39,15 @@ import java.util.Map; import java.util.Properties; import java.util.concurrent.Executor; +import java.util.concurrent.Semaphore; import java.util.logging.Level; import java.util.logging.Logger; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; import org.apache.ignite.internal.processors.odbc.ClientListenerResponse; import org.apache.ignite.internal.processors.odbc.SqlStateCode; -import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteRequest; -import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteResult; +import org.apache.ignite.internal.processors.odbc.jdbc.JdbcOrderedBatchExecuteRequest; +import org.apache.ignite.internal.processors.odbc.jdbc.JdbcOrderedBatchExecuteResult; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQuery; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQueryExecuteRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest; @@ -55,6 +57,7 @@ import org.apache.ignite.internal.processors.query.QueryUtils; import org.apache.ignite.internal.sql.command.SqlCommand; import org.apache.ignite.internal.sql.command.SqlSetStreamingCommand; +import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.lang.IgniteProductVersion; @@ -91,7 +94,7 @@ public class JdbcThinConnection implements Connection { private boolean readOnly; /** Streaming flag. */ - private volatile boolean stream; + private volatile StreamState streamState; /** Current transaction holdability. */ private int holdability; @@ -108,15 +111,6 @@ public class JdbcThinConnection implements Connection { /** Connection properties. */ private ConnectionProperties connProps; - /** Batch size for streaming. */ - private int streamBatchSize; - - /** Batch for streaming. */ - private List streamBatch; - - /** Last added query to recognize batches. */ - private String lastStreamQry; - /** Connected. */ private boolean connected; @@ -172,7 +166,7 @@ private synchronized void ensureConnected() throws SQLException { * @return Whether this connection is streamed or not. */ boolean isStream() { - return stream; + return streamState != null; } /** @@ -182,24 +176,28 @@ boolean isStream() { */ void executeNative(String sql, SqlCommand cmd) throws SQLException { if (cmd instanceof SqlSetStreamingCommand) { - // If streaming is already on, we have to disable it first. - if (stream) { - // We have to send request regardless of actual batch size. - executeBatch(true); + SqlSetStreamingCommand cmd0 = (SqlSetStreamingCommand)cmd; + + // If streaming is already on, we have to close it first. + if (streamState != null) { + streamState.close(); - stream = false; + streamState = null; } boolean newVal = ((SqlSetStreamingCommand)cmd).isTurnOn(); // Actual ON, if needed. if (newVal) { + if (!cmd0.isOrdered() && !cliIo.igniteVersion().greaterThanEqual(2, 5, 0)) { + throw new SQLException("Streaming without order doesn't supported by server [remoteNodeVer=" + + cliIo.igniteVersion() + ']', SqlStateCode.INTERNAL_ERROR); + } + sendRequest(new JdbcQueryExecuteRequest(JdbcStatementType.ANY_STATEMENT_TYPE, schema, 1, 1, sql, null)); - streamBatchSize = ((SqlSetStreamingCommand)cmd).batchSize(); - - stream = true; + streamState = new StreamState((SqlSetStreamingCommand)cmd); } } else @@ -214,39 +212,9 @@ void executeNative(String sql, SqlCommand cmd) throws SQLException { * @throws SQLException On error. */ void addBatch(String sql, List args) throws SQLException { - boolean newQry = (args == null || !F.eq(lastStreamQry, sql)); - - // Providing null as SQL here allows for recognizing subbatches on server and handling them more efficiently. - JdbcQuery q = new JdbcQuery(newQry ? sql : null, args != null ? args.toArray() : null); - - if (streamBatch == null) - streamBatch = new ArrayList<>(streamBatchSize); - - streamBatch.add(q); - - // Null args means "addBatch(String)" was called on non-prepared Statement, - // we don't want to remember its query string. - lastStreamQry = (args != null ? sql : null); - - if (streamBatch.size() == streamBatchSize) - executeBatch(false); - } - - /** - * @param lastBatch Whether open data streamers must be flushed and closed after this batch. - * @throws SQLException if failed. - */ - private void executeBatch(boolean lastBatch) throws SQLException { - JdbcBatchExecuteResult res = sendRequest(new JdbcBatchExecuteRequest(schema, streamBatch, lastBatch)); - - streamBatch = null; + assert isStream(); - lastStreamQry = null; - - if (res.errorCode() != ClientListenerResponse.STATUS_SUCCESS) { - throw new BatchUpdateException(res.errorMessage(), IgniteQueryErrorCode.codeToSqlState(res.errorCode()), - res.errorCode(), res.updateCounts()); - } + streamState.addBatch(sql, args); } /** {@inheritDoc} */ @@ -399,13 +367,10 @@ private void checkCursorOptions(int resSetType, int resSetConcurrency, if (isClosed()) return; - if (!F.isEmpty(streamBatch)) { - try { - executeBatch(true); - } - catch (SQLException e) { - LOG.log(Level.WARNING, "Exception during batch send on streamed connection close", e); - } + if (streamState != null) { + streamState.close(); + + streamState = null; } closed = true; @@ -797,6 +762,28 @@ R sendRequest(JdbcRequest req) throws SQLException { } } + /** + * Send request for execution via {@link #cliIo}. Response is waited at the separate thread + * (see {@link StreamState#asyncRespReaderThread}). + * @param req Request. + * @throws SQLException On any error. + */ + private void sendRequestNotWaitResponse(JdbcOrderedBatchExecuteRequest req) throws SQLException { + ensureConnected(); + + try { + cliIo.sendBatchRequestNoWaitResponse(req); + } + catch (SQLException e) { + throw e; + } + catch (Exception e) { + onDisconnect(); + + throw new SQLException("Failed to communicate with Ignite cluster.", SqlStateCode.CONNECTION_FAILURE, e); + } + } + /** * @return Connection URL. */ @@ -815,9 +802,11 @@ private void onDisconnect() { connected = false; - streamBatch = null; + if (streamState != null) { + streamState.close0(); - lastStreamQry = null; + streamState = null; + } synchronized (stmtsMux) { for (JdbcThinStatement s : stmts) @@ -846,4 +835,203 @@ private static String normalizeSchema(String schemaName) { return res; } + + /** + * Streamer state and + */ + private class StreamState { + /** Maximum requests count that may be sent before any responses. */ + private static final int MAX_REQUESTS_BEFORE_RESPONSE = 10; + + /** Wait timeout. */ + private static final long WAIT_TIMEOUT = 1; + + /** Batch size for streaming. */ + private int streamBatchSize; + + /** Batch for streaming. */ + private List streamBatch; + + /** Last added query to recognize batches. */ + private String lastStreamQry; + + /** Keep request order on execution. */ + private long order; + + /** Async response reader thread. */ + private Thread asyncRespReaderThread; + + /** Async response error. */ + private volatile Exception err; + + /** The order of the last batch request at the stream. */ + private long lastRespOrder = -1; + + /** Last response future. */ + private final GridFutureAdapter lastRespFut = new GridFutureAdapter<>(); + + /** Response semaphore sem. */ + private Semaphore respSem = new Semaphore(MAX_REQUESTS_BEFORE_RESPONSE); + + /** + * @param cmd Stream cmd. + */ + StreamState(SqlSetStreamingCommand cmd) { + streamBatchSize = cmd.batchSize(); + + asyncRespReaderThread = new Thread(this::readResponses); + + asyncRespReaderThread.start(); + } + + /** + * Add another query for batched execution. + * @param sql Query. + * @param args Arguments. + * @throws SQLException On error. + */ + void addBatch(String sql, List args) throws SQLException { + checkError(); + + boolean newQry = (args == null || !F.eq(lastStreamQry, sql)); + + // Providing null as SQL here allows for recognizing subbatches on server and handling them more efficiently. + JdbcQuery q = new JdbcQuery(newQry ? sql : null, args != null ? args.toArray() : null); + + if (streamBatch == null) + streamBatch = new ArrayList<>(streamBatchSize); + + streamBatch.add(q); + + // Null args means "addBatch(String)" was called on non-prepared Statement, + // we don't want to remember its query string. + lastStreamQry = (args != null ? sql : null); + + if (streamBatch.size() == streamBatchSize) + executeBatch(false); + } + + /** + * @param lastBatch Whether open data streamers must be flushed and closed after this batch. + * @throws SQLException if failed. + */ + private void executeBatch(boolean lastBatch) throws SQLException { + checkError(); + + if (lastBatch) + lastRespOrder = order; + + try { + respSem.acquire(); + + sendRequestNotWaitResponse( + new JdbcOrderedBatchExecuteRequest(schema, streamBatch, lastBatch, order)); + + streamBatch = null; + + lastStreamQry = null; + + if (lastBatch) { + try { + lastRespFut.get(); + } + catch (IgniteCheckedException e) { + // No-op. + // No exceptions are expected here. + } + + checkError(); + } + else + order++; + } + catch (InterruptedException e) { + throw new SQLException("Streaming operation was interrupted", SqlStateCode.INTERNAL_ERROR, e); + } + } + + /** + * Throws at the user thread exception that was thrown at the {@link #asyncRespReaderThread} thread. + * @throws SQLException Saved exception. + */ + void checkError() throws SQLException { + if (err != null) { + Exception err0 = err; + + err = null; + + if (err0 instanceof SQLException) + throw (SQLException)err0; + else { + onDisconnect(); + + throw new SQLException("Failed to communicate with Ignite cluster on JDBC streaming.", + SqlStateCode.CONNECTION_FAILURE, err0); + } + } + } + + /** + * @throws SQLException On error. + */ + void close() throws SQLException { + close0(); + + checkError(); + } + + /** + */ + void close0() { + if (connected) { + try { + executeBatch(true); + } + catch (SQLException e) { + err = e; + + LOG.log(Level.WARNING, "Exception during batch send on streamed connection close", e); + } + } + + if (asyncRespReaderThread != null) + asyncRespReaderThread.interrupt(); + } + + /** + * + */ + void readResponses () { + try { + while (true) { + JdbcResponse resp = cliIo.readResponse(); + + if (resp.response() instanceof JdbcOrderedBatchExecuteResult) { + JdbcOrderedBatchExecuteResult res = (JdbcOrderedBatchExecuteResult)resp.response(); + + respSem.release(); + + if (res.errorCode() != ClientListenerResponse.STATUS_SUCCESS) { + err = new BatchUpdateException(res.errorMessage(), + IgniteQueryErrorCode.codeToSqlState(res.errorCode()), + res.errorCode(), res.updateCounts()); + } + + // Receive the response for the last request. + if (res.order() == lastRespOrder) { + lastRespFut.onDone(); + + break; + } + } + + if (resp.status() != ClientListenerResponse.STATUS_SUCCESS) + err = new SQLException(resp.error(), IgniteQueryErrorCode.codeToSqlState(resp.status())); + } + } + catch (Exception e) { + err = e; + } + } + } } \ No newline at end of file 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 4631e5da713cb..44c19847b4ddb 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 @@ -38,6 +38,7 @@ import org.apache.ignite.internal.processors.odbc.ClientListenerRequest; import org.apache.ignite.internal.processors.odbc.SqlStateCode; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteRequest; +import org.apache.ignite.internal.processors.odbc.jdbc.JdbcOrderedBatchExecuteRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQuery; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQueryCloseRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQueryFetchRequest; @@ -417,6 +418,44 @@ private void handshake_2_1_0() throws IOException, SQLException { } } + /** + * @param req Request. + * @throws IOException In case of IO error. + * @throws SQLException On error. + */ + void sendBatchRequestNoWaitResponse(JdbcOrderedBatchExecuteRequest req) throws IOException, SQLException { + synchronized (mux) { + if (ownThread != null) { + throw new SQLException("Concurrent access to JDBC connection is not allowed" + + " [ownThread=" + ownThread.getName() + + ", curThread=" + Thread.currentThread().getName(), SqlStateCode.CONNECTION_FAILURE); + } + + ownThread = Thread.currentThread(); + } + + try { + if (!igniteVer.greaterThanEqual(2, 5, 0)) { + throw new SQLException("Streaming without response doesn't supported by server [driverProtocolVer=" + + CURRENT_VER + ", remoteNodeVer=" + igniteVer + ']', SqlStateCode.INTERNAL_ERROR); + } + + int cap = guessCapacity(req); + + BinaryWriterExImpl writer = new BinaryWriterExImpl(null, new BinaryHeapOutputStream(cap), + null, null); + + req.writeBinary(writer); + + send(writer.array()); + } + finally { + synchronized (mux) { + ownThread = null; + } + } + } + /** * @param req Request. * @return Server response. @@ -444,13 +483,7 @@ JdbcResponse sendRequest(JdbcRequest req) throws SQLException, IOException { send(writer.array()); - BinaryReaderExImpl reader = new BinaryReaderExImpl(null, new BinaryHeapInputStream(read()), null, null, false); - - JdbcResponse res = new JdbcResponse(); - - res.readBinary(reader); - - return res; + return readResponse(); } finally { synchronized (mux) { @@ -459,6 +492,22 @@ JdbcResponse sendRequest(JdbcRequest req) throws SQLException, IOException { } } + /** + * @return Server response. + * @throws IOException In case of IO error. + */ + @SuppressWarnings("unchecked") + JdbcResponse readResponse() throws IOException { + BinaryReaderExImpl reader = new BinaryReaderExImpl(null, new BinaryHeapInputStream(read()), null, null, false); + + JdbcResponse res = new JdbcResponse(); + + res.readBinary(reader); + + return res; + } + + /** * Try to guess request capacity. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java index 407c1a02efc34..be55ab9a6210e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java @@ -159,16 +159,18 @@ public ClientListenerNioListener(GridKernalContext ctx, GridSpinBusyLock busyLoc ClientListenerResponse resp = handler.handle(req); - if (log.isDebugEnabled()) { - long dur = (System.nanoTime() - startTime) / 1000; + if (resp != null) { + if (log.isDebugEnabled()) { + long dur = (System.nanoTime() - startTime) / 1000; - log.debug("Client request processed [reqId=" + req.requestId() + ", dur(mcs)=" + dur + - ", resp=" + resp.status() + ']'); - } + log.debug("Client request processed [reqId=" + req.requestId() + ", dur(mcs)=" + dur + + ", resp=" + resp.status() + ']'); + } - byte[] outMsg = parser.encode(resp); + byte[] outMsg = parser.encode(resp); - ses.send(outMsg); + ses.send(outMsg); + } } catch (Exception e) { U.error(log, "Failed to process client request [req=" + req + ']', e); @@ -216,7 +218,7 @@ private void onHandshake(GridNioSession ses, byte[] msg) { ClientListenerConnectionContext connCtx = null; try { - connCtx = prepareContext(clientType); + connCtx = prepareContext(ses, clientType); ensureClientPermissions(clientType); @@ -270,17 +272,18 @@ private void onHandshake(GridNioSession ses, byte[] msg) { /** * Prepare context. * + * @param ses Session. * @param clientType Client type. * @return Context. * @throws IgniteCheckedException If failed. */ - private ClientListenerConnectionContext prepareContext(byte clientType) throws IgniteCheckedException { + private ClientListenerConnectionContext prepareContext(GridNioSession ses, byte clientType) throws IgniteCheckedException { switch (clientType) { case ODBC_CLIENT: return new OdbcConnectionContext(ctx, busyLock, maxCursors); case JDBC_CLIENT: - return new JdbcConnectionContext(ctx, busyLock, maxCursors); + return new JdbcConnectionContext(ctx, ses, busyLock, maxCursors); case THIN_CLIENT: return new ClientConnectionContext(ctx, maxCursors); 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 73fd04f9d2f66..bdc558c0daad7 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 @@ -52,9 +52,18 @@ public JdbcBatchExecuteRequest() { super(BATCH_EXEC); } + /** + * Constructor for child requests. + * @param type Request type/ + */ + protected JdbcBatchExecuteRequest(byte type) { + super(type); + } + /** * @param schemaName Schema name. * @param queries Queries. + * @param lastStreamBatch {@code true} in case the request is the last batch at the stream. */ public JdbcBatchExecuteRequest(String schemaName, List queries, boolean lastStreamBatch) { super(BATCH_EXEC); @@ -66,6 +75,24 @@ public JdbcBatchExecuteRequest(String schemaName, List queries, boole this.lastStreamBatch = lastStreamBatch; } + /** + * Constructor for child requests. + * + * @param type Request type. + * @param schemaName Schema name. + * @param queries Queries. + * @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) { + super(type); + + assert lastStreamBatch || !F.isEmpty(queries); + + this.schemaName = schemaName; + this.queries = queries; + this.lastStreamBatch = lastStreamBatch; + } + /** * @return Schema name. */ 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 917e60ae75ccc..3fc9dd7c5ab21 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 @@ -36,18 +36,26 @@ public class JdbcBatchExecuteResult extends JdbcResult { private String errMsg; /** - * Condtructor. + * Constructor. */ - public JdbcBatchExecuteResult() { + JdbcBatchExecuteResult() { super(BATCH_EXEC); } + /** + * Constructor for child results. + * @param type Result type. + */ + JdbcBatchExecuteResult(byte type) { + super(type); + } + /** * @param updateCnts Update counts for batch. * @param errCode Error code. * @param errMsg Error message. */ - public JdbcBatchExecuteResult(int [] updateCnts, int errCode, String errMsg) { + JdbcBatchExecuteResult(int [] updateCnts, int errCode, String errMsg) { super(BATCH_EXEC); this.updateCnts = updateCnts; @@ -55,6 +63,18 @@ public JdbcBatchExecuteResult(int [] updateCnts, int errCode, String errMsg) { this.errMsg = errMsg; } + /** + * @param type Result type. + * @param res Result. + */ + JdbcBatchExecuteResult(byte type, JdbcBatchExecuteResult res) { + super(type); + + this.updateCnts = res.updateCnts; + this.errCode = res.errCode; + this.errMsg = res.errMsg; + } + /** * @return Update count for DML queries. */ 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 2fe3b9c932203..272c2f0ea41e2 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 @@ -17,10 +17,10 @@ package org.apache.ignite.internal.processors.odbc.jdbc; -import java.io.IOException; import java.util.HashSet; import java.util.Set; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.processors.authentication.AuthorizationContext; @@ -28,7 +28,9 @@ 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.odbc.ClientListenerResponse; import org.apache.ignite.internal.util.GridSpinBusyLock; +import org.apache.ignite.internal.util.nio.GridNioSession; import org.apache.ignite.internal.util.typedef.F; /** @@ -59,9 +61,15 @@ public class JdbcConnectionContext implements ClientListenerConnectionContext { /** Context. */ private final GridKernalContext ctx; + /** Session. */ + private final GridNioSession ses; + /** Shutdown busy lock. */ private final GridSpinBusyLock busyLock; + /** Logger. */ + private final IgniteLogger log; + /** Maximum allowed cursors. */ private final int maxCursors; @@ -83,13 +91,17 @@ public class JdbcConnectionContext implements ClientListenerConnectionContext { /** * Constructor. * @param ctx Kernal Context. + * @param ses Session. * @param busyLock Shutdown busy lock. * @param maxCursors Maximum allowed cursors. */ - public JdbcConnectionContext(GridKernalContext ctx, GridSpinBusyLock busyLock, int maxCursors) { + public JdbcConnectionContext(GridKernalContext ctx, GridNioSession ses, GridSpinBusyLock busyLock, int maxCursors) { this.ctx = ctx; + this.ses = ses; this.busyLock = busyLock; this.maxCursors = maxCursors; + + log = ctx.log(getClass()); } /** {@inheritDoc} */ @@ -146,11 +158,23 @@ public JdbcConnectionContext(GridKernalContext ctx, GridSpinBusyLock busyLock, i catch (Exception e) { throw new IgniteCheckedException("Handshake error: " + e.getMessage(), e); } + parser = new JdbcMessageParser(ctx); - handler = new JdbcRequestHandler(ctx, busyLock, maxCursors, distributedJoins, enforceJoinOrder, - collocated, replicatedOnly, autoCloseCursors, lazyExec, skipReducerOnUpdate, actx, ver); + JdbcResponseSender sender = new JdbcResponseSender() { + @Override public void send(ClientListenerResponse resp) { + if (resp != null) { + if (log.isDebugEnabled()) + log.debug("Async response: [resp=" + resp.status() + ']'); - parser = new JdbcMessageParser(ctx); + byte[] outMsg = parser.encode(resp); + + ses.send(outMsg); + } + } + }; + + handler = new JdbcRequestHandler(ctx, busyLock, sender, maxCursors, distributedJoins, enforceJoinOrder, + collocated, replicatedOnly, autoCloseCursors, lazyExec, skipReducerOnUpdate, actx, ver); } /** {@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 new file mode 100644 index 0000000000000..3e847310af232 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcOrderedBatchExecuteRequest.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.internal.processors.odbc.jdbc; + +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.util.typedef.internal.S; +import org.jetbrains.annotations.NotNull; + +/** + * JDBC batch execute ordered request. + */ +public class JdbcOrderedBatchExecuteRequest extends JdbcBatchExecuteRequest + implements Comparable { + /** Order. */ + private long order; + + /** + * Default constructor. + */ + public JdbcOrderedBatchExecuteRequest() { + super(BATCH_EXEC_ORDERED); + } + + /** + * @param schemaName Schema name. + * @param queries Queries. + * @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); + + this.order = order; + } + + /** + * @return Request order. + */ + public long order() { + return order; + } + + /** {@inheritDoc} */ + @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { + super.writeBinary(writer); + + writer.writeLong(order); + } + + /** {@inheritDoc} */ + @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { + super.readBinary(reader); + + order = reader.readLong(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(JdbcOrderedBatchExecuteRequest.class, this); + } + + /** {@inheritDoc} */ + @Override public int compareTo(@NotNull JdbcOrderedBatchExecuteRequest o) { + return Long.compare(order, o.order); + } +} 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 new file mode 100644 index 0000000000000..84853d484f3c2 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcOrderedBatchExecuteResult.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.odbc.jdbc; + +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.util.typedef.internal.S; + +/** + * JDBC batch execute ordered result. + */ +public class JdbcOrderedBatchExecuteResult extends JdbcBatchExecuteResult { + /** Order. */ + private long order; + + /** + * Constructor. + */ + public JdbcOrderedBatchExecuteResult() { + super(BATCH_EXEC_ORDERED); + } + + /** + * @param res Result. + * @param order Order. + */ + public JdbcOrderedBatchExecuteResult(JdbcBatchExecuteResult res, long order) { + super(BATCH_EXEC_ORDERED, res); + + this.order = order; + } + + /** + * @return Order. + */ + public long order() { + return order; + } + + /** {@inheritDoc} */ + @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException { + super.writeBinary(writer); + + writer.writeLong(order); + } + + + /** {@inheritDoc} */ + @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException { + super.readBinary(reader); + + order = reader.readLong(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(JdbcOrderedBatchExecuteResult.class, this); + } +} 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 22522ad8b6ec2..3d5b86973282d 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 @@ -63,6 +63,9 @@ public class JdbcRequest extends ClientListenerRequestNoId implements JdbcRawBin /** Send a batch of a data from client to server. */ static final byte BULK_LOAD_BATCH = 13; + /** Ordered batch request. */ + static final byte BATCH_EXEC_ORDERED = 14; + /** Request type. */ private byte type; @@ -161,6 +164,11 @@ public static JdbcRequest readRequest(BinaryReaderExImpl reader) throws BinaryOb break; + case BATCH_EXEC_ORDERED: + req = new JdbcOrderedBatchExecuteRequest(); + + break; + default: throw new IgniteException("Unknown SQL listener request ID: [request ID=" + reqType + ']'); } 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 cf0e98b4a28d4..7ed3e1fa1e82a 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 @@ -28,19 +28,23 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.PriorityQueue; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.LockSupport; +import javax.cache.configuration.Factory; 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; import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.processors.authentication.AuthorizationContext; import org.apache.ignite.internal.processors.bulkload.BulkLoadAckClientParameters; import org.apache.ignite.internal.processors.bulkload.BulkLoadProcessor; -import org.apache.ignite.internal.processors.authentication.AuthorizationContext; import org.apache.ignite.internal.processors.cache.QueryCursorImpl; import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; import org.apache.ignite.internal.processors.cache.query.SqlFieldsQueryEx; @@ -60,6 +64,7 @@ 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; +import org.apache.ignite.internal.util.worker.GridWorker; import org.apache.ignite.lang.IgniteBiTuple; import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest.CMD_CONTINUE; @@ -68,6 +73,7 @@ import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext.VER_2_3_0; import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext.VER_2_4_0; import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.BATCH_EXEC; +import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.BATCH_EXEC_ORDERED; import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.BULK_LOAD_BATCH; import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_COLUMNS; import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_INDEXES; @@ -108,6 +114,15 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler { /** Current bulk load processors. */ private final ConcurrentHashMap bulkLoadRequests = new ConcurrentHashMap<>(); + /** Ordered batches queue. */ + private final PriorityQueue orderedBatchesQueue = new PriorityQueue<>(); + + /** Ordered batches mutex. */ + private final Object orderedBatchesMux = new Object(); + + /** Response sender. */ + private final JdbcResponseSender sender; + /** Automatic close of cursors. */ private final boolean autoCloseCursors; @@ -121,6 +136,7 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler { * Constructor. * @param ctx Context. * @param busyLock Shutdown latch. + * @param sender Results sender. * @param maxCursors Maximum allowed cursors. * @param distributedJoins Distributed joins flag. * @param enforceJoinOrder Enforce join order flag. @@ -132,14 +148,23 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler { * @param actx Authentication context. * @param protocolVer Protocol version. */ - public JdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, int maxCursors, + public JdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, + JdbcResponseSender sender, int maxCursors, boolean distributedJoins, boolean enforceJoinOrder, boolean collocated, boolean replicatedOnly, boolean autoCloseCursors, boolean lazy, boolean skipReducerOnUpdate, AuthorizationContext actx, ClientListenerProtocolVersion protocolVer) { this.ctx = ctx; + this.sender = sender; + + Factory orderedFactory = new Factory() { + @Override public GridWorker create() { + return new OrderedBatchWorker(); + } + }; this.cliCtx = new SqlClientContext( ctx, + orderedFactory, distributedJoins, enforceJoinOrder, collocated, @@ -189,6 +214,9 @@ public JdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, int case BATCH_EXEC: return executeBatch((JdbcBatchExecuteRequest)req); + case BATCH_EXEC_ORDERED: + return dispatchBatchOrdered((JdbcOrderedBatchExecuteRequest)req); + case META_TABLES: return getTablesMeta((JdbcMetaTablesRequest)req); @@ -221,6 +249,55 @@ public JdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, int } } + /** + * @param req Ordered batch request. + * @return Response. + */ + private ClientListenerResponse dispatchBatchOrdered(JdbcOrderedBatchExecuteRequest req) { + synchronized (orderedBatchesMux) { + orderedBatchesQueue.add(req); + + orderedBatchesMux.notify(); + } + + if (!cliCtx.isStreamOrdered()) + executeBatchOrdered(req); + + return null; + } + + /** + * @param req Ordered batch request. + * @return Response. + */ + private ClientListenerResponse executeBatchOrdered(JdbcOrderedBatchExecuteRequest req) { + try { + if (req.isLastStreamBatch()) + cliCtx.waitTotalProcessedOrderedRequests(req.order()); + + JdbcResponse resp = (JdbcResponse)executeBatch(req); + + if (resp.response() instanceof JdbcBatchExecuteResult) { + resp = new JdbcResponse( + new JdbcOrderedBatchExecuteResult((JdbcBatchExecuteResult)resp.response(), req.order())); + } + + sender.send(resp); + } catch (Exception e) { + U.error(null, "Error processing file batch", e); + + sender.send(new JdbcResponse(IgniteQueryErrorCode.UNKNOWN, "Server error: " + e)); + } + + synchronized (orderedBatchesMux) { + orderedBatchesQueue.poll(); + } + + cliCtx.orderedRequestProcessed(); + + return null; + } + /** * Processes a file batch sent from client as part of bulk load COPY command. * @@ -938,4 +1015,42 @@ private JdbcResponse exceptionToResult(Exception e) { else return new JdbcResponse(IgniteQueryErrorCode.UNKNOWN, e.toString()); } + + /** + * Ordered batch worker. + */ + private class OrderedBatchWorker extends GridWorker { + /** + * Constructor. + */ + OrderedBatchWorker() { + super(ctx.igniteInstanceName(), "ordered-batch", JdbcRequestHandler.this.log); + } + + /** {@inheritDoc} */ + @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException { + long nextBatchOrder = 0; + + while (true) { + if (!cliCtx.isStream()) + return; + + JdbcOrderedBatchExecuteRequest req; + + synchronized (orderedBatchesMux) { + req = orderedBatchesQueue.peek(); + + if (req == null || req.order() != nextBatchOrder) { + orderedBatchesMux.wait(); + + continue; + } + } + + executeBatchOrdered(req); + + nextBatchOrder++; + } + } + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResponseSender.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResponseSender.java new file mode 100644 index 0000000000000..128bcee889bc5 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResponseSender.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.odbc.jdbc; + +import org.apache.ignite.internal.processors.odbc.ClientListenerResponse; + +/** + * JDBC response result sender. + */ +public interface JdbcResponseSender { + /** + * Send response to the client. Used for asynchronous result send. + * @param resp JDBC response. + */ + public void send(ClientListenerResponse resp); +} 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 43631e96a6cbc..556917cf92516 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 @@ -68,6 +68,9 @@ public class JdbcResult implements JdbcRawBinarylizable { /** A request to send file from client to server. */ static final byte BULK_LOAD_ACK = 16; + /** A result of the processing ordered batch request. */ + static final byte BATCH_EXEC_ORDERED = 18; + /** Success status. */ private byte type; @@ -171,6 +174,16 @@ public static JdbcResult readResult(BinaryReaderExImpl reader) throws BinaryObje break; + case META_COLUMNS_V4: + res = new JdbcMetaColumnsResultV4(); + + break; + + case BATCH_EXEC_ORDERED: + res = new JdbcOrderedBatchExecuteResult(); + + break; + default: throw new IgniteException("Unknown SQL listener request ID: [request ID=" + resId + ']'); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java index e8c29329e992c..42dbae6c29e12 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java @@ -20,10 +20,14 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import javax.cache.configuration.Factory; import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.util.worker.GridWorker; +import org.apache.ignite.thread.IgniteThread; /** * Container for connection properties passed by various drivers (JDBC drivers, possibly ODBC) having notion of an @@ -53,6 +57,9 @@ public class SqlClientContext implements AutoCloseable { /** Skip reducer on update flag. */ private final boolean skipReducerOnUpdate; + /** Monitor. */ + private final Object mux = new Object(); + /** Allow overwrites for duplicate keys on streamed {@code INSERT}s. */ private boolean streamAllowOverwrite; @@ -65,14 +72,28 @@ public class SqlClientContext implements AutoCloseable { /** Auto flush frequency for streaming. */ private long streamFlushTimeout; + /** Stream ordered. */ + private boolean streamOrdered; + /** Streamers for various caches. */ - private Map> streamers; + private volatile Map> streamers; + + /** Ordered batch thread. */ + private IgniteThread orderedBatchThread; + + /** Ordered batch worker factory. */ + private Factory orderedBatchWorkerFactory; + + /** Count of the processed ordered batch requests. Used to wait end of processing all request before starts + * the processing the last request. */ + private long totalProcessedOrderedReqs; /** Logger. */ private final IgniteLogger log; /** * @param ctx Kernal context. + * @param orderedBatchWorkerFactory Ordered batch worker factory. * @param distributedJoins Distributed joins flag. * @param enforceJoinOrder Enforce join order flag. * @param collocated Collocated flag. @@ -80,9 +101,11 @@ public class SqlClientContext implements AutoCloseable { * @param lazy Lazy query execution flag. * @param skipReducerOnUpdate Skip reducer on update flag. */ - public SqlClientContext(GridKernalContext ctx, boolean distributedJoins, boolean enforceJoinOrder, + public SqlClientContext(GridKernalContext ctx, Factory orderedBatchWorkerFactory, + boolean distributedJoins, boolean enforceJoinOrder, boolean collocated, boolean replicatedOnly, boolean lazy, boolean skipReducerOnUpdate) { this.ctx = ctx; + this.orderedBatchWorkerFactory = orderedBatchWorkerFactory; this.distributedJoins = distributedJoins; this.enforceJoinOrder = enforceJoinOrder; this.collocated = collocated; @@ -100,37 +123,52 @@ public SqlClientContext(GridKernalContext ctx, boolean distributedJoins, boolean * @param flushFreq Flush frequency for streamers. * @param perNodeBufSize Per node streaming buffer size. * @param perNodeParOps Per node streaming parallel operations number. + * @param ordered Ordered stream flag. */ - public void enableStreaming(boolean allowOverwrite, long flushFreq, int perNodeBufSize, int perNodeParOps) { - if (isStream()) - return; + public void enableStreaming(boolean allowOverwrite, long flushFreq, int perNodeBufSize, + int perNodeParOps, boolean ordered) { + synchronized (mux) { + if (isStream()) + return; + + streamers = new HashMap<>(); - streamers = new HashMap<>(); + this.streamAllowOverwrite = allowOverwrite; + this.streamFlushTimeout = flushFreq; + this.streamNodeBufSize = perNodeBufSize; + this.streamNodeParOps = perNodeParOps; + this.streamOrdered = ordered; - this.streamAllowOverwrite = allowOverwrite; - this.streamFlushTimeout = flushFreq; - this.streamNodeBufSize = perNodeBufSize; - this.streamNodeParOps = perNodeParOps; + if (ordered) { + orderedBatchThread = new IgniteThread(orderedBatchWorkerFactory.create()); + + orderedBatchThread.start(); + } + } } /** * Turn off streaming on this client context - with closing all open streamers, if any. */ public void disableStreaming() { - if (!isStream()) - return; + synchronized (mux) { + if (!isStream()) + return; - Iterator> it = streamers.values().iterator(); + Iterator> it = streamers.values().iterator(); - while (it.hasNext()) { - IgniteDataStreamer streamer = it.next(); + while (it.hasNext()) { + IgniteDataStreamer streamer = it.next(); - U.close(streamer, log); + U.close(streamer, log); - it.remove(); - } + it.remove(); + } + + streamers = null; - streamers = null; + orderedBatchThread = null; + } } /** @@ -179,7 +217,18 @@ public boolean isSkipReducerOnUpdate() { * @return Streaming state flag (on or off). */ public boolean isStream() { - return streamers != null; + synchronized (mux) { + return streamers != null; + } + } + + /** + * @return Stream ordered flag. + */ + public boolean isStreamOrdered() { + synchronized (mux) { + return streamOrdered; + } } /** @@ -187,29 +236,59 @@ public boolean isStream() { * @return Streamer for given cache. */ public IgniteDataStreamer streamerForCache(String cacheName) { - if (streamers == null) - return null; + synchronized (mux) { + if (streamers == null) + return null; - IgniteDataStreamer res = streamers.get(cacheName); + IgniteDataStreamer res = streamers.get(cacheName); - if (res != null) - return res; + if (res != null) + return res; - res = ctx.grid().dataStreamer(cacheName); + res = ctx.grid().dataStreamer(cacheName); - res.autoFlushFrequency(streamFlushTimeout); + res.autoFlushFrequency(streamFlushTimeout); - res.allowOverwrite(streamAllowOverwrite); + res.allowOverwrite(streamAllowOverwrite); - if (streamNodeBufSize > 0) - res.perNodeBufferSize(streamNodeBufSize); + if (streamNodeBufSize > 0) + res.perNodeBufferSize(streamNodeBufSize); - if (streamNodeParOps > 0) - res.perNodeParallelOperations(streamNodeParOps); + if (streamNodeParOps > 0) + res.perNodeParallelOperations(streamNodeParOps); - streamers.put(cacheName, res); + streamers.put(cacheName, res); - return res; + return res; + } + } + + /** + * Waits when total processed ordered requests count to be equal to specified value. + * @param total Expected total processed request. + */ + public void waitTotalProcessedOrderedRequests(long total) { + synchronized (mux) { + while (totalProcessedOrderedReqs < total) { + try { + mux.wait(); + } + catch (InterruptedException e) { + throw new IgniteException("Waiting for end of processing the last batch is interrupted", e); + } + } + } + } + + /** + * + */ + public void orderedRequestProcessed() { + synchronized (mux) { + totalProcessedOrderedReqs++; + + mux.notify(); + } } /** {@inheritDoc} */ 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 0fd08f48c4340..be7648249365e 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 @@ -185,6 +185,9 @@ public class SqlKeyword { /** Keyword: ON. */ public static final String ON = "ON"; + /** Keyword: ORDERED. */ + public static final String ORDERED = "ORDERED"; + /** Keyword: PER_NODE_PARALLEL_OPERATIONS. */ public static final String PER_NODE_PARALLEL_OPERATIONS = "PER_NODE_PARALLEL_OPERATIONS"; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlSetStreamingCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlSetStreamingCommand.java index c492c61d0ce41..42f17d572dd8c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlSetStreamingCommand.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/command/SqlSetStreamingCommand.java @@ -53,6 +53,9 @@ public class SqlSetStreamingCommand implements SqlCommand { /** Streamer flush timeout. */ private long flushFreq; + /** Ordered streamer. */ + private boolean ordered; + /** {@inheritDoc} */ @Override public SqlCommand parse(SqlLexer lex) { turnOn = parseBoolean(lex); @@ -116,6 +119,15 @@ public class SqlSetStreamingCommand implements SqlCommand { break; + case SqlKeyword.ORDERED: + lex.shift(); + + checkOffLast(lex); + + ordered = true; + + break; + default: return this; } @@ -179,6 +191,13 @@ public long flushFrequency() { return flushFreq; } + /** + * @return {@code true} if the streamer keep the order of the statements. Otherwise returns {@code false}. + */ + public boolean isOrdered() { + return ordered; + } + /** {@inheritDoc} */ @Override public String schemaName() { return null; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserSetStreamingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserSetStreamingSelfTest.java index 65bb599a58589..7e699f6adabb7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserSetStreamingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserSetStreamingSelfTest.java @@ -28,18 +28,25 @@ public class SqlParserSetStreamingSelfTest extends SqlParserAbstractSelfTest { * */ public void testParseSetStreaming() { - parseValidate("set streaming on", true, false, 2048, 0, 0, 0); - parseValidate("set streaming 1", true, false, 2048, 0, 0, 0); - parseValidate("set streaming off", false, false, 2048, 0, 0, 0); - parseValidate("set streaming 0", false, false, 2048, 0, 0, 0); - parseValidate("set streaming on batch_size 100", true, false, 100, 0, 0, 0); - parseValidate("set streaming on flush_frequency 500", true, false, 2048, 0, 0, 500); - parseValidate("set streaming on per_node_buffer_size 100", true, false, 2048, 0, 100, 0); - parseValidate("set streaming on per_node_parallel_operations 4", true, false, 2048, 4, 0, 0); - parseValidate("set streaming on allow_overwrite on", true, true, 2048, 0, 0, 0); - parseValidate("set streaming on allow_overwrite off", true, false, 2048, 0, 0, 0); + parseValidate("set streaming on", true, false, 2048, 0, 0, 0, false); + parseValidate("set streaming 1", true, false, 2048, 0, 0, 0, false); + parseValidate("set streaming off", false, false, 2048, 0, 0, 0, false); + parseValidate("set streaming 0", false, false, 2048, 0, 0, 0, false); + parseValidate("set streaming on batch_size 100", true, false, 100, 0, 0, 0, false); + parseValidate("set streaming on flush_frequency 500", true, false, 2048, 0, 0, 500, false); + parseValidate("set streaming on per_node_buffer_size 100", true, false, 2048, 0, 100, 0, false); + parseValidate("set streaming on per_node_parallel_operations 4", true, false, 2048, 4, 0, 0, false); + parseValidate("set streaming on allow_overwrite on", true, true, 2048, 0, 0, 0, false); + parseValidate("set streaming on allow_overwrite off", true, false, 2048, 0, 0, 0, false); parseValidate("set streaming on per_node_buffer_size 50 flush_frequency 500 " + - "per_node_parallel_operations 4 allow_overwrite on batch_size 100", true, true, 100, 4, 50, 500); + "per_node_parallel_operations 4 allow_overwrite on batch_size 100", true, true, 100, 4, 50, 500, false); + + parseValidate("set streaming on ordered", true, false, 2048, 0, 0, 0, true); + parseValidate("set streaming 1 ordered", true, false, 2048, 0, 0, 0, true); + parseValidate("set streaming on batch_size 100 ordered", true, false, 100, 0, 0, 0, true); + parseValidate("set streaming on per_node_buffer_size 50 flush_frequency 500 " + + "per_node_parallel_operations 4 allow_overwrite on batch_size 100 ordered", true, true, 100, 4, 50, 500, true); + assertParseError(QueryUtils.DFLT_SCHEMA, "set", "Failed to parse SQL statement \"set[*]\": Unexpected end of command (expected: \"STREAMING\")"); @@ -84,6 +91,10 @@ public void testParseSetStreaming() { "Failed to parse SQL statement \"set streaming off [*]allow_overwrite\": Unexpected token: " + "\"ALLOW_OVERWRITE\""); + assertParseError(QueryUtils.DFLT_SCHEMA, "set streaming off ordered", + "Failed to parse SQL statement \"set streaming off [*]ordered\": Unexpected token: " + + "\"ORDERED\""); + assertParseError(QueryUtils.DFLT_SCHEMA, "set streaming off batch_size", "Failed to parse SQL statement \"set streaming off [*]batch_size\": Unexpected token: " + "\"BATCH_SIZE\""); @@ -102,6 +113,7 @@ public void testParseSetStreaming() { assertParseError(QueryUtils.DFLT_SCHEMA, "set streaming off table", "Failed to parse SQL statement \"set streaming off [*]table\": Unexpected token: \"TABLE\""); + } /** @@ -114,9 +126,10 @@ public void testParseSetStreaming() { * @param expParOps Expected per-node parallael operations. * @param expBufSize Expected per node buffer size. * @param expFlushFreq Expected flush frequency. + * @param ordered Ordered stream flag. */ private static void parseValidate(String sql, boolean expOn, boolean expAllowOverwrite, int expBatchSize, - int expParOps, int expBufSize, long expFlushFreq) { + int expParOps, int expBufSize, long expFlushFreq, boolean ordered) { SqlSetStreamingCommand cmd = (SqlSetStreamingCommand)new SqlParser(QueryUtils.DFLT_SCHEMA, sql).nextCommand(); assertEquals(expOn, cmd.isTurnOn()); @@ -130,5 +143,7 @@ private static void parseValidate(String sql, boolean expOn, boolean expAllowOve assertEquals(expBufSize, cmd.perNodeBufferSize()); assertEquals(expFlushFreq, cmd.flushFrequency()); + + assertEquals(ordered, cmd.isOrdered()); } } 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 7b1a093a1ab95..2d019d3d64322 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 @@ -1561,7 +1561,7 @@ else if (cmd instanceof SqlSetStreamingCommand) { if (on) cliCtx.enableStreaming(setCmd.allowOverwrite(), setCmd.flushFrequency(), - setCmd.perNodeBufferSize(), setCmd.perNodeParallelOperations()); + setCmd.perNodeBufferSize(), setCmd.perNodeParallelOperations(), setCmd.isOrdered()); else cliCtx.disableStreaming(); diff --git a/modules/yardstick/config/ignite-localhost-config.xml b/modules/yardstick/config/ignite-localhost-config.xml index 3e57f4f0edf08..9b868506c64e2 100644 --- a/modules/yardstick/config/ignite-localhost-config.xml +++ b/modules/yardstick/config/ignite-localhost-config.xml @@ -27,8 +27,6 @@ - - diff --git a/modules/yardstick/config/upload/benchmark-jdbc-thin-streaming.properties b/modules/yardstick/config/upload/benchmark-jdbc-thin-streaming.properties new file mode 100644 index 0000000000000..e0def4b62fc03 --- /dev/null +++ b/modules/yardstick/config/upload/benchmark-jdbc-thin-streaming.properties @@ -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. +# + +# +# Benchmarks for data upload in inmemory mode (persistence disabled). +# + +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 + + +# 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="\ +-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} -dn InsertBenchmark -sn IgniteNode -ds ${ver}sql-upload-inmemory-streaming-batch-256-order-on \ + --upload-rows 1000000 -cl --clientNodesAfterId 0 \ + --use-streaming true \ + --streamer-local-batch-size 256 \ +, \ +-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} -dn InsertBenchmark -sn IgniteNode -ds ${ver}sql-upload-inmemory-streaming-batch-1024-order-on \ + --upload-rows 1000000 -cl --clientNodesAfterId 0 \ + --use-streaming true \ + --streamer-local-batch-size 1024 \ +, \ +-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} -dn InsertBenchmark -sn IgniteNode -ds ${ver}sql-upload-inmemory-streaming-batch-4096-order-on \ + --upload-rows 1000000 -cl --clientNodesAfterId 0 \ + --use-streaming true \ + --streamer-local-batch-size 4096 \ +, \ +-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} -dn InsertBenchmark -sn IgniteNode -ds ${ver}sql-upload-inmemory-streaming-batch-256-order-off \ + --upload-rows 1000000 -cl --clientNodesAfterId 0 \ + --use-streaming true \ + --streamer-ordered false \ + --streamer-local-batch-size 256 \ +, \ +-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} -dn InsertBenchmark -sn IgniteNode -ds ${ver}sql-upload-inmemory-streaming-batch-1024-order-off \ + --upload-rows 1000000 -cl --clientNodesAfterId 0 \ + --use-streaming true \ + --streamer-ordered false \ + --streamer-local-batch-size 1024 \ +, \ +-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} -dn InsertBenchmark -sn IgniteNode -ds ${ver}sql-upload-inmemory-streaming-batch-4096-order-off \ + --upload-rows 1000000 -cl --clientNodesAfterId 0 \ + --use-streaming true \ + --streamer-ordered false \ + --streamer-local-batch-size 4096 \ +, \ +" diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/StreamerParams.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/StreamerParams.java index 995201d9940ba..f6e8014e6ec08 100644 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/StreamerParams.java +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/StreamerParams.java @@ -43,4 +43,9 @@ public interface StreamerParams { * @return Allow overwrite flag. */ @Nullable public Boolean streamerAllowOverwrite(); -} + + /** + * @return Ordered flag. + */ + public boolean streamerOrdered(); +} \ No newline at end of file diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/UploadBenchmarkArguments.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/UploadBenchmarkArguments.java index 635ba6bde5899..7a1b1163a1782 100644 --- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/UploadBenchmarkArguments.java +++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/upload/UploadBenchmarkArguments.java @@ -74,6 +74,10 @@ public class UploadBenchmarkArguments implements StreamerParams { description = "Streamer benchmarks only: set allowOverwrite streamer parameter.") private Boolean streamerAllowOverwrite = null; + @Parameter(names = {"--streamer-ordered"}, arity = 1, + description = "Streamer benchmarks only: set streamer ordered flag.") + private boolean streamerOrdered = true; + /** How many rows to upload during warmup. */ @Parameter(names = {"--upload-warmup-rows"}) private long warmupRowsCnt = 3_000_000; @@ -111,14 +115,14 @@ public List uploadJdbcParams() { /** * @return Value for {@link IgniteDataStreamer#perNodeBufferSize(int)}. */ - @Nullable public Integer streamerPerNodeBufferSize() { + @Override @Nullable public Integer streamerPerNodeBufferSize() { return streamerNodeBufSize; } /** * @return Value for {@link IgniteDataStreamer#perNodeParallelOperations(int)}. */ - @Nullable public Integer streamerPerNodeParallelOperations() { + @Override @Nullable public Integer streamerPerNodeParallelOperations() { return streamerNodeParOps; } @@ -128,17 +132,24 @@ public List uploadJdbcParams() { * or set STREAMING sql command parameter.
    * If set to 1, {@link IgniteDataStreamer#addData(Object, Object)} method will be used. */ - @Nullable public Integer streamerLocalBatchSize() { + @Override @Nullable public Integer streamerLocalBatchSize() { return streamerLocBatchSize; } /** * Bypass corresponding parameter to streamer. */ - @Nullable public Boolean streamerAllowOverwrite() { + @Override @Nullable public Boolean streamerAllowOverwrite() { return streamerAllowOverwrite; } + /** + * Bypass corresponding parameter to streamer. + */ + @Override public boolean streamerOrdered() { + return streamerOrdered; + } + /** * See {@link #warmupRowsCnt}. */ 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 87efa83bc6a3d..3ff4cb4166cc8 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 @@ -208,6 +208,8 @@ public String turnOnStreaming(StreamerParams p) { if (p.streamerPerNodeBufferSize() != null) cmd.append(" PER_NODE_BUFFER_SIZE ").append(p.streamerPerNodeBufferSize()); + cmd.append(" ORDERED ").append(p.streamerOrdered() ? "ON" : "OFF"); + return cmd.append(';').toString(); } } From 53e7eefa5c2612b51b8f85c8a607d589b7172bfe Mon Sep 17 00:00:00 2001 From: devozerov Date: Fri, 11 May 2018 18:00:07 +0300 Subject: [PATCH 0171/1463] IGNITE-7999: JDBC Thin Driver: added unordered streaming mode. This closes #3789. --- .../org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java | 2 +- .../ignite/internal/processors/odbc/jdbc/JdbcResult.java | 5 ----- 2 files changed, 1 insertion(+), 6 deletions(-) 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 44c19847b4ddb..37223c4bde887 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 @@ -435,7 +435,7 @@ void sendBatchRequestNoWaitResponse(JdbcOrderedBatchExecuteRequest req) throws I } try { - if (!igniteVer.greaterThanEqual(2, 5, 0)) { + if (!igniteVer.greaterThanEqual(2, 4, 6)) { throw new SQLException("Streaming without response doesn't supported by server [driverProtocolVer=" + CURRENT_VER + ", remoteNodeVer=" + igniteVer + ']', SqlStateCode.INTERNAL_ERROR); } 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 556917cf92516..9467ab57cb96a 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 @@ -174,11 +174,6 @@ public static JdbcResult readResult(BinaryReaderExImpl reader) throws BinaryObje break; - case META_COLUMNS_V4: - res = new JdbcMetaColumnsResultV4(); - - break; - case BATCH_EXEC_ORDERED: res = new JdbcOrderedBatchExecuteResult(); From 17ac5fe9f736b9e166fb0b71ea8d83a4f32e7082 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Fri, 11 May 2018 18:02:08 +0300 Subject: [PATCH 0172/1463] IGNITE-7999: JDBC Thin Driver: added unordered streaming mode. This closes #3789. --- .../org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 37223c4bde887..44c19847b4ddb 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 @@ -435,7 +435,7 @@ void sendBatchRequestNoWaitResponse(JdbcOrderedBatchExecuteRequest req) throws I } try { - if (!igniteVer.greaterThanEqual(2, 4, 6)) { + if (!igniteVer.greaterThanEqual(2, 5, 0)) { throw new SQLException("Streaming without response doesn't supported by server [driverProtocolVer=" + CURRENT_VER + ", remoteNodeVer=" + igniteVer + ']', SqlStateCode.INTERNAL_ERROR); } From fd26eb3299ea5f46acb2a9052f06f550e8bd6949 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 11 May 2018 18:45:38 +0300 Subject: [PATCH 0173/1463] IGNITE-5874 Store TTL expire times in B+ tree on per-partition basis - Fixes #3231. Signed-off-by: Ivan Rakov (cherry picked from commit 89c7757) --- .../JettyRestProcessorAbstractSelfTest.java | 21 - .../PdsWithTtlCompatibilityTest.java | 191 ++++++++ .../IgniteCompatibilityBasicTestSuite.java | 3 + .../apache/ignite/IgniteSystemProperties.java | 11 + .../MetaPageUpdateLastAllocatedIndex.java | 6 +- .../processors/cache/CacheGroupContext.java | 16 +- .../processors/cache/GridCacheMapEntry.java | 41 +- .../cache/IgniteCacheOffheapManager.java | 21 +- .../cache/IgniteCacheOffheapManagerImpl.java | 128 +++--- .../dht/GridDhtLocalPartition.java | 156 +++---- .../dht/GridDhtPartitionsStateValidator.java | 1 + .../GridDhtPartitionsExchangeFuture.java | 14 +- .../persistence/GridCacheOffheapManager.java | 415 +++++++++++++----- .../UpgradePendingTreeToPerPartitionTask.java | 380 ++++++++++++++++ .../cache/persistence/tree/io/PageIO.java | 11 +- .../tree/io/PagePartitionMetaIO.java | 35 +- .../tree/io/PagePartitionMetaIOV2.java | 90 ++++ ...CachePartitionsStateValidatorSelfTest.java | 10 +- .../IgnitePdsContinuousRestartTest.java | 89 +++- .../IgnitePdsContinuousRestartTest2.java | 281 ------------ ...ContinuousRestartTestWithExpiryPolicy.java | 67 +++ ...IgniteBaselineAbstractFullApiSelfTest.java | 9 +- .../persistence/db/IgnitePdsWithTtlTest.java | 197 +++++++++ .../ignite/testsuites/IgnitePdsTestSuite.java | 2 + .../testsuites/IgnitePdsTestSuite2.java | 5 +- 25 files changed, 1582 insertions(+), 618 deletions(-) create mode 100644 modules/compatibility/src/test/java/org/apache/ignite/compatibility/PdsWithTtlCompatibilityTest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/migration/UpgradePendingTreeToPerPartitionTask.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PagePartitionMetaIOV2.java delete mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest2.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTestWithExpiryPolicy.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWithTtlTest.java diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java index 0285f3af530ed..96391e95a16b8 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java @@ -978,27 +978,10 @@ public void testPut() throws Exception { assertCacheOperation(ret, true); } - /** */ - private void failIgnite_5874() { - DataStorageConfiguration dsCfg = ignite(0).configuration().getDataStorageConfiguration(); - - if (dsCfg.getDefaultDataRegionConfiguration().isPersistenceEnabled()) - fail("IGNITE-5874"); - - if (!F.isEmpty(dsCfg.getDataRegionConfigurations())) { - for (DataRegionConfiguration dataRegCfg : dsCfg.getDataRegionConfigurations()) { - if (dataRegCfg.isPersistenceEnabled()) - fail("IGNITE-5874"); - } - } - } - /** * @throws Exception If failed. */ public void testPutWithExpiration() throws Exception { - failIgnite_5874(); - String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_PUT, "key", "putKey", "val", "putVal", @@ -1035,8 +1018,6 @@ public void testAdd() throws Exception { * @throws Exception If failed. */ public void testAddWithExpiration() throws Exception { - failIgnite_5874(); - String ret = content(DEFAULT_CACHE_NAME, GridRestCommand.CACHE_ADD, "key", "addKey", "val", "addVal", @@ -1176,8 +1157,6 @@ public void testReplace() throws Exception { * @throws Exception If failed. */ public void testReplaceWithExpiration() throws Exception { - failIgnite_5874(); - jcache().put("replaceKey", "replaceVal"); assertEquals("replaceVal", jcache().get("replaceKey")); diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/PdsWithTtlCompatibilityTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/PdsWithTtlCompatibilityTest.java new file mode 100644 index 0000000000000..f3649f6de83cc --- /dev/null +++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/PdsWithTtlCompatibilityTest.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.compatibility; + +import java.util.Collection; +import java.util.concurrent.TimeUnit; +import javax.cache.Cache; +import javax.cache.expiry.AccessedExpiryPolicy; +import javax.cache.expiry.Duration; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.compatibility.persistence.IgnitePersistenceCompatibilityAbstractTest; +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.PersistentStoreConfiguration; +import org.apache.ignite.configuration.WALMode; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.processors.cache.GridCacheAbstractFullApiSelfTest; +import org.apache.ignite.internal.processors.cache.persistence.migration.UpgradePendingTreeToPerPartitionTask; +import org.apache.ignite.internal.util.typedef.PA; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.testframework.GridTestUtils; + +/** + * Test PendingTree upgrading to per-partition basis. Test fill cache with persistence enabled and with ExpirePolicy + * configured on ignite-2.1 version and check if entries will be correctly expired when a new version node started. + * + * Note: Test for ignite-2.3 version will always fails due to entry ttl update fails with assertion on checkpoint lock + * check. + */ +public class PdsWithTtlCompatibilityTest extends IgnitePersistenceCompatibilityAbstractTest { + /** */ + static final String TEST_CACHE_NAME = PdsWithTtlCompatibilityTest.class.getSimpleName(); + + /** */ + static final int DURATION_SEC = 10; + + /** */ + private static final int ENTRIES_CNT = 100; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setPeerClassLoadingEnabled(false); + + cfg.setDataStorageConfiguration( + new DataStorageConfiguration() + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration() + .setMaxSize(32L * 1024 * 1024) + .setPersistenceEnabled(true) + ).setWalMode(WALMode.LOG_ONLY)); + + return cfg; + } + + /** + * Tests opportunity to read data from previous Ignite DB version. + * + * @throws Exception If failed. + */ + public void testNodeStartByOldVersionPersistenceData_2_1() throws Exception { + doTestStartupWithOldVersion("2.1.0"); + } + + /** + * Tests opportunity to read data from previous Ignite DB version. + * + * @param igniteVer 3-digits version of ignite + * @throws Exception If failed. + */ + protected void doTestStartupWithOldVersion(String igniteVer) throws Exception { + try { + startGrid(1, igniteVer, new ConfigurationClosure(), new PostStartupClosure()); + + stopAllGrids(); + + IgniteEx ignite = startGrid(0); + + assertEquals(1, ignite.context().discovery().topologyVersion()); + + ignite.active(true); + + validateResultingCacheData(ignite, ignite.cache(TEST_CACHE_NAME)); + } + finally { + stopAllGrids(); + } + } + + /** + * @param cache to be filled by different keys and values. Results may be validated in {@link + * #validateResultingCacheData(Ignite, IgniteCache)}. + */ + public static void saveCacheData(Cache cache) { + for (int i = 0; i < ENTRIES_CNT; i++) + cache.put(i, "data-" + i); + + //Touch + for (int i = 0; i < ENTRIES_CNT; i++) + assertNotNull(cache.get(i)); + } + + /** + * Asserts cache contained all expected values as it was saved before. + * + * @param cache cache should be filled using {@link #saveCacheData(Cache)}. + */ + public static void validateResultingCacheData(Ignite ignite, + IgniteCache cache) throws IgniteInterruptedCheckedException { + + final long expireTime = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(DURATION_SEC + 1); + + final IgniteFuture> future = ignite.compute().broadcastAsync(new UpgradePendingTreeToPerPartitionTask()); + + GridTestUtils.waitForCondition(new PA() { + @Override public boolean apply() { + return future.isDone() && expireTime < System.currentTimeMillis(); + } + }, TimeUnit.SECONDS.toMillis(DURATION_SEC + 2)); + + for (Boolean res : future.get()) + assertTrue(res); + + for (int i = 0; i < ENTRIES_CNT; i++) + assertNull(cache.get(i)); + } + + /** */ + public static class ConfigurationClosure implements IgniteInClosure { + /** {@inheritDoc} */ + @Override public void apply(IgniteConfiguration cfg) { + cfg.setLocalHost("127.0.0.1"); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + disco.setIpFinder(GridCacheAbstractFullApiSelfTest.LOCAL_IP_FINDER); + + cfg.setDiscoverySpi(disco); + + cfg.setPeerClassLoadingEnabled(false); + + cfg.setPersistentStoreConfiguration(new PersistentStoreConfiguration().setWalMode(WALMode.LOG_ONLY)); + } + } + + /** */ + public static class PostStartupClosure implements IgniteInClosure { + /** {@inheritDoc} */ + @Override public void apply(Ignite ignite) { + ignite.active(true); + + CacheConfiguration cacheCfg = new CacheConfiguration<>(); + cacheCfg.setName(TEST_CACHE_NAME); + cacheCfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + cacheCfg.setBackups(1); + cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + cacheCfg.setExpiryPolicyFactory(AccessedExpiryPolicy.factoryOf(new Duration(TimeUnit.SECONDS, DURATION_SEC))); + cacheCfg.setEagerTtl(true); + cacheCfg.setGroupName("myGroup"); + + IgniteCache cache = ignite.createCache(cacheCfg); + + saveCacheData(cache); + + ignite.active(false); + } + } +} diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java index eaa38afdd6d61..fcfd5a7939ca4 100644 --- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java +++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java @@ -19,6 +19,7 @@ import junit.framework.TestSuite; import org.apache.ignite.compatibility.persistence.DummyPersistenceCompatibilityTest; +import org.apache.ignite.compatibility.PdsWithTtlCompatibilityTest; import org.apache.ignite.compatibility.persistence.FoldersReuseCompatibilityTest; import org.apache.ignite.compatibility.persistence.IgniteUuidCompatibilityTest; import org.apache.ignite.compatibility.persistence.MigratingToWalV2SerializerWithCompactionTest; @@ -36,6 +37,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(DummyPersistenceCompatibilityTest.class); + suite.addTestSuite(PdsWithTtlCompatibilityTest.class); + suite.addTestSuite(FoldersReuseCompatibilityTest.class); suite.addTestSuite(MigratingToWalV2SerializerWithCompactionTest.class); 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 008974c6ff68e..727e8092b7cf7 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -887,6 +887,17 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_DISABLE_WAL_DURING_REBALANCING = "IGNITE_DISABLE_WAL_DURING_REBALANCING"; + /** + * When set to {@code true}, Ignite will skip partitions sizes check on partition validation after rebalance has finished. + * Partitions sizes may differs on nodes when Expiry Policy is in use and it is ok due to lazy entry eviction mechanics. + * + * There is no need to disable partition size validation either in normal case or when expiry policy is configured for cache. + * But it should be disabled manually when policy is used on per entry basis to hint Ignite to skip this check. + * + * Default is {@code false}. + */ + public static final String IGNITE_SKIP_PARTITION_SIZE_VALIDATION = "IGNITE_SKIP_PARTITION_SIZE_VALIDATION"; + /** * Enforces singleton. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java index 39f6a0335b1bd..324227be6fc4c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java @@ -41,9 +41,11 @@ public MetaPageUpdateLastAllocatedIndex(int grpId, long pageId, int lastAllocate /** {@inheritDoc} */ @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException { - assert PageIO.getType(pageAddr) == PageIO.T_META || PageIO.getType(pageAddr) == PageIO.T_PART_META; + int type = PageIO.getType(pageAddr); - PageMetaIO io = PageMetaIO.VERSIONS.forVersion(PageIO.getVersion(pageAddr)); + assert type == PageIO.T_META || type == PageIO.T_PART_META; + + PageMetaIO io = PageIO.getPageIO(type, PageIO.getVersion(pageAddr)); io.setLastAllocatedPageCount(pageAddr, lastAllocatedIdx); } 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 5f750d5996416..d1bdbb6342c3f 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 @@ -39,9 +39,9 @@ import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtAffinityAssignmentRequest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtAffinityAssignmentResponse; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionsEvictor; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopologyImpl; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionsEvictor; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader; import org.apache.ignite.internal.processors.cache.persistence.DataRegion; import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager; @@ -313,7 +313,7 @@ private void addCacheContext(GridCacheContext cctx) { drEnabled = true; this.caches = caches; - } + } /** * @param cctx Cache context. @@ -372,8 +372,8 @@ public GridCacheContext singleCacheContext() { List caches = this.caches; assert !sharedGroup() && caches.size() == 1 : - "stopping=" + ctx.kernalContext().isStopping() + ", groupName=" + ccfg.getGroupName() + - ", caches=" + caches; + "stopping=" + ctx.kernalContext().isStopping() + ", groupName=" + ccfg.getGroupName() + + ", caches=" + caches; return caches.get(0); } @@ -434,6 +434,7 @@ public void addRebalanceEvent(int part, int type, ClusterNode discoNode, int dis } } } + /** * Adds partition unload event. * @@ -513,13 +514,6 @@ public boolean queriesEnabled() { return qryEnabled; } - /** - * @return {@code True} if fast eviction is allowed. - */ - public boolean allowFastEviction() { - return persistenceEnabled() && !queriesEnabled(); - } - /** * @return {@code True} in case replication is enabled. */ 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 9f3686aad47ec..767c314773055 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.NodeStoppingException; import org.apache.ignite.internal.pagemem.wal.StorageException; import org.apache.ignite.internal.pagemem.wal.WALPointer; import org.apache.ignite.internal.pagemem.wal.record.DataEntry; @@ -401,7 +402,7 @@ protected GridDhtLocalPartition localPartition() { checkObsolete(); if (isStartVersion() && ((flags & IS_UNSWAPPED_MASK) == 0)) { - assert row == null || row.key() == key: "Unexpected row key"; + assert row == null || row.key() == key : "Unexpected row key"; CacheDataRow read = row == null ? cctx.offheap().read(this) : row; @@ -1411,7 +1412,7 @@ private boolean notifyContinuousQueries(@Nullable IgniteInternalTx tx) { if (readThrough && needVal && old == null && (cctx.readThrough() && (op == GridCacheOperation.TRANSFORM || cctx.loadPreviousValue()))) { - old0 = readThrough(null, key, false, subjId, taskName); + old0 = readThrough(null, key, false, subjId, taskName); old = cctx.toCacheObject(old0); @@ -2462,7 +2463,14 @@ private void updateTtl(long ttl) throws IgniteCheckedException, GridCacheEntryRe ttlAndExpireTimeExtras(ttl, expireTime); - storeValue(val, expireTime, ver, null); + cctx.shared().database().checkpointReadLock(); + + try { + storeValue(val, expireTime, ver, null); + } + finally { + cctx.shared().database().checkpointReadUnlock(); + } } /** @@ -3108,7 +3116,8 @@ private GridCacheVersion nextVersion() { GridCacheMvcc mvcc = mvccExtras(); return mvcc != null && mvcc.isLocallyOwnedByIdOrThread(lockVer, threadId); - } finally { + } + finally { unlockEntry(); } } @@ -3347,6 +3356,10 @@ private GridCacheVersion nextVersion() { obsolete = true; } } + catch (NodeStoppingException ignore) { + if (log.isDebugEnabled()) + log.warning("Node is stopping while removing expired value.", ignore); + } catch (IgniteCheckedException e) { U.error(log, "Failed to clean up expired cache entry: " + this, e); } @@ -3406,7 +3419,14 @@ private boolean onExpired(CacheObject expiredVal, GridCacheVersion obsoleteVer) if (log.isTraceEnabled()) log.trace("onExpired clear [key=" + key + ", entry=" + System.identityHashCode(this) + ']'); - removeValue(); + cctx.shared().database().checkpointReadLock(); + + try { + removeValue(); + } + finally { + cctx.shared().database().checkpointReadUnlock(); + } if (cctx.events().isRecordable(EVT_CACHE_OBJECT_EXPIRED)) { cctx.events().addEvent(partition(), @@ -3586,8 +3606,9 @@ protected boolean storeValue(@Nullable CacheObject val, * @param ver New entry version. * @param oldRow Old row if available. * @param predicate Optional predicate. - * @throws IgniteCheckedException If update failed. + * * @return {@code True} if storage was modified. + * @throws IgniteCheckedException If update failed. */ protected boolean storeValue( @Nullable CacheObject val, @@ -3599,7 +3620,7 @@ protected boolean storeValue( UpdateClosure closure = new UpdateClosure(this, val, ver, expireTime, predicate); - cctx.offheap().invoke(cctx, key, localPartition(), closure); + cctx.offheap().invoke(cctx, key, localPartition(), closure); return closure.treeOp != IgniteTree.OperationType.NOOP; } @@ -4051,7 +4072,7 @@ protected final void deletedUnlocked(boolean deleted) { } /** - * Increments public size of map. + * Increments public size of map. */ protected void incrementMapPublicSize() { GridDhtLocalPartition locPart = localPartition(); @@ -4782,7 +4803,7 @@ private void initResultOnCancelUpdate(@Nullable CacheObject storeLoadedVal, bool needUpdate = true; } - else if (updateExpireTime && expiryPlc != null && entry.val != null){ + else if (updateExpireTime && expiryPlc != null && entry.val != null) { long ttl = expiryPlc.forAccess(); if (ttl != CU.TTL_NOT_CHANGED) { @@ -4929,7 +4950,7 @@ else if (interceptorVal != updated0) { if (entry.val == null) { boolean new0 = entry.isStartVersion(); - assert entry.deletedUnlocked() || new0 || entry.isInternal(): "Invalid entry [entry=" + entry + + assert entry.deletedUnlocked() || new0 || entry.isInternal() : "Invalid entry [entry=" + entry + ", locNodeId=" + cctx.localNodeId() + ']'; if (!new0 && !entry.isInternal()) 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 a12c0334912a5..fa25412a81b17 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 @@ -21,12 +21,13 @@ 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.preloader.IgniteDhtDemandedPartitionsMap; 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.persistence.CacheDataRow; 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.version.GridCacheVersion; import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; import org.apache.ignite.internal.util.GridAtomicLong; @@ -47,7 +48,7 @@ public interface IgniteCacheOffheapManager { * @param grp Cache group. * @throws IgniteCheckedException If failed. */ - public void start(GridCacheSharedContext ctx, CacheGroupContext grp) throws IgniteCheckedException;; + public void start(GridCacheSharedContext ctx, CacheGroupContext grp) throws IgniteCheckedException; /** * @param cctx Cache context. @@ -142,6 +143,8 @@ public interface IgniteCacheOffheapManager { /** * @param cctx Cache context. * @param c Closure. + * @param amount Limit of processed entries by single call, {@code -1} for no limit. + * @return {@code True} if unprocessed expired entries remains. * @throws IgniteCheckedException If failed. */ public boolean expire(GridCacheContext cctx, IgniteInClosure2X c, int amount) @@ -167,9 +170,9 @@ public void invoke(GridCacheContext cctx, KeyCacheObject key, GridDhtLocalPartit /** * @param cctx Cache context. - * @param key Key. - * @param val Value. - * @param ver Version. + * @param key Key. + * @param val Value. + * @param ver Version. * @param expireTime Expire time. * @param oldRow Old row if available. * @param part Partition. @@ -537,5 +540,13 @@ public GridCursor cursor(int cacheId, KeyCacheObject low * @param rowCacheCleaner Rows cache cleaner. */ public void setRowCacheCleaner(GridQueryRowCacheCleaner rowCacheCleaner); + + /** + * Return PendingTree for data store. + * + * @return PendingTree instance. + * @throws IgniteCheckedException + */ + PendingEntriesTree pendingTree(); } } 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 5c78eb5b4b4a4..bf0de02cb3440 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,16 +101,16 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager protected final ConcurrentMap partDataStores = new ConcurrentHashMap<>(); /** */ - protected PendingEntriesTree pendingEntries; + private PendingEntriesTree pendingEntries; /** */ - private volatile boolean hasPendingEntries; + protected volatile boolean hasPendingEntries; /** */ private final GridAtomicLong globalRmvId = new GridAtomicLong(U.currentTimeMillis() * 1000_000); /** */ - private final GridSpinBusyLock busyLock = new GridSpinBusyLock(); + protected final GridSpinBusyLock busyLock = new GridSpinBusyLock(); /** */ private int updateValSizeThreshold; @@ -148,19 +148,29 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager /** {@inheritDoc} */ public void onCacheStarted(GridCacheContext cctx) throws IgniteCheckedException { + initPendingTree(cctx); + } + + /** + * @param cctx Cache context. + * @throws IgniteCheckedException If failed. + */ + protected void initPendingTree(GridCacheContext cctx) throws IgniteCheckedException { + assert !cctx.group().persistenceEnabled(); + if (cctx.affinityNode() && cctx.ttl().eagerTtlEnabled() && pendingEntries == null) { String name = "PendingEntries"; - long rootPage = allocateForTree(); + long rootPage = allocateForTree(); - pendingEntries = new PendingEntriesTree( - grp, - name, - grp.dataRegion().pageMemory(), - rootPage, - grp.reuseList(), - true); - } + pendingEntries = new PendingEntriesTree( + grp, + name, + grp.dataRegion().pageMemory(), + rootPage, + grp.reuseList(), + true); + } } /** @@ -204,11 +214,11 @@ private void removeCacheData(int cacheId) { try { if (grp.sharedGroup()) { assert cacheId != CU.UNDEFINED_CACHE_ID; - assert ctx.database().checkpointLockIsHeldByThread(); for (CacheDataStore store : cacheDataStores()) store.clear(cacheId); + // Clear non-persistent pending tree if needed. if (pendingEntries != null) { PendingRow row = new PendingRow(cacheId); @@ -241,6 +251,14 @@ public CacheDataStore dataStore(GridDhtLocalPartition part) { } } + /** + * @param part Partition. + * @return Data store for given entry. + */ + public CacheDataStore dataStore(int part) { + return grp.isLocal() ? locCacheDataStore : partDataStores.get(part); + } + /** {@inheritDoc} */ @Override public long cacheEntriesCount(int cacheId) { long size = 0; @@ -1011,51 +1029,56 @@ protected final String treeName(int p) { ) throws IgniteCheckedException { assert !cctx.isNear() : cctx.name(); - if (hasPendingEntries && pendingEntries != null) { - GridCacheVersion obsoleteVer = null; + if (!hasPendingEntries || pendingEntries == null) + return false; - long now = U.currentTimeMillis(); + GridCacheVersion obsoleteVer = null; - GridCursor cur; + long now = U.currentTimeMillis(); - if (grp.sharedGroup()) - cur = pendingEntries.find(new PendingRow(cctx.cacheId()), new PendingRow(cctx.cacheId(), now, 0)); - else - cur = pendingEntries.find(null, new PendingRow(CU.UNDEFINED_CACHE_ID, now, 0)); + GridCursor cur; - if (!cur.next()) - return false; + if (grp.sharedGroup()) + cur = pendingEntries.find(new PendingRow(cctx.cacheId()), new PendingRow(cctx.cacheId(), now, 0)); + else + cur = pendingEntries.find(null, new PendingRow(CU.UNDEFINED_CACHE_ID, now, 0)); - int cleared = 0; + if (!cur.next()) + return false; - cctx.shared().database().checkpointReadLock(); + int cleared = 0; - try { - do { - PendingRow row = cur.get(); + if (!busyLock.enterBusy()) + return false; - if (amount != -1 && cleared > amount) - return true; + try { + do { + if (amount != -1 && cleared > amount) + return true; - if (row.key.partition() == -1) - row.key.partition(cctx.affinity().partition(row.key)); + PendingRow row = cur.get(); - assert row.key != null && row.link != 0 && row.expireTime != 0 : row; + if (row.key.partition() == -1) + row.key.partition(cctx.affinity().partition(row.key)); - if (pendingEntries.removex(row)) { - if (obsoleteVer == null) - obsoleteVer = ctx.versions().next(); + assert row.key != null && row.link != 0 && row.expireTime != 0 : row; - c.apply(cctx.cache().entryEx(row.key), obsoleteVer); - } + if (pendingEntries.removex(row)) { + if (obsoleteVer == null) + obsoleteVer = ctx.versions().next(); + + GridCacheEntryEx entry = cctx.cache().entryEx(row.key); - cleared++; + if (entry != null) + c.apply(entry, obsoleteVer); } - while (cur.next()); - } - finally { - cctx.shared().database().checkpointReadUnlock(); + + cleared++; } + while (cur.next()); + } + finally { + busyLock.leaveBusy(); } return false; @@ -1395,15 +1418,15 @@ private void finishUpdate(GridCacheContext cctx, CacheDataRow newRow, @Nullable if (oldRow != null) { assert oldRow.link() != 0 : oldRow; - if (pendingEntries != null && oldRow.expireTime() != 0) - pendingEntries.removex(new PendingRow(cacheId, oldRow.expireTime(), oldRow.link())); + if (pendingTree() != null && oldRow.expireTime() != 0) + pendingTree().removex(new PendingRow(cacheId, oldRow.expireTime(), oldRow.link())); if (newRow.link() != oldRow.link()) rowStore.removeRow(oldRow.link()); } - if (pendingEntries != null && expireTime != 0) { - pendingEntries.putx(new PendingRow(cacheId, expireTime, newRow.link())); + if (pendingTree() != null && expireTime != 0) { + pendingTree().putx(new PendingRow(cacheId, expireTime, newRow.link())); hasPendingEntries = true; } @@ -1444,8 +1467,8 @@ private void finishRemove(GridCacheContext cctx, KeyCacheObject key, @Nullable C assert cacheId == CU.UNDEFINED_CACHE_ID || oldRow.cacheId() == cacheId : "Incorrect cache ID [expected=" + cacheId + ", actual=" + oldRow.cacheId() + "]."; - if (pendingEntries != null && oldRow.expireTime() != 0) - pendingEntries.removex(new PendingRow(cacheId, oldRow.expireTime(), oldRow.link())); + if (pendingTree() != null && oldRow.expireTime() != 0) + pendingTree().removex(new PendingRow(cacheId, oldRow.expireTime(), oldRow.link())); decrementSize(cctx.cacheId()); } @@ -1543,7 +1566,6 @@ private void finishRemove(GridCacheContext cctx, KeyCacheObject key, @Nullable C /** {@inheritDoc} */ @Override public void clear(int cacheId) throws IgniteCheckedException { assert cacheId != CU.UNDEFINED_CACHE_ID; - assert ctx.database().checkpointLockIsHeldByThread(); if (cacheSize(cacheId) == 0) return; @@ -1624,6 +1646,11 @@ private void finishRemove(GridCacheContext cctx, KeyCacheObject key, @Nullable C } } + /** {@inheritDoc} */ + @Override public PendingEntriesTree pendingTree() { + return pendingEntries; + } + /** * @param cctx Cache context. * @param key Key. @@ -1676,5 +1703,4 @@ private int valueLength(GridCacheContext cctx, @Nullable CacheObject val) { return 0; } } - } 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 be74eff5cf674..a199f6cc9ef30 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 @@ -47,7 +47,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader; import org.apache.ignite.internal.processors.cache.extras.GridCacheObsoleteEntryExtras; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; -import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -57,9 +56,9 @@ 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.util.deque.FastSizeDeque; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import org.apache.ignite.util.deque.FastSizeDeque; import static org.apache.ignite.IgniteSystemProperties.IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_CACHE_REMOVED_ENTRIES_TTL; @@ -342,9 +341,6 @@ public int reservations() { * @return {@code True} if partition is empty. */ public boolean isEmpty() { - if (grp.allowFastEviction()) - return internalSize() == 0; - return store.fullSize() == 0 && internalSize() == 0; } @@ -981,78 +977,76 @@ private long clearAll() throws NodeStoppingException { long cleared = 0; - if (!grp.allowFastEviction()) { - CacheMapHolder hld = grp.sharedGroup() ? null : singleCacheEntryMap; + CacheMapHolder hld = grp.sharedGroup() ? null : singleCacheEntryMap; - try { - GridIterator it0 = grp.offheap().partitionIterator(id); + try { + GridIterator it0 = grp.offheap().partitionIterator(id); - while (it0.hasNext()) { - ctx.database().checkpointReadLock(); + while (it0.hasNext()) { + ctx.database().checkpointReadLock(); - try { - CacheDataRow row = it0.next(); - - // Do not clear fresh rows in case of single partition clearing. - if (row.version().compareTo(clearVer) >= 0 && (state() == MOVING && clear)) - continue; - - if (grp.sharedGroup() && (hld == null || hld.cctx.cacheId() != row.cacheId())) - hld = cacheMapHolder(ctx.cacheContext(row.cacheId())); - - assert hld != null; - - GridCacheMapEntry cached = putEntryIfObsoleteOrAbsent( - hld, - hld.cctx, - grp.affinity().lastVersion(), - row.key(), - true, - false); - - if (cached instanceof GridDhtCacheEntry && ((GridDhtCacheEntry)cached).clearInternal(clearVer, extras)) { - removeEntry(cached); - - if (rec && !hld.cctx.config().isEventsDisabled()) { - hld.cctx.events().addEvent(cached.partition(), - cached.key(), - ctx.localNodeId(), - (IgniteUuid)null, - null, - EVT_CACHE_REBALANCE_OBJECT_UNLOADED, - null, - false, - cached.rawGet(), - cached.hasValue(), - null, - null, - null, - false); - } - - cleared++; + try { + CacheDataRow row = it0.next(); + + // Do not clear fresh rows in case of single partition clearing. + if (row.version().compareTo(clearVer) >= 0 && (state() == MOVING && clear)) + continue; + + if (grp.sharedGroup() && (hld == null || hld.cctx.cacheId() != row.cacheId())) + hld = cacheMapHolder(ctx.cacheContext(row.cacheId())); + + assert hld != null; + + GridCacheMapEntry cached = putEntryIfObsoleteOrAbsent( + hld, + hld.cctx, + grp.affinity().lastVersion(), + row.key(), + true, + false); + + if (cached instanceof GridDhtCacheEntry && ((GridDhtCacheEntry)cached).clearInternal(clearVer, extras)) { + removeEntry(cached); + + if (rec && !hld.cctx.config().isEventsDisabled()) { + hld.cctx.events().addEvent(cached.partition(), + cached.key(), + ctx.localNodeId(), + (IgniteUuid)null, + null, + EVT_CACHE_REBALANCE_OBJECT_UNLOADED, + null, + false, + cached.rawGet(), + cached.hasValue(), + null, + null, + null, + false); } - } - catch (GridDhtInvalidPartitionException e) { - assert isEmpty() && state() == EVICTED : "Invalid error [e=" + e + ", part=" + this + ']'; - break; // Partition is already concurrently cleared and evicted. - } - finally { - ctx.database().checkpointReadUnlock(); + cleared++; } } - } - catch (NodeStoppingException e) { - if (log.isDebugEnabled()) - log.debug("Failed to get iterator for evicted partition: " + id); + catch (GridDhtInvalidPartitionException e) { + assert isEmpty() && state() == EVICTED : "Invalid error [e=" + e + ", part=" + this + ']'; - throw e; - } - catch (IgniteCheckedException e) { - U.error(log, "Failed to get iterator for evicted partition: " + id, e); + break; // Partition is already concurrently cleared and evicted. + } + finally { + ctx.database().checkpointReadUnlock(); + } } } + catch (NodeStoppingException e) { + if (log.isDebugEnabled()) + log.debug("Failed to get iterator for evicted partition: " + id); + + throw e; + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to get iterator for evicted partition: " + id, e); + } return cleared; } @@ -1405,38 +1399,10 @@ private void registerClearingCallback() { } } - /** - * Recreate cache data store after successful clearing and allowed fast eviction. - */ - private void recreateCacheDataStore() { - assert grp.offheap() instanceof GridCacheOffheapManager; - - try { - CacheDataStore store0 = store; - - store = ((GridCacheOffheapManager) grp.offheap()).recreateCacheDataStore(store0); - - // Inject row cache cleaner on store creation - // Used in case the cache with enabled SqlOnheapCache is single cache at the cache group - if (ctx.kernalContext().query().moduleEnabled()) { - GridQueryRowCacheCleaner cleaner = ctx.kernalContext().query().getIndexing() - .rowCacheCleaner(grp.groupId()); - - if (store != null && cleaner != null) - store.setRowCacheCleaner(cleaner); - } - } catch (IgniteCheckedException e) { - finish(e); - } - } - /** * Successfully finishes the future. */ public void finish() { - if (state() == MOVING && clear && grp.allowFastEviction()) - recreateCacheDataStore(); - synchronized (this) { onDone(); finished = true; 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 cc0542c5d0e5b..866c5133b8aad 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 @@ -95,6 +95,7 @@ public void validatePartitionCountersAndSizes(GridDhtPartitionsExchangeFuture fu // Validate cache sizes. result = validatePartitionsSizes(top, messages, ignoringNodes); + 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/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java index 39f4ed11cfcf2..1b79b767daaa8 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 @@ -136,6 +136,13 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte /** */ private static final IgniteProductVersion FORCE_AFF_REASSIGNMENT_SINCE = IgniteProductVersion.fromString("2.4.3"); + /** + * This may be useful when per-entry (not per-cache based) partition policy is in use. + * See {@link IgniteSystemProperties#IGNITE_SKIP_PARTITION_SIZE_VALIDATION} for details. + * Default value is {@code false}. + */ + private static final boolean SKIP_PARTITION_SIZE_VALIDATION = Boolean.getBoolean(IgniteSystemProperties.IGNITE_SKIP_PARTITION_SIZE_VALIDATION); + /** */ @GridToStringExclude private final Object mux = new Object(); @@ -2755,13 +2762,16 @@ private void validatePartitionsState() { grpCtx.topology() : cctx.exchange().clientTopology(grpId, events().discoveryCache()); - // Do not validate read or write through caches or caches with disabled rebalance. + // Do not validate read or write through caches or caches with disabled rebalance + // or ExpiryPolicy is set or validation is disabled. if (grpCtx == null || grpCtx.config().isReadThrough() || grpCtx.config().isWriteThrough() || grpCtx.config().getCacheStoreFactory() != null || grpCtx.config().getRebalanceDelay() == -1 - || grpCtx.config().getRebalanceMode() == CacheRebalanceMode.NONE) + || grpCtx.config().getRebalanceMode() == CacheRebalanceMode.NONE + || grpCtx.config().getExpiryPolicyFactory() == null + || SKIP_PARTITION_SIZE_VALIDATION) continue; try { 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 5feaa252dd3fa..aec9481a1d0d5 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 @@ -40,6 +40,7 @@ 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; +import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageInitRecord; import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageUpdateNextSnapshotId; @@ -49,6 +50,8 @@ import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; +import org.apache.ignite.internal.processors.cache.GridCacheTtlManager; import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl; import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; @@ -56,6 +59,7 @@ 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.persistence.freelist.CacheFreeListImpl; +import org.apache.ignite.internal.processors.cache.persistence.migration.UpgradePendingTreeToPerPartitionTask; import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx; import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId; import org.apache.ignite.internal.processors.cache.persistence.partstate.PagesAllocationRange; @@ -64,6 +68,7 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageMetaIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionCountersIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIOV2; 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; @@ -71,10 +76,13 @@ 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.PendingEntriesTree; +import org.apache.ignite.internal.processors.cache.tree.PendingRow; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner; import org.apache.ignite.internal.util.lang.GridCursor; +import org.apache.ignite.internal.util.lang.IgniteInClosure2X; import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; @@ -94,8 +102,15 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple /** */ private ReuseListImpl reuseList; + /** {@inheritDoc} */ + @Override protected void initPendingTree(GridCacheContext cctx) throws IgniteCheckedException { + // No-op. Per-partition PendingTree should be used. + } + /** {@inheritDoc} */ @Override protected void initDataStructures() throws IgniteCheckedException { + assert ctx.database().checkpointLockIsHeldByThread(); + Metas metas = getOrAllocateCacheMetas(); RootPage reuseListRoot = metas.reuseListRoot; @@ -122,29 +137,12 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple ((GridCacheDatabaseSharedManager)ctx.database()).addCheckpointListener(this); } - /** {@inheritDoc} */ - @Override public void onCacheStarted(GridCacheContext cctx) throws IgniteCheckedException { - if (cctx.affinityNode() && cctx.ttl().eagerTtlEnabled() && pendingEntries == null) { - ctx.database().checkpointReadLock(); - - try { - final String name = "PendingEntries"; - - RootPage pendingRootPage = indexStorage.getOrAllocateForTree(name); - - pendingEntries = new PendingEntriesTree( - grp, - name, - grp.dataRegion().pageMemory(), - pendingRootPage.pageId().pageId(), - reuseList, - pendingRootPage.isAllocated() - ); - } - finally { - ctx.database().checkpointReadUnlock(); - } - } + /** + * Get internal IndexStorage. + * See {@link UpgradePendingTreeToPerPartitionTask} for details. + */ + public IndexStorage getIndexStorage() { + return indexStorage; } /** {@inheritDoc} */ @@ -218,8 +216,8 @@ private boolean saveStoreMetadata( int grpId = grp.groupId(); long partMetaId = pageMem.partitionMetaPageId(grpId, store.partId()); - long partMetaPage = pageMem.acquirePage(grpId, partMetaId); + long partMetaPage = pageMem.acquirePage(grpId, partMetaId); try { long partMetaPageAddr = pageMem.writeLock(grpId, partMetaId, partMetaPage); @@ -274,7 +272,7 @@ private boolean saveStoreMetadata( if (needSnapshot) { pageCnt = this.ctx.pageStore().pages(grpId, store.partId()); - io.setCandidatePageCount(partMetaPageAddr, size == 0 ? 0: pageCnt); + io.setCandidatePageCount(partMetaPageAddr, size == 0 ? 0 : pageCnt); if (saveMeta) { saveMeta(ctx); @@ -285,7 +283,7 @@ private boolean saveStoreMetadata( if (state == OWNING) { assert part != null; - if(!addPartition( + if (!addPartition( part, ctx.partitionStatMap(), partMetaPageAddr, @@ -295,8 +293,8 @@ private boolean saveStoreMetadata( this.ctx.pageStore().pages(grpId, store.partId()), store.fullSize() )) - U.warn(log,"Partition was concurrently evicted grpId=" + grpId + - ", partitionId=" + part.id()); + U.warn(log, "Partition was concurrently evicted grpId=" + grpId + + ", partitionId=" + part.id()); } else if (state == MOVING || state == RENTING) { if (ctx.partitionStatMap().forceSkipIndexPartition(grpId)) { @@ -333,7 +331,7 @@ else if (state == MOVING || state == RENTING) { } } else if (needSnapshot) - tryAddEmptyPartitionToSnapshot(store, ctx);; + tryAddEmptyPartitionToSnapshot(store, ctx); } else if (needSnapshot) tryAddEmptyPartitionToSnapshot(store, ctx); @@ -350,8 +348,8 @@ else if (needSnapshot) private void tryAddEmptyPartitionToSnapshot(CacheDataStore store, Context ctx) { if (getPartition(store).state() == OWNING) { ctx.partitionStatMap().put( - new GroupPartitionId(grp.groupId(), store.partId()), - new PagesAllocationRange(0, 0)); + new GroupPartitionId(grp.groupId(), store.partId()), + new PagesAllocationRange(0, 0)); } } @@ -362,7 +360,7 @@ private void tryAddEmptyPartitionToSnapshot(CacheDataStore store, Context ctx) { */ private GridDhtLocalPartition getPartition(CacheDataStore store) { return grp.topology().localPartition(store.partId(), - AffinityTopologyVersion.NONE, false, true); + AffinityTopologyVersion.NONE, false, true); } /** @@ -385,7 +383,7 @@ private GridDhtLocalPartition getPartition(CacheDataStore store) { long nextId = cntrsPageId; - while (true){ + while (true) { final long curId = nextId; final long curPage = pageMem.acquirePage(grpId, curId); @@ -542,19 +540,19 @@ private void saveMeta(Context ctx) throws IgniteCheckedException { * @param currAllocatedPageCnt total number of pages allocated for partition [partition, grpId] */ private static boolean addPartition( - GridDhtLocalPartition part, - final PartitionAllocationMap map, - final long metaPageAddr, - final PageMetaIO io, - final int grpId, - final int partId, - final int currAllocatedPageCnt, - final long partSize + GridDhtLocalPartition part, + final PartitionAllocationMap map, + final long metaPageAddr, + final PageMetaIO io, + final int grpId, + final int partId, + final int currAllocatedPageCnt, + final long partSize ) { if (part != null) { boolean reserved = part.reserve(); - if(!reserved) + if (!reserved) return false; } else @@ -596,43 +594,6 @@ private static boolean addPartition( } } - /** - * Destroys given {@code store} and creates new with the same update counters as in given. - * - * @param store Store to destroy. - * @return New cache data store. - * @throws IgniteCheckedException If failed. - */ - public CacheDataStore recreateCacheDataStore(CacheDataStore store) throws IgniteCheckedException { - long updCounter = store.updateCounter(); - long initUpdCounter = store.initialUpdateCounter(); - - int p = store.partId(); - - PageMemoryEx pageMemory = (PageMemoryEx)grp.dataRegion().pageMemory(); - - int tag = pageMemory.invalidate(grp.groupId(), p); - - ctx.pageStore().onPartitionDestroyed(grp.groupId(), p, tag); - - CacheDataStore store0; - - partStoreLock.lock(p); - - try { - store0 = createCacheDataStore0(p); - store0.updateCounter(updCounter); - store0.updateInitialCounter(initUpdCounter); - - partDataStores.put(p, store0); - } - finally { - partStoreLock.unlock(p); - } - - return store0; - } - /** {@inheritDoc} */ @Override public void onPartitionCounterUpdated(int part, long cntr) { CacheDataStore store = partDataStores.get(part); @@ -743,7 +704,8 @@ private Metas getOrAllocateCacheMetas() throws IgniteCheckedException { return new Metas( new RootPage(new FullPageId(metastoreRoot, grpId), allocated), - new RootPage(new FullPageId(reuseListRoot, grpId), allocated)); + new RootPage(new FullPageId(reuseListRoot, grpId), allocated), + null); } finally { pageMem.writeUnlock(grpId, metaId, metaPage, null, allocated); @@ -787,6 +749,47 @@ private Metas getOrAllocateCacheMetas() throws IgniteCheckedException { return iterator; } + /** {@inheritDoc} */ + @Override public boolean expire( + GridCacheContext cctx, + IgniteInClosure2X c, + int amount + ) throws IgniteCheckedException { + assert !cctx.isNear() : cctx.name(); + + if (!hasPendingEntries) + return false; + + if (!busyLock.enterBusy()) + return false; + + try { + int cleared = 0; + + for (CacheDataStore store : cacheDataStores()) { + cleared += ((GridCacheDataStore)store).purgeExpired(cctx, c, amount - cleared); + + if (amount != -1 && cleared >= amount) + return true; + } + } + finally { + busyLock.leaveBusy(); + } + + return false; + } + + /** {@inheritDoc} */ + @Override public long expiredSize() throws IgniteCheckedException { + long size = 0; + + for (CacheDataStore store : cacheDataStores()) + size += ((GridCacheDataStore)store).expiredSize(); + + return size; + } + /** * Calculates free space of all partition data stores - number of bytes available for use in allocated pages. * @@ -1098,13 +1101,18 @@ private static class Metas { @GridToStringInclude private final RootPage treeRoot; + /** */ + @GridToStringInclude + private final RootPage pendingTreeRoot; + /** * @param treeRoot Metadata storage root. * @param reuseListRoot Reuse list root. */ - Metas(RootPage treeRoot, RootPage reuseListRoot) { + Metas(RootPage treeRoot, RootPage reuseListRoot, RootPage pendingTreeRoot) { this.treeRoot = treeRoot; this.reuseListRoot = reuseListRoot; + this.pendingTreeRoot = pendingTreeRoot; } /** {@inheritDoc} */ @@ -1116,7 +1124,7 @@ private static class Metas { /** * */ - private class GridCacheDataStore implements CacheDataStore { + public class GridCacheDataStore implements CacheDataStore { /** */ private final int partId; @@ -1126,6 +1134,9 @@ private class GridCacheDataStore implements CacheDataStore { /** */ private volatile CacheFreeListImpl freeList; + /** */ + private PendingEntriesTree pendingTree; + /** */ private volatile CacheDataStore delegate; @@ -1164,11 +1175,10 @@ private CacheDataStore init0(boolean checkExists) throws IgniteCheckedException return null; } - IgniteCacheDatabaseSharedManager dbMgr = ctx.database(); - - dbMgr.checkpointReadLock(); - if (init.compareAndSet(false, true)) { + IgniteCacheDatabaseSharedManager dbMgr = ctx.database(); + + dbMgr.checkpointReadLock(); try { Metas metas = getOrAllocatePartitionMetas(); @@ -1183,6 +1193,7 @@ private CacheDataStore init0(boolean checkExists) throws IgniteCheckedException ctx.wal(), reuseRoot.pageId().pageId(), reuseRoot.isAllocated()) { + /** {@inheritDoc} */ @Override protected long allocatePageNoReuse() throws IgniteCheckedException { assert grp.shared().database().checkpointLockIsHeldByThread(); @@ -1201,6 +1212,24 @@ private CacheDataStore init0(boolean checkExists) throws IgniteCheckedException rowStore, treeRoot.pageId().pageId(), treeRoot.isAllocated()) { + /** {@inheritDoc} */ + @Override protected long allocatePageNoReuse() throws IgniteCheckedException { + assert grp.shared().database().checkpointLockIsHeldByThread(); + + return pageMem.allocatePage(grpId, partId, PageIdAllocator.FLAG_DATA); + } + }; + + RootPage pendingTreeRoot = metas.pendingTreeRoot; + + final PendingEntriesTree pendingTree0 = new PendingEntriesTree( + grp, + "PendingEntries-" + partId, + grp.dataRegion().pageMemory(), + pendingTreeRoot.pageId().pageId(), + reuseList, + pendingTreeRoot.isAllocated()) { + /** {@inheritDoc} */ @Override protected long allocatePageNoReuse() throws IgniteCheckedException { assert grp.shared().database().checkpointLockIsHeldByThread(); @@ -1210,7 +1239,17 @@ private CacheDataStore init0(boolean checkExists) throws IgniteCheckedException PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory(); - delegate0 = new CacheDataStoreImpl(partId, name, rowStore, dataTree); + delegate0 = new CacheDataStoreImpl(partId, name, rowStore, dataTree) { + /** {@inheritDoc} */ + @Override public PendingEntriesTree pendingTree() { + return pendingTree0; + } + }; + + pendingTree = pendingTree0; + + if (!hasPendingEntries && pendingTree0.size() > 0) + hasPendingEntries = true; int grpId = grp.groupId(); long partMetaId = pageMem.partitionMetaPageId(grpId, partId); @@ -1258,8 +1297,6 @@ private CacheDataStore init0(boolean checkExists) throws IgniteCheckedException } } else { - dbMgr.checkpointReadUnlock(); - U.await(latch); delegate0 = delegate; @@ -1280,13 +1317,15 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { int grpId = grp.groupId(); long partMetaId = pageMem.partitionMetaPageId(grpId, partId); + long partMetaPage = pageMem.acquirePage(grpId, partMetaId); try { boolean allocated = false; - long pageAddr = pageMem.writeLock(grpId, partMetaId, partMetaPage); + boolean pendingTreeAllocated = false; + long pageAddr = pageMem.writeLock(grpId, partMetaId, partMetaPage); try { - long treeRoot, reuseListRoot; + long treeRoot, reuseListRoot, pendingTreeRoot; // Initialize new page. if (PageIO.getType(pageAddr) != PageIO.T_PART_META) { @@ -1296,22 +1335,18 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { treeRoot = pageMem.allocatePage(grpId, partId, PageMemory.FLAG_DATA); reuseListRoot = pageMem.allocatePage(grpId, partId, PageMemory.FLAG_DATA); + pendingTreeRoot = pageMem.allocatePage(grpId, partId, PageMemory.FLAG_DATA); assert PageIdUtils.flag(treeRoot) == PageMemory.FLAG_DATA; assert PageIdUtils.flag(reuseListRoot) == PageMemory.FLAG_DATA; + assert PageIdUtils.flag(pendingTreeRoot) == PageMemory.FLAG_DATA; io.setTreeRoot(pageAddr, treeRoot); io.setReuseListRoot(pageAddr, reuseListRoot); + io.setPendingTreeRoot(pageAddr, pendingTreeRoot); if (PageHandler.isWalDeltaRecordNeeded(pageMem, grpId, partMetaId, partMetaPage, wal, null)) - wal.log(new MetaPageInitRecord( - grpId, - partMetaId, - io.getType(), - io.getVersion(), - treeRoot, - reuseListRoot - )); + wal.log(new PageSnapshot(new FullPageId(partMetaId, grpId), pageAddr, pageMem.pageSize())); allocated = true; } @@ -1321,6 +1356,33 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { treeRoot = io.getTreeRoot(pageAddr); reuseListRoot = io.getReuseListRoot(pageAddr); + int pageVersion = PagePartitionMetaIO.getVersion(pageAddr); + + if (pageVersion < 2) { + assert pageVersion == 1; + + if (log.isDebugEnabled()) + log.info("Upgrade partition meta page version: [part=" + partId + + ", grpId=" + grpId + ", oldVer=" + pageVersion + + ", newVer=" + io.getVersion() + ); + + io = PagePartitionMetaIO.VERSIONS.latest(); + + ((PagePartitionMetaIOV2)io).upgradePage(pageAddr); + + pendingTreeRoot = pageMem.allocatePage(grpId, partId, PageMemory.FLAG_DATA); + + io.setPendingTreeRoot(pageAddr, pendingTreeRoot); + + if (PageHandler.isWalDeltaRecordNeeded(pageMem, grpId, partMetaId, partMetaPage, wal, null)) + wal.log(new PageSnapshot(new FullPageId(partMetaId, grpId), pageAddr, pageMem.pageSize())); + + pendingTreeAllocated = true; + } + else + pendingTreeRoot = io.getPendingTreeRoot(pageAddr); + if (PageIdUtils.flag(treeRoot) != PageMemory.FLAG_DATA) throw new StorageException("Wrong tree root page id flag: treeRoot=" + U.hexLong(treeRoot) + ", part=" + partId + ", grpId=" + grpId); @@ -1328,14 +1390,19 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { if (PageIdUtils.flag(reuseListRoot) != PageMemory.FLAG_DATA) throw new StorageException("Wrong reuse list root page id flag: reuseListRoot=" + U.hexLong(reuseListRoot) + ", part=" + partId + ", grpId=" + grpId); + + if (PageIdUtils.flag(pendingTreeRoot) != PageMemory.FLAG_DATA) + throw new StorageException("Wrong pending tree root page id flag: pendingTreeRoot=" + + U.hexLong(pendingTreeRoot) + ", part=" + partId + ", grpId=" + grpId); } return new Metas( new RootPage(new FullPageId(treeRoot, grpId), allocated), - new RootPage(new FullPageId(reuseListRoot, grpId), allocated)); + new RootPage(new FullPageId(reuseListRoot, grpId), allocated), + new RootPage(new FullPageId(pendingTreeRoot, grpId), allocated || pendingTreeAllocated)); } finally { - pageMem.writeUnlock(grpId, partMetaId, partMetaPage, null, allocated); + pageMem.writeUnlock(grpId, partMetaId, partMetaPage, null, allocated || pendingTreeAllocated); } } finally { @@ -1485,6 +1552,8 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { long expireTime, @Nullable CacheDataRow oldRow ) throws IgniteCheckedException { + assert ctx.database().checkpointLockIsHeldByThread(); + CacheDataStore delegate = init0(false); delegate.update(cctx, key, val, ver, expireTime, oldRow); @@ -1498,6 +1567,8 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { GridCacheVersion ver, long expireTime, @Nullable CacheDataRow oldRow) throws IgniteCheckedException { + assert ctx.database().checkpointLockIsHeldByThread(); + CacheDataStore delegate = init0(false); return delegate.createRow(cctx, key, val, ver, expireTime, oldRow); @@ -1506,6 +1577,8 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public void invoke(GridCacheContext cctx, KeyCacheObject key, OffheapInvokeClosure c) throws IgniteCheckedException { + assert ctx.database().checkpointLockIsHeldByThread(); + CacheDataStore delegate = init0(false); delegate.invoke(cctx, key, c); @@ -1514,6 +1587,8 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public void remove(GridCacheContext cctx, KeyCacheObject key, int partId) throws IgniteCheckedException { + assert ctx.database().checkpointLockIsHeldByThread(); + CacheDataStore delegate = init0(false); delegate.remove(cctx, key, partId); @@ -1583,10 +1658,142 @@ private Metas getOrAllocatePartitionMetas() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public void clear(int cacheId) throws IgniteCheckedException { - CacheDataStore delegate = init0(true); + CacheDataStore delegate0 = init0(true); - if (delegate != null) - delegate.clear(cacheId); + if (delegate0 == null) + return; + + ctx.database().checkpointReadLock(); + try { + // Clear persistent pendingTree + if (pendingTree != null) { + PendingRow row = new PendingRow(cacheId); + + GridCursor cursor = pendingTree.find(row, row, PendingEntriesTree.WITHOUT_KEY); + + while (cursor.next()) { + PendingRow row0 = cursor.get(); + + assert row0.link != 0 : row; + + boolean res = pendingTree.removex(row0); + + assert res; + } + } + + delegate0.clear(cacheId); + } + finally { + ctx.database().checkpointReadUnlock(); + } + } + + /** + * Gets the number of entries pending expire. + * + * @return Number of pending entries. + * @throws IgniteCheckedException If failed to get number of pending entries. + */ + public long expiredSize() throws IgniteCheckedException { + CacheDataStore delegate0 = init0(true); + + return delegate0 == null ? 0 : pendingTree.size(); + } + + /** + * Removes expired entries from data store. + * + * @param cctx Cache context. + * @param c Expiry closure that should be applied to expired entry. See {@link GridCacheTtlManager} for details. + * @param amount Limit of processed entries by single call, {@code -1} for no limit. + * @return {@code True} if unprocessed expired entries remains. + * @throws IgniteCheckedException If failed. + */ + public int purgeExpired(GridCacheContext cctx, + IgniteInClosure2X c, + int amount) throws IgniteCheckedException { + CacheDataStore delegate0 = init0(true); + + if (delegate0 == null || pendingTree == null) + return 0; + + GridDhtLocalPartition part = cctx.topology().localPartition(partId, AffinityTopologyVersion.NONE, false, false); + + // Skip non-owned partitions. + if (part == null || part.state() != OWNING || pendingTree.size() == 0) + return 0; + + cctx.shared().database().checkpointReadLock(); + try { + if (!part.reserve()) + return 0; + + try { + if (part.state() != OWNING) + return 0; + + long now = U.currentTimeMillis(); + + GridCursor cur; + + if (grp.sharedGroup()) + cur = pendingTree.find(new PendingRow(cctx.cacheId()), new PendingRow(cctx.cacheId(), now, 0)); + else + cur = pendingTree.find(null, new PendingRow(CU.UNDEFINED_CACHE_ID, now, 0)); + + if (!cur.next()) + return 0; + + GridCacheVersion obsoleteVer = null; + + int cleared = 0; + + do { + PendingRow row = cur.get(); + + if (amount != -1 && cleared > amount) + return cleared; + + assert row.key != null && row.link != 0 && row.expireTime != 0 : row; + + row.key.partition(partId); + + if (pendingTree.removex(row)) { + if (obsoleteVer == null) + obsoleteVer = ctx.versions().next(); + + GridCacheEntryEx e1 = cctx.cache().entryEx(row.key); + + if (e1 != null) + c.apply(e1, obsoleteVer); + } + + cleared++; + } + while (cur.next()); + + return cleared; + } + finally { + part.release(); + } + } + finally { + cctx.shared().database().checkpointReadUnlock(); + } + } + + /** {@inheritDoc} */ + @Override public PendingEntriesTree pendingTree() { + try { + CacheDataStore delegate0 = init0(true); + + return delegate0 == null ? null : pendingTree; + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/migration/UpgradePendingTreeToPerPartitionTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/migration/UpgradePendingTreeToPerPartitionTask.java new file mode 100644 index 0000000000000..6fa039dd09fc4 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/migration/UpgradePendingTreeToPerPartitionTask.java @@ -0,0 +1,380 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.migration; + +import java.util.Set; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.pagemem.PageIdUtils; +import org.apache.ignite.internal.pagemem.PageMemory; +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.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManager; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter; +import org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager; +import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.IndexStorage; +import org.apache.ignite.internal.processors.cache.persistence.RootPage; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; +import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree; +import org.apache.ignite.internal.processors.cache.tree.PendingRow; +import org.apache.ignite.internal.util.IgniteTree; +import org.apache.ignite.internal.util.lang.GridCursor; +import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.resources.LoggerResource; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree.WITHOUT_KEY; + +/** + * Ignite native persistence migration task upgrades existed PendingTrees to per-partition basis. It's ignore possible + * assertions errors when a pointer to an entry exists in tree but the entry itself was removed due to some reason (e.g. + * when partition was evicted after restart). + * + * Task goes through persistent cache groups and copy entries to certain partitions. + */ +public class UpgradePendingTreeToPerPartitionTask implements IgniteCallable { + /** */ + private static final String PENDING_ENTRIES_TREE_NAME = "PendingEntries"; + + /** */ + private static final long serialVersionUID = 0L; + + /** */ + public static final int BATCH_SIZE = 500; + + /** */ + @IgniteInstanceResource + private IgniteEx node; + + /** */ + @LoggerResource + private IgniteLogger log; + + /** {@inheritDoc} */ + @Override public Boolean call() throws IgniteException { + GridCacheSharedContext sharedCtx = node.context().cache().context(); + + for (CacheGroupContext grp : sharedCtx.cache().cacheGroups()) { + if (!grp.persistenceEnabled() || !grp.affinityNode()) { + if (!grp.persistenceEnabled()) + log.info("Skip pending tree upgrade for non-persistent cache group: [grpId=" + grp.groupId() + + ", grpName=" + grp.name() + ']'); + else + log.info("Skip pending tree upgrade on non-affinity node for cache group: [grpId=" + grp.groupId() + + ", grpName=" + grp.name() + ']'); + + continue; + } + + try { + processCacheGroup(grp); + } + catch (Exception ex) { + if (Thread.interrupted() || X.hasCause(ex, InterruptedException.class)) + log.info("Upgrade pending tree has been cancelled."); + else + log.warning("Failed to upgrade pending tree for cache group: [grpId=" + grp.groupId() + + ", grpName=" + grp.name() + ']', ex); + + return false; + } + + if (Thread.interrupted()) { + log.info("Upgrade pending tree has been cancelled."); + + return false; + } + } + + log.info("All pending trees upgraded successfully."); + + return true; + } + + /** + * Converts CacheGroup pending tree to per-partition basis. + * + * @param grp Cache group. + * @throws IgniteCheckedException If error occurs. + */ + private void processCacheGroup(CacheGroupContext grp) throws IgniteCheckedException { + assert grp.offheap() instanceof GridCacheOffheapManager; + + PendingEntriesTree oldPendingTree; + + final IgniteCacheDatabaseSharedManager db = grp.shared().database(); + + db.checkpointReadLock(); + try { + IndexStorage indexStorage = ((GridCacheOffheapManager)grp.offheap()).getIndexStorage(); + + //TODO: IGNITE-5874: replace with some check-method to avoid unnecessary page allocation. + RootPage pendingRootPage = indexStorage.getOrAllocateForTree(PENDING_ENTRIES_TREE_NAME); + + if (pendingRootPage.isAllocated()) { + log.info("No pending tree found for cache group: [grpId=" + grp.groupId() + + ", grpName=" + grp.name() + ']'); + + // Nothing to do here as just allocated tree is obviously empty. + indexStorage.dropRootPage(PENDING_ENTRIES_TREE_NAME); + + return; + } + + oldPendingTree = new PendingEntriesTree( + grp, + PENDING_ENTRIES_TREE_NAME, + grp.dataRegion().pageMemory(), + pendingRootPage.pageId().pageId(), + ((GridCacheOffheapManager)grp.offheap()).reuseListForIndex(null), + false + ); + } + finally { + db.checkpointReadUnlock(); + } + + processPendingTree(grp, oldPendingTree); + + if (Thread.currentThread().isInterrupted()) + return; + + db.checkpointReadLock(); + try { + oldPendingTree.destroy(); + } + finally { + db.checkpointReadUnlock(); + } + } + + /** + * Move pending rows for CacheGroup entries to per-partition PendingTree. + * Invalid pending rows will be ignored. + * + * @param grp Cache group. + * @param oldPendingEntries Old-style PendingTree. + * @throws IgniteCheckedException If error occurs. + */ + private void processPendingTree(CacheGroupContext grp, PendingEntriesTree oldPendingEntries) + throws IgniteCheckedException { + final PageMemory pageMemory = grp.dataRegion().pageMemory(); + + final IgniteCacheDatabaseSharedManager db = grp.shared().database(); + + final Set cacheIds = grp.cacheIds(); + + PendingRow row = null; + + int processedEntriesCnt = 0; + int skippedEntries = 0; + + // Re-acquire checkpoint lock for every next batch. + while (!Thread.currentThread().isInterrupted()) { + int cnt = 0; + + db.checkpointReadLock(); + try { + GridCursor cursor = oldPendingEntries.find(row, null, WITHOUT_KEY); + + while (cnt++ < BATCH_SIZE && cursor.next()) { + row = cursor.get(); + + assert row.link != 0 && row.expireTime != 0 : row; + + GridCacheEntryEx entry; + + // Lost cache or lost entry. + if (!cacheIds.contains(row.cacheId) || (entry = getEntry(grp, row)) == null) { + skippedEntries++; + + oldPendingEntries.removex(row); + + continue; + } + + entry.lockEntry(); + try { + if (processRow(pageMemory, grp, row)) + processedEntriesCnt++; + else + skippedEntries++; + } + finally { + entry.unlockEntry(); + } + + oldPendingEntries.removex(row); + } + + if (cnt < BATCH_SIZE) + break; + } + finally { + db.checkpointReadUnlock(); + } + } + + log.info("PendingTree upgraded: " + + "[grpId=" + grp.groupId() + + ", grpName=" + grp.name() + + ", processedEntries=" + processedEntriesCnt + + ", failedEntries=" + skippedEntries + + ']'); + } + + /** + * Return CacheEntry instance for lock purpose. + * + * @param grp Cache group + * @param row Pending row. + * @return CacheEntry if found or null otherwise. + */ + private GridCacheEntryEx getEntry(CacheGroupContext grp, PendingRow row) { + try { + CacheDataRowAdapter rowData = new CacheDataRowAdapter(row.link); + + rowData.initFromLink(grp, CacheDataRowAdapter.RowData.KEY_ONLY); + + GridCacheContext cctx = grp.shared().cacheContext(row.cacheId); + + assert cctx != null; + + return cctx.cache().entryEx(rowData.key()); + } + catch (Throwable ex) { + if (Thread.currentThread().isInterrupted() || X.hasCause(ex, InterruptedException.class)) + throw new IgniteException(new InterruptedException()); + + log.warning("Failed to move old-version pending entry " + + "to per-partition PendingTree: key not found (skipping): " + + "[grpId=" + grp.groupId() + + ", grpName=" + grp.name() + + ", pendingRow=" + row + "]"); + + return null; + } + + } + + /** + * Validates PendingRow and add it to per-partition PendingTree. + * + * @param pageMemory Page memory. + * @param grp Cache group. + * @param row Pending row. + * @return {@code True} if pending row successfully moved, {@code False} otherwise. + */ + private boolean processRow(PageMemory pageMemory, CacheGroupContext grp, PendingRow row) { + final long pageId = PageIdUtils.pageId(row.link); + + final int partition = PageIdUtils.partId(pageId); + + assert partition >= 0; + + try { + final long page = pageMemory.acquirePage(grp.groupId(), pageId); + long pageAddr = pageMemory.readLock(grp.groupId(), pageId, page); + try { + assert PageIO.getType(pageAddr) != 0; + assert PageIO.getVersion(pageAddr) != 0; + + IgniteCacheOffheapManager.CacheDataStore store = + ((GridCacheOffheapManager)grp.offheap()).dataStore(partition); + + if (store == null) { + log.warning("Failed to move old-version pending entry " + + "to per-partition PendingTree: Node has no partition anymore (skipping): " + + "[grpId=" + grp.groupId() + + ", grpName=" + grp.name() + + ", partId=" + partition + + ", pendingRow=" + row + "]"); + + return false; + } + + assert store instanceof GridCacheOffheapManager.GridCacheDataStore; + assert store.pendingTree() != null; + + store.pendingTree().invoke(row, WITHOUT_KEY, new PutIfAbsentClosure(row)); + } + finally { + pageMemory.readUnlock(grp.groupId(), pageId, page); + } + } + catch (AssertionError | Exception ex) { + if (Thread.currentThread().isInterrupted() || X.hasCause(ex, InterruptedException.class)) { + Thread.currentThread().interrupt(); + + throw new IgniteException(ex); + } + + String msg = "Unexpected error occurs while moving old-version pending entry " + + "to per-partition PendingTree. Seems page doesn't longer exists (skipping): " + + "[grpId=" + grp.groupId() + + ", grpName=" + grp.name() + + ", partId=" + partition + + ", pendingRow=" + row + ']'; + + if (log.isDebugEnabled()) + log.warning(msg, ex); + else + log.warning(msg); + + return false; + } + + return true; + } + + /** */ + private static class PutIfAbsentClosure implements IgniteTree.InvokeClosure { + /** */ + private final PendingRow pendingRow; + + /** */ + private IgniteTree.OperationType op; + + /** */ + PutIfAbsentClosure(PendingRow pendingRow) { + this.pendingRow = pendingRow; + } + + /** {@inheritDoc} */ + @Override public void call(@Nullable PendingRow oldRow) throws IgniteCheckedException { + op = (oldRow == null) ? IgniteTree.OperationType.PUT : IgniteTree.OperationType.NOOP; + } + + /** {@inheritDoc} */ + @Override public PendingRow newRow() { + return pendingRow; + } + + /** {@inheritDoc} */ + @Override public IgniteTree.OperationType operationType() { + return op; + } + } +} \ No newline at end of file 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 a5236c210e88f..c940c3974f109 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 @@ -191,7 +191,6 @@ public abstract class PageIO { /** */ public static final short T_DATA_REF_METASTORAGE_LEAF = 23; - /** Index for payload == 1. */ public static final short T_H2_EX_REF_LEAF_START = 10000; @@ -215,8 +214,8 @@ public abstract class PageIO { * @param ver Page format version. */ protected PageIO(int type, int ver) { - assert ver > 0 && ver < 65535: ver; - assert type > 0 && type < 65535: type; + assert ver > 0 && ver < 65535 : ver; + assert type > 0 && type < 65535 : type; this.type = type; this.ver = ver; @@ -245,7 +244,7 @@ public static int getType(long pageAddr) { public static void setType(long pageAddr, int type) { PageUtils.putShort(pageAddr, TYPE_OFF, (short)type); - assert getType(pageAddr) == type; + assert getType(pageAddr) == type : getType(pageAddr); } /** @@ -268,7 +267,7 @@ public static int getVersion(long pageAddr) { * @param pageAddr Page address. * @param ver Version. */ - private static void setVersion(long pageAddr, int ver) { + protected static void setVersion(long pageAddr, int ver) { PageUtils.putShort(pageAddr, VER_OFF, (short)ver); assert getVersion(pageAddr) == ver; @@ -580,7 +579,7 @@ public static boolean isDataPageType(int type) { * @param pageSize Page size. * @param sb Sb. */ - protected abstract void printPage(long addr, int pageSize, GridStringBuilder sb) throws IgniteCheckedException ; + protected abstract void printPage(long addr, int pageSize, GridStringBuilder sb) throws IgniteCheckedException; /** * @param addr Address. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PagePartitionMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PagePartitionMetaIO.java index 3d798841bf473..fe6b7a5bc866c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PagePartitionMetaIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PagePartitionMetaIO.java @@ -42,9 +42,13 @@ public class PagePartitionMetaIO extends PageMetaIO { /** */ private static final int NEXT_PART_META_PAGE_OFF = PARTITION_STATE_OFF + 1; + /** End of page partition meta. */ + static final int END_OF_PARTITION_PAGE_META = NEXT_PART_META_PAGE_OFF + 8; + /** */ public static final IOVersions VERSIONS = new IOVersions<>( - new PagePartitionMetaIO(1) + new PagePartitionMetaIO(1), + new PagePartitionMetaIOV2(2) ); /** {@inheritDoc} */ @@ -150,6 +154,7 @@ public boolean setPartitionState(long pageAddr, byte state) { /** * Returns partition counters page identifier, page with caches in cache group sizes. + * * @param pageAddr Partition metadata page address. * @return Next meta partial page ID or {@code 0} if it does not exist. */ @@ -167,19 +172,39 @@ public void setCountersPageId(long pageAddr, long cntrsPageId) { PageUtils.putLong(pageAddr, NEXT_PART_META_PAGE_OFF, cntrsPageId); } + /** + * Returns partition pending tree root. Pending tree is used to tracking expiring entries. + * + * @param pageAddr Page address. + * @return Pending Tree root page. + */ + public long getPendingTreeRoot(long pageAddr) { + throw new UnsupportedOperationException("Per partition pending tree is not supported by " + + "this PagePartitionMetaIO version: ver=" + getVersion()); + } + + /** + * Sets new partition pending tree root. + * + * @param pageAddr Page address. + * @param treeRoot Pending Tree root + */ + public void setPendingTreeRoot(long pageAddr, long treeRoot) { + throw new UnsupportedOperationException("Per partition pending tree is not supported by " + + "this PagePartitionMetaIO version: ver=" + getVersion()); + } + /** {@inheritDoc} */ @Override protected void printPage(long pageAddr, int pageSize, GridStringBuilder sb) throws IgniteCheckedException { super.printPage(pageAddr, pageSize, sb); byte state = getPartitionState(pageAddr); - sb - .a(",\nPagePartitionMeta[\n\tsize=").a(getSize(pageAddr)) + sb.a(",\nPagePartitionMeta[\n\tsize=").a(getSize(pageAddr)) .a(",\n\tupdateCounter=").a(getUpdateCounter(pageAddr)) .a(",\n\tglobalRemoveId=").a(getGlobalRemoveId(pageAddr)) .a(",\n\tpartitionState=").a(state).a("(").a(GridDhtPartitionState.fromOrdinal(state)).a(")") .a(",\n\tcountersPageId=").a(getCountersPageId(pageAddr)) - .a("\n]") - ; + .a("\n]"); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PagePartitionMetaIOV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PagePartitionMetaIOV2.java new file mode 100644 index 0000000000000..70556a11eeb43 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PagePartitionMetaIOV2.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.ignite.internal.processors.cache.persistence.tree.io; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.PageUtils; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState; +import org.apache.ignite.internal.util.GridStringBuilder; + +/** + * IO for partition metadata pages. + * Persistent partition contains it's own PendingTree. + */ +public class PagePartitionMetaIOV2 extends PagePartitionMetaIO { + /** */ + private static final int PENDING_TREE_ROOT_OFF = PagePartitionMetaIO.END_OF_PARTITION_PAGE_META; + + /** + * @param ver Version. + */ + public PagePartitionMetaIOV2(int ver) { + super(ver); + } + + /** {@inheritDoc} */ + @Override public void initNewPage(long pageAddr, long pageId, int pageSize) { + super.initNewPage(pageAddr, pageId, pageSize); + + setPendingTreeRoot(pageAddr, 0L); + } + + /** {@inheritDoc} */ + @Override public long getPendingTreeRoot(long pageAddr) { + return PageUtils.getLong(pageAddr, PENDING_TREE_ROOT_OFF); + } + + /** {@inheritDoc} */ + @Override public void setPendingTreeRoot(long pageAddr, long treeRoot) { + PageUtils.putLong(pageAddr, PENDING_TREE_ROOT_OFF, treeRoot); + } + + /** {@inheritDoc} */ + @Override protected void printPage(long pageAddr, int pageSize, GridStringBuilder sb) throws IgniteCheckedException { + byte state = getPartitionState(pageAddr); + + sb.a("PagePartitionMeta[\n\ttreeRoot=").a(getReuseListRoot(pageAddr)); + sb.a(",\n\tpendingTreeRoot=").a(getLastSuccessfulFullSnapshotId(pageAddr)); + sb.a(",\n\tlastSuccessfulFullSnapshotId=").a(getLastSuccessfulFullSnapshotId(pageAddr)); + sb.a(",\n\tlastSuccessfulSnapshotId=").a(getLastSuccessfulSnapshotId(pageAddr)); + sb.a(",\n\tnextSnapshotTag=").a(getNextSnapshotTag(pageAddr)); + sb.a(",\n\tlastSuccessfulSnapshotTag=").a(getLastSuccessfulSnapshotTag(pageAddr)); + sb.a(",\n\tlastAllocatedPageCount=").a(getLastAllocatedPageCount(pageAddr)); + sb.a(",\n\tcandidatePageCount=").a(getCandidatePageCount(pageAddr)); + sb.a(",\n\tsize=").a(getSize(pageAddr)); + sb.a(",\n\tupdateCounter=").a(getUpdateCounter(pageAddr)); + sb.a(",\n\tglobalRemoveId=").a(getGlobalRemoveId(pageAddr)); + sb.a(",\n\tpartitionState=").a(state).a("(").a(GridDhtPartitionState.fromOrdinal(state)).a(")"); + sb.a(",\n\tcountersPageId=").a(getCountersPageId(pageAddr)); + sb.a("\n]"); + } + + /** + * Upgrade page to PagePartitionMetaIOV2 + * + * @param pageAddr Page address. + */ + public void upgradePage(long pageAddr) { + assert PageIO.getType(pageAddr) == getType(); + assert PageIO.getVersion(pageAddr) < 2; + + PageIO.setVersion(pageAddr, getVersion()); + setPendingTreeRoot(pageAddr, 0); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidatorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidatorSelfTest.java index 43a23031dd99a..ebe6f29bfbf71 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidatorSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidatorSelfTest.java @@ -55,10 +55,11 @@ public class GridCachePartitionsStateValidatorSelfTest extends GridCommonAbstrac Mockito.when(topologyMock.partitions()).thenReturn(3); List localPartitions = Lists.newArrayList( - partitionMock(0, 1, 1), - partitionMock(1, 2, 2), - partitionMock(2, 3, 3) + partitionMock(0, 1, 1), + partitionMock(1, 2, 2), + partitionMock(2, 3, 3) ); + Mockito.when(topologyMock.localPartitions()).thenReturn(localPartitions); Mockito.when(topologyMock.currentLocalPartitions()).thenReturn(localPartitions); } @@ -82,10 +83,13 @@ private GridDhtLocalPartition partitionMock(int id, long updateCounter, long siz */ private GridDhtPartitionsSingleMessage from(@Nullable Map> countersMap, @Nullable Map sizesMap) { GridDhtPartitionsSingleMessage msg = new GridDhtPartitionsSingleMessage(); + if (countersMap != null) msg.addPartitionUpdateCounters(0, countersMap); + if (sizesMap != null) msg.addPartitionSizes(0, sizesMap); + return msg; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest.java index 99614ed7ab697..a02ed11549fff 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest.java @@ -17,32 +17,35 @@ package org.apache.ignite.internal.processors.cache.persistence; +import java.io.Serializable; import java.util.Map; +import java.util.Objects; import java.util.Random; import java.util.TreeMap; import java.util.concurrent.Callable; import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteDataStreamer; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +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.configuration.WALMode; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; /** - * + * Cause by https://issues.apache.org/jira/browse/IGNITE-7278 */ public class IgnitePdsContinuousRestartTest extends GridCommonAbstractTest { /** */ @@ -52,7 +55,7 @@ public class IgnitePdsContinuousRestartTest extends GridCommonAbstractTest { private static final int ENTRIES_COUNT = 10_000; /** */ - public static final String CACHE_NAME = "cache1"; + protected static final String CACHE_NAME = "cache1"; /** Checkpoint delay. */ private volatile int checkpointDelay = -1; @@ -79,21 +82,23 @@ public IgnitePdsContinuousRestartTest(boolean cancel) { DataStorageConfiguration memCfg = new DataStorageConfiguration() .setDefaultDataRegionConfiguration( - new DataRegionConfiguration().setMaxSize(400 * 1024 * 1024).setPersistenceEnabled(true)) + new DataRegionConfiguration() + .setMaxSize(400 * 1024 * 1024) + .setPersistenceEnabled(true)) .setWalMode(WALMode.LOG_ONLY) .setCheckpointFrequency(checkpointDelay); cfg.setDataStorageConfiguration(memCfg); - CacheConfiguration ccfg1 = new CacheConfiguration(); + CacheConfiguration ccfg = new CacheConfiguration(); - ccfg1.setName(CACHE_NAME); - ccfg1.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); - ccfg1.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); - ccfg1.setAffinity(new RendezvousAffinityFunction(false, 128)); - ccfg1.setBackups(2); + ccfg.setName(CACHE_NAME); + ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + ccfg.setAffinity(new RendezvousAffinityFunction(false, 128)); + ccfg.setBackups(2); - cfg.setCacheConfiguration(ccfg1); + cfg.setCacheConfiguration(ccfg); return cfg; } @@ -197,7 +202,6 @@ public void testRebalancingDuringLoad_8000_8000_8_16() throws Exception { } /** - * * @throws Exception if failed. */ public void testRebalncingDuringLoad_10_10_1_1() throws Exception { @@ -205,7 +209,6 @@ public void testRebalncingDuringLoad_10_10_1_1() throws Exception { } /** - * * @throws Exception if failed. */ public void testRebalncingDuringLoad_10_500_8_16() throws Exception { @@ -227,7 +230,7 @@ private void checkRebalancingDuringLoad( final Ignite load = ignite(0); - load.active(true); + load.cluster().active(true); try (IgniteDataStreamer s = load.dataStreamer(CACHE_NAME)) { s.allowOverwrite(true); @@ -245,10 +248,13 @@ private void checkRebalancingDuringLoad( Random rnd = ThreadLocalRandom.current(); while (!done.get()) { - Map map = new TreeMap<>(); + Map map = new TreeMap<>(); - for (int i = 0; i < batch; i++) - map.put(rnd.nextInt(ENTRIES_COUNT), rnd.nextInt()); + for (int i = 0; i < batch; i++) { + int key = rnd.nextInt(ENTRIES_COUNT); + + map.put(key, new Person("fn" + key, "ln" + key)); + } cache.putAll(map); } @@ -277,4 +283,51 @@ private void checkRebalancingDuringLoad( busyFut.get(); } + + /** + * + */ + 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; + + IgnitePersistentStoreCacheGroupsTest.Person person = (IgnitePersistentStoreCacheGroupsTest.Person)o; + + return Objects.equals(fName, person.fName) && Objects.equals(lName, person.lName); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return Objects.hash(fName, lName); + } + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest2.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest2.java deleted file mode 100644 index 66b20477252c7..0000000000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTest2.java +++ /dev/null @@ -1,281 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.processors.cache.persistence; - -import java.util.Map; -import java.util.Random; -import java.util.TreeMap; -import java.util.concurrent.Callable; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.atomic.AtomicBoolean; -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.CacheWriteSynchronizationMode; -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.IgniteInternalFuture; -import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.testframework.GridTestUtils; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; - -/** - * Cause by https://issues.apache.org/jira/browse/IGNITE-7278 - */ -public class IgnitePdsContinuousRestartTest2 extends GridCommonAbstractTest { - /** */ - private static final int GRID_CNT = 4; - - /** */ - private static final int ENTRIES_COUNT = 10_000; - - /** */ - public static final String CACHE_NAME = "cache1"; - - /** Checkpoint delay. */ - private volatile int checkpointDelay = -1; - - /** */ - private boolean cancel; - - /** - * Default constructor. - */ - public IgnitePdsContinuousRestartTest2() { - - } - - /** - * @param cancel Cancel. - */ - public IgnitePdsContinuousRestartTest2(boolean cancel) { - this.cancel = cancel; - } - - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - DataStorageConfiguration memCfg = new DataStorageConfiguration() - .setDefaultDataRegionConfiguration( - new DataRegionConfiguration() - .setMaxSize(400 * 1024 * 1024) - .setPersistenceEnabled(true)) - .setWalMode(WALMode.LOG_ONLY) - .setCheckpointFrequency(checkpointDelay); - - cfg.setDataStorageConfiguration(memCfg); - - CacheConfiguration ccfg = new CacheConfiguration(); - - ccfg.setName(CACHE_NAME); - ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); - ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); - ccfg.setAffinity(new RendezvousAffinityFunction(false, 128)); - ccfg.setBackups(2); - - cfg.setCacheConfiguration(ccfg); - - return cfg; - } - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - stopAllGrids(); - - cleanPersistenceDir(); - } - - /** {@inheritDoc} */ - @Override protected void afterTest() throws Exception { - stopAllGrids(); - - cleanPersistenceDir(); - } - - /** - * @throws Exception if failed. - */ - public void testRebalancingDuringLoad_1000_500_1_1() throws Exception { - checkRebalancingDuringLoad(1000, 500, 1, 1); - } - - /** - * @throws Exception if failed. - */ - public void testRebalancingDuringLoad_8000_500_1_1() throws Exception { - checkRebalancingDuringLoad(8000, 500, 1, 1); - } - - /** - * @throws Exception if failed. - */ - public void testRebalancingDuringLoad_1000_20000_1_1() throws Exception { - checkRebalancingDuringLoad(1000, 20000, 1, 1); - } - - /** - * @throws Exception if failed. - */ - public void testRebalancingDuringLoad_8000_8000_1_1() throws Exception { - checkRebalancingDuringLoad(8000, 8000, 1, 1); - } - - /** - * @throws Exception if failed. - */ - public void testRebalancingDuringLoad_1000_500_8_1() throws Exception { - checkRebalancingDuringLoad(1000, 500, 8, 1); - } - - /** - * @throws Exception if failed. - */ - public void testRebalancingDuringLoad_8000_500_8_1() throws Exception { - checkRebalancingDuringLoad(8000, 500, 8, 1); - } - - /** - * @throws Exception if failed. - */ - public void testRebalancingDuringLoad_1000_20000_8_1() throws Exception { - checkRebalancingDuringLoad(1000, 20000, 8, 1); - } - - /** - * @throws Exception if failed. - */ - public void testRebalancingDuringLoad_8000_8000_8_1() throws Exception { - checkRebalancingDuringLoad(8000, 8000, 8, 1); - } - - /** - * @throws Exception if failed. - */ - public void testRebalancingDuringLoad_1000_500_8_16() throws Exception { - checkRebalancingDuringLoad(1000, 500, 8, 16); - } - - /** - * @throws Exception if failed. - */ - public void testRebalancingDuringLoad_8000_500_8_16() throws Exception { - checkRebalancingDuringLoad(8000, 500, 8, 16); - } - - /** - * @throws Exception if failed. - */ - public void testRebalancingDuringLoad_1000_20000_8_16() throws Exception { - checkRebalancingDuringLoad(1000, 20000, 8, 16); - } - - /** - * @throws Exception if failed. - */ - public void testRebalancingDuringLoad_8000_8000_8_16() throws Exception { - checkRebalancingDuringLoad(8000, 8000, 8, 16); - } - - /** - * - * @throws Exception if failed. - */ - public void testRebalncingDuringLoad_10_10_1_1() throws Exception { - checkRebalancingDuringLoad(10, 10, 1, 1); - } - - /** - * - * @throws Exception if failed. - */ - public void testRebalncingDuringLoad_10_500_8_16() throws Exception { - checkRebalancingDuringLoad(10, 500, 8, 16); - } - - /** - * @throws Exception if failed. - */ - private void checkRebalancingDuringLoad( - int restartDelay, - int checkpointDelay, - int threads, - final int batch - ) throws Exception { - this.checkpointDelay = checkpointDelay; - - startGrids(GRID_CNT); - - final Ignite load = ignite(0); - - load.cluster().active(true); - - try (IgniteDataStreamer s = load.dataStreamer(CACHE_NAME)) { - s.allowOverwrite(true); - - for (int i = 0; i < ENTRIES_COUNT; i++) - s.addData(i, i); - } - - final AtomicBoolean done = new AtomicBoolean(false); - - IgniteInternalFuture busyFut = GridTestUtils.runMultiThreadedAsync(new Callable() { - /** {@inheritDoc} */ - @Override public Object call() throws Exception { - IgniteCache cache = load.cache(CACHE_NAME); - Random rnd = ThreadLocalRandom.current(); - - while (!done.get()) { - Map map = new TreeMap<>(); - - for (int i = 0; i < batch; i++) - map.put(rnd.nextInt(ENTRIES_COUNT), rnd.nextInt()); - - cache.putAll(map); - } - - return null; - } - }, threads, "updater"); - - long end = System.currentTimeMillis() + 90_000; - - Random rnd = ThreadLocalRandom.current(); - - while (System.currentTimeMillis() < end) { - int idx = rnd.nextInt(GRID_CNT - 1) + 1; - - stopGrid(idx, cancel); - - U.sleep(restartDelay); - - startGrid(idx); - - U.sleep(restartDelay); - } - - done.set(true); - - busyFut.get(); - } -} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTestWithExpiryPolicy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTestWithExpiryPolicy.java new file mode 100644 index 0000000000000..d5b3f5527a6f0 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsContinuousRestartTestWithExpiryPolicy.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.persistence; + +import java.util.concurrent.TimeUnit; +import javax.cache.expiry.CreatedExpiryPolicy; +import javax.cache.expiry.Duration; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; + +/** + * Cause by https://issues.apache.org/jira/browse/IGNITE-5879 + */ +public class IgnitePdsContinuousRestartTestWithExpiryPolicy extends IgnitePdsContinuousRestartTest { + /** Ip finder. */ + private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** + * Default constructor. + */ + public IgnitePdsContinuousRestartTestWithExpiryPolicy() { + super(false); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + TcpDiscoverySpi discoverySpi = (TcpDiscoverySpi)cfg.getDiscoverySpi(); + discoverySpi.setIpFinder(ipFinder); + + CacheConfiguration ccfg = new CacheConfiguration(); + + ccfg.setName(CACHE_NAME); + ccfg.setGroupName("Group1"); + ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + ccfg.setAffinity(new RendezvousAffinityFunction(false, 128)); + ccfg.setBackups(2); + ccfg.setExpiryPolicyFactory(CreatedExpiryPolicy.factoryOf(new Duration(TimeUnit.SECONDS, 1))); + + cfg.setCacheConfiguration(ccfg); + + return cfg; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteBaselineAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteBaselineAbstractFullApiSelfTest.java index 182566681171e..03dc445297e88 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteBaselineAbstractFullApiSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteBaselineAbstractFullApiSelfTest.java @@ -33,13 +33,20 @@ public abstract class IgniteBaselineAbstractFullApiSelfTest extends GridCacheAbs cfg.setDataStorageConfiguration(new DataStorageConfiguration() .setDefaultDataRegionConfiguration( new DataRegionConfiguration() - .setMaxSize(200 * 1024 * 1024) + .setMaxSize(256 * 1024 * 1024) .setPersistenceEnabled(true)) .setWalMode(WALMode.LOG_ONLY)); return cfg; } + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + cleanPersistenceDir(); + } + /** {@inheritDoc} */ @Override protected int gridCount() { return 4; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWithTtlTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWithTtlTest.java new file mode 100644 index 0000000000000..be09e70e256c5 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWithTtlTest.java @@ -0,0 +1,197 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.persistence.db; + +import com.google.common.base.Strings; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import javax.cache.expiry.AccessedExpiryPolicy; +import javax.cache.expiry.Duration; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheRebalanceMode; +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.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; +import org.apache.ignite.internal.util.typedef.PA; +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; + +/** + * Test TTL worker with persistence enabled + */ +public class IgnitePdsWithTtlTest extends GridCommonAbstractTest { + /** */ + public static final String CACHE = "expirableCache"; + + /** */ + private static final int EXPIRATION_TIMEOUT = 10; + + /** */ + public static final int ENTRIES = 7000; + + /** */ + private static final TcpDiscoveryVmIpFinder FINDER = new TcpDiscoveryVmIpFinder(true); + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + //protection if test failed to finish, e.g. by error + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + final IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + disco.setIpFinder(FINDER); + + cfg.setDiscoverySpi(disco); + + final CacheConfiguration ccfg = new CacheConfiguration(); + ccfg.setName(CACHE); + ccfg.setAffinity(new RendezvousAffinityFunction(false, 128)); + ccfg.setExpiryPolicyFactory(AccessedExpiryPolicy.factoryOf(new Duration(TimeUnit.SECONDS, EXPIRATION_TIMEOUT))); + ccfg.setEagerTtl(true); + ccfg.setGroupName("group1"); + + ccfg.setRebalanceMode(CacheRebalanceMode.SYNC); + cfg.setDataStorageConfiguration( + new DataStorageConfiguration() + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration() + .setMaxSize(256L * 1024 * 1024) + .setPersistenceEnabled(true) + ).setWalMode(WALMode.DEFAULT)); + + cfg.setCacheConfiguration(ccfg); + return cfg; + } + + /** + * @throws Exception if failed. + */ + public void testTtlIsApplied() throws Exception { + loadAndWaitForCleanup(false); + } + + /** + * @throws Exception if failed. + */ + public void testTtlIsAppliedAfterRestart() throws Exception { + loadAndWaitForCleanup(true); + } + + /** + * @throws Exception if failed. + */ + private void loadAndWaitForCleanup(boolean restartGrid) throws Exception { + IgniteEx srv = startGrid(0); + srv.cluster().active(true); + + fillCache(srv.cache(CACHE)); + + if (restartGrid) { + stopGrid(0); + srv = startGrid(0); + srv.cluster().active(true); + } + + final IgniteCache cache = srv.cache(CACHE); + + pringStatistics((IgniteCacheProxy)cache, "After restart from LFS"); + + waitAndCheckExpired(cache); + + stopAllGrids(); + } + + /** + * @throws Exception if failed. + */ + public void testRebalancingWithTtlExpirable() throws Exception { + IgniteEx srv = startGrid(0); + srv.cluster().active(true); + + fillCache(srv.cache(CACHE)); + + //causes rebalancing start + srv = startGrid(1); + + final IgniteCache cache = srv.cache(CACHE); + + pringStatistics((IgniteCacheProxy)cache, "After rebalancing start"); + + waitAndCheckExpired(cache); + + stopAllGrids(); + } + + /** */ + protected void fillCache(IgniteCache cache) { + cache.putAll(new TreeMap() {{ + for (int i = 0; i < ENTRIES; i++) + put(i, Strings.repeat("Some value " + i, 125)); + }}); + + //Touch entries. + for (int i = 0; i < ENTRIES; i++) + cache.get(i); // touch entries + + pringStatistics((IgniteCacheProxy)cache, "After cache puts"); + } + + /** */ + protected void waitAndCheckExpired(final IgniteCache cache) throws IgniteInterruptedCheckedException { + GridTestUtils.waitForCondition(new PA() { + @Override public boolean apply() { + return cache.size() == 0; + } + }, TimeUnit.SECONDS.toMillis(EXPIRATION_TIMEOUT + 1)); + + pringStatistics((IgniteCacheProxy)cache, "After timeout"); + + for (int i = 0; i < ENTRIES; i++) + assertNull(cache.get(i)); + } + + /** */ + private void pringStatistics(IgniteCacheProxy cache, String msg) { + System.out.println(msg + " {{"); + cache.context().printMemoryStats(); + System.out.println("}} " + msg); + } +} \ No newline at end of file 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 d11ceb36a863d..0e94874373174 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 @@ -25,6 +25,7 @@ import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsSingleNodePutGetPersistenceTest; import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsCacheRestoreTest; import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsDataRegionMetricsTest; +import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsWithTtlTest; import org.apache.ignite.internal.processors.cache.persistence.db.file.DefaultPageSizeBackwardsCompatibilityTest; import org.apache.ignite.internal.processors.cache.persistence.db.file.IgnitePdsCheckpointSimulationWithRealCpDisabledTest; import org.apache.ignite.internal.processors.cache.persistence.db.file.IgnitePdsPageReplacementTest; @@ -110,6 +111,7 @@ public static void addRealPageStoreTests(TestSuite suite) { // TODO uncomment when https://issues.apache.org/jira/browse/IGNITE-7510 is fixed // suite.addTestSuite(IgnitePdsClientNearCachePutGetTest.class); suite.addTestSuite(IgniteDbPutGetWithCacheStoreTest.class); + suite.addTestSuite(IgnitePdsWithTtlTest.class); suite.addTestSuite(IgniteClusterActivateDeactivateTestWithPersistence.class); 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 71c6c32a983cb..cbe9720f9a10e 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 @@ -20,7 +20,7 @@ import junit.framework.TestSuite; import org.apache.ignite.internal.processors.cache.persistence.IgniteDataStorageMetricsSelfTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsContinuousRestartTest; -import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsContinuousRestartTest2; +import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsContinuousRestartTestWithExpiryPolicy; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsContinuousRestartTestWithSharedGroupAndIndexes; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsCorruptedStoreTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsExchangeDuringCheckpointTest; @@ -85,7 +85,7 @@ private static void addRealPageStoreTestsLongRunning(TestSuite suite) { // Rebalancing test suite.addTestSuite(IgnitePdsContinuousRestartTest.class); - suite.addTestSuite(IgnitePdsContinuousRestartTest2.class); + suite.addTestSuite(IgnitePdsContinuousRestartTestWithExpiryPolicy.class); suite.addTestSuite(IgnitePdsContinuousRestartTestWithSharedGroupAndIndexes.class); @@ -108,7 +108,6 @@ public static void addRealPageStoreTests(TestSuite suite) { suite.addTestSuite(IgnitePdsWholeClusterRestartTest.class); - // Rebalancing test suite.addTestSuite(IgniteWalHistoryReservationsTest.class); From e40efe88ce949e9234e58dc3497ecd8251004a95 Mon Sep 17 00:00:00 2001 From: dpavlov Date: Mon, 14 May 2018 13:39:21 +0300 Subject: [PATCH 0174/1463] IGNITE-8138 Fix for incorrect uptime in Ignite metrics for long running server node Cherry-picked from 4ea7f926e6f3803c616ab51e1d2e79765862efa1 --- .../ignite/internal/util/typedef/X.java | 44 ++++++++++++++++--- .../ignite/spi/IgniteSpiMBeanAdapter.java | 2 +- .../org/apache/ignite/lang/GridXSelfTest.java | 13 ++++++ 3 files changed, 53 insertions(+), 6 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java b/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java index 49732b6817630..be69b24821394 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java @@ -53,8 +53,24 @@ public final class X { /** An empty immutable {@code Object} array. */ public static final Object[] EMPTY_OBJECT_ARRAY = new Object[0]; + /** Millis in second. */ + private static final long MILLIS_IN_SECOND = 1000L; + + /** Seconds in minute. */ + private static final long SECONDS_IN_MINUTE = 60L; + + /** Minuses in hour. */ + private static final long MINUTES_IN_HOUR = 60L; + + /** Hours in day. */ + private static final long HOURS_IN_DAY = 24L; + /** Time span dividers. */ - private static final long[] SPAN_DIVS = new long[] {1000L, 60L, 60L, 60L}; + private static final long[] SPAN_DIVS = new long[] { + MILLIS_IN_SECOND, SECONDS_IN_MINUTE, MINUTES_IN_HOUR, HOURS_IN_DAY}; + + /** Millis in day. */ + private static final long MILLIS_IN_DAY = MILLIS_IN_SECOND * SECONDS_IN_MINUTE * MINUTES_IN_HOUR * HOURS_IN_DAY; /** The names of methods commonly used to access a wrapped exception. */ private static final String[] CAUSE_MTD_NAMES = new String[] { @@ -75,9 +91,6 @@ public final class X { /** The Method object for Java 1.4 getCause. */ private static final Method THROWABLE_CAUSE_METHOD; - /** - * - */ static { Method causeMtd; @@ -200,7 +213,7 @@ public static void printerr(@Nullable String s1, @Nullable Object... rest) { * Creates string presentation of given time {@code span} in hh:mm:ss.msec {@code HMSM} format. * * @param span Time span. - * @return String presentation. + * @return String presentation. If duration if longer than 1 day, days count is ignored. */ public static String timeSpan2HMSM(long span) { long[] t = new long[4]; @@ -235,6 +248,27 @@ public static String timeSpan2HMS(long span) { (t[1] < 10 ? "0" + t[1] : Long.toString(t[1])); } + /** + * Creates string presentation of given time {@code span} in days, hh:mm:ss.mmm {@code HMS} format. + * + * @param span Time span. + * @return String presentation. + */ + public static String timeSpan2DHMSM(long span) { + String days = ""; + + String hmsm = timeSpan2HMSM(span % MILLIS_IN_DAY); + + long daysCnt = span / MILLIS_IN_DAY; + + if (daysCnt == 1) + days = "1 day, "; + else if (daysCnt > 1) + days = daysCnt + " days, "; + + return days + hmsm; + } + /** * Clones a passed in object. If parameter {@code deep} is set to {@code true} * then this method will use deep cloning algorithm based on deep reflection diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiMBeanAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiMBeanAdapter.java index 8035333d984a4..f48b7e5839177 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiMBeanAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiMBeanAdapter.java @@ -46,7 +46,7 @@ public IgniteSpiMBeanAdapter(IgniteSpiAdapter spiAdapter) { /** {@inheritDoc} */ @Override public final String getUpTimeFormatted() { - return X.timeSpan2HMSM(getUpTime()); + return X.timeSpan2DHMSM(getUpTime()); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/lang/GridXSelfTest.java b/modules/core/src/test/java/org/apache/ignite/lang/GridXSelfTest.java index e7608b7ea625e..8fd6df638668d 100644 --- a/modules/core/src/test/java/org/apache/ignite/lang/GridXSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/lang/GridXSelfTest.java @@ -52,6 +52,19 @@ public void testHasCause() { assert gridEx.getCause(NumberFormatException.class) == null; } + /** + * Tests string presentation of given time. + */ + public void testTimeSpan() { + assertEquals(X.timeSpan2DHMSM(86400001L), "1 day, 00:00:00.001"); + + assertEquals(X.timeSpan2DHMSM(172800004L), "2 days, 00:00:00.004"); + + assertEquals(X.timeSpan2DHMSM(1L), "00:00:00.001"); + + assertEquals(X.timeSpan2HMSM(172800004L), "00:00:00.004"); + } + /** * */ From 0734883b0464b712b2dc35244a0060e8aaf1a12b Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Wed, 16 May 2018 15:25:51 +0300 Subject: [PATCH 0175/1463] IGNITE-8508 Proper ordering of ZK discovery custom events ACKs --- .../discovery/zk/internal/ZkDiscoveryEventsData.java | 11 +++++++++++ .../discovery/zk/internal/ZookeeperDiscoveryImpl.java | 2 +- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventsData.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventsData.java index dce861b523e4c..6520b8cb97cde 100644 --- a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventsData.java +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZkDiscoveryEventsData.java @@ -21,6 +21,8 @@ import java.util.Collection; import java.util.TreeMap; import java.util.UUID; + +import org.apache.ignite.internal.util.typedef.internal.S; import org.jetbrains.annotations.Nullable; /** @@ -118,4 +120,13 @@ void addEvent(Collection nodes, ZkDiscoveryEventData evt) evt.initRemainingAcks(nodes); } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(ZkDiscoveryEventsData.class, this, + "topVer", topVer, + "evtIdGen", evtIdGen, + "procCustEvt", procCustEvt, + "evts", evts); + } } 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 0604458e461b6..43d6aeb9f3a6a 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 @@ -3658,7 +3658,7 @@ private ZkDiscoveryCustomEventData createAckEvent( ZkDiscoveryCustomEventData ackEvtData = new ZkDiscoveryCustomEventData( evtId, origEvt.eventId(), - origEvt.topologyVersion(), // Use topology version from original event. + rtState.evtsData.topVer, // Use actual topology version because topology version must be growing. locNode.id(), null, null); From df32b485c787018a5bf261446155e13eb3c47e8d Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Tue, 24 Apr 2018 18:22:52 +0300 Subject: [PATCH 0176/1463] IGNITE-8313 Add trace logs on exchange phases and affinity calculation. - Fixes #3881. Signed-off-by: dpavlov (cherry picked from commit f4646e4) --- .../affinity/GridAffinityAssignmentCache.java | 37 ++++++++++++++ .../dht/GridDhtPartitionTopologyImpl.java | 51 +++++++++++++++++++ 2 files changed, 88 insertions(+) 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 2153e59f634bc..edbdaeebf090e 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 @@ -48,6 +48,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.S; +import org.apache.ignite.internal.util.typedef.internal.SB; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgnitePredicate; import org.jetbrains.annotations.Nullable; @@ -217,6 +218,12 @@ public void initialize(AffinityTopologyVersion topVer, List> a // In case if value was replaced there is no sense to clean the history. if (hAff == null) onHistoryAdded(); + + if (log.isTraceEnabled()) { + log.trace("New affinity assignment [grp=" + cacheOrGrpName + + ", topVer=" + topVer + + ", aff=" + fold(affAssignment) + "]"); + } } /** @@ -766,6 +773,36 @@ public Collection cachedVersions() { return affCache.keySet(); } + /** + * @param affAssignment Affinity assignment. + * @return String representation of given {@code affAssignment}. + */ + private static String fold(List> affAssignment) { + SB sb = new SB(); + + for (int p = 0; p < affAssignment.size(); p++) { + sb.a("Part ["); + sb.a("id=" + p + ", "); + + SB partOwners = new SB(); + + List affOwners = affAssignment.get(p); + + for (ClusterNode node : affOwners) { + partOwners.a(node.consistentId()); + partOwners.a(' '); + } + + sb.a("owners=["); + sb.a(partOwners); + sb.a(']'); + + sb.a("] "); + } + + return sb.toString(); + } + /** * Affinity ready future. Will remove itself from ready futures map. */ 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 aa32902a4ccdd..2ab838433729e 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 @@ -61,6 +61,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.LT; +import org.apache.ignite.internal.util.typedef.internal.SB; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.jetbrains.annotations.NotNull; @@ -556,6 +557,11 @@ else if (localNode(p, aff)) log.debug("Partition map after beforeExchange [grp=" + grp.cacheOrGroupName() + ", " + "exchId=" + exchFut.exchangeId() + ", fullMap=" + fullMapString() + ']'); } + + if (log.isTraceEnabled()) { + log.trace("Partition states after beforeExchange [grp=" + grp.cacheOrGroupName() + + ", exchId=" + exchFut.exchangeId() + ", states=" + dumpPartitionStates() + ']'); + } } finally { lock.writeLock().unlock(); @@ -687,6 +693,11 @@ private boolean partitionLocalNode(int p, AffinityTopologyVersion topVer) { ", fullMap=" + fullMapString() + ']'); } + if (log.isTraceEnabled()) { + log.trace("Partition states before afterExchange [grp=" + grp.cacheOrGroupName() + + ", exchVer=" + exchFut.exchangeId() + ", states=" + dumpPartitionStates() + ']'); + } + long updateSeq = this.updateSeq.incrementAndGet(); for (int p = 0; p < num; p++) { @@ -765,6 +776,11 @@ else if (log.isDebugEnabled()) updateRebalanceVersion(aff.assignment()); consistencyCheck(); + + if (log.isTraceEnabled()) { + log.trace("Partition states after afterExchange [grp=" + grp.cacheOrGroupName() + + ", exchVer=" + exchFut.exchangeId() + ", states=" + dumpPartitionStates() + ']'); + } } finally { lock.writeLock().unlock(); @@ -1327,6 +1343,11 @@ private boolean shouldOverridePartitionMap(GridDhtPartitionMap currentMap, GridD lock.writeLock().lock(); try { + if (log.isTraceEnabled() && exchangeVer != null) { + log.trace("Partition states before full update [grp=" + grp.cacheOrGroupName() + + ", exchVer=" + exchangeVer + ", states=" + dumpPartitionStates() + ']'); + } + if (stopping || !lastTopChangeVer.initialized() || // Ignore message not-related to exchange if exchange is in progress. (exchangeVer == null && !lastTopChangeVer.equals(readyTopVer))) @@ -1530,6 +1551,11 @@ else if (state == MOVING) { ", map=" + fullMapString() + ']'); } + if (log.isTraceEnabled() && exchangeVer != null) { + log.trace("Partition states after full update [grp=" + grp.cacheOrGroupName() + + ", exchVer=" + exchangeVer + ", states=" + dumpPartitionStates() + ']'); + } + if (changed) ctx.exchange().scheduleResendPartitions(); @@ -2770,6 +2796,31 @@ private void consistencyCheck() { // no-op } + /** + * Collects states of local partitions. + * + * @return String representation of all local partition states. + */ + private String dumpPartitionStates() { + SB sb = new SB(); + + for (int p = 0; p < locParts.length(); p++) { + GridDhtLocalPartition part = locParts.get(p); + + if (part == null) + continue; + + sb.a("Part ["); + sb.a("id=" + part.id() + ", "); + sb.a("state=" + part.state() + ", "); + sb.a("initCounter=" + part.initialUpdateCounter() + ", "); + sb.a("updCounter=" + part.updateCounter() + ", "); + sb.a("size=" + part.fullSize() + "] "); + } + + return sb.toString(); + } + /** * Iterator over current local partitions. */ From 26cb8d55f9cafbd3f8655915dd3ce826517a11f3 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Wed, 16 May 2018 20:13:30 +0300 Subject: [PATCH 0177/1463] IGNITE-8499 validate_indexes command doesn't detect absent rows in cache data tree (cherry picked from commit 88a6bfd) --- .../internal/commandline/CommandHandler.java | 21 +- .../ValidateIndexesPartitionResult.java | 31 +- .../verify/VisorValidateIndexesJobResult.java | 38 ++- .../visor/verify/ValidateIndexesClosure.java | 264 ++++++++++++++---- .../verify/VisorValidateIndexesTask.java | 6 +- .../IgniteCacheWithIndexingTestSuite.java | 3 + .../util/GridCommandHandlerIndexingTest.java | 203 +++++++++++++- 7 files changed, 477 insertions(+), 89 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java index 7d457fd1f1265..04578e53f916d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java @@ -100,8 +100,8 @@ import static org.apache.ignite.internal.commandline.Command.CACHE; import static org.apache.ignite.internal.commandline.Command.DEACTIVATE; import static org.apache.ignite.internal.commandline.Command.STATE; -import static org.apache.ignite.internal.commandline.Command.WAL; import static org.apache.ignite.internal.commandline.Command.TX; +import static org.apache.ignite.internal.commandline.Command.WAL; import static org.apache.ignite.internal.visor.baseline.VisorBaselineOperation.ADD; import static org.apache.ignite.internal.visor.baseline.VisorBaselineOperation.COLLECT; import static org.apache.ignite.internal.visor.baseline.VisorBaselineOperation.REMOVE; @@ -635,9 +635,9 @@ private void cacheValidateIndexes(GridClient client, CacheArguments cacheArgs) t boolean errors = false; for (Map.Entry nodeEntry : taskRes.results().entrySet()) { - Map map = nodeEntry.getValue().response(); + Map partRes = nodeEntry.getValue().partitionResult(); - for (Map.Entry e : map.entrySet()) { + for (Map.Entry e : partRes.entrySet()) { ValidateIndexesPartitionResult res = e.getValue(); if (!res.issues().isEmpty()) { @@ -649,6 +649,21 @@ private void cacheValidateIndexes(GridClient client, CacheArguments cacheArgs) t log(is.toString()); } } + + Map idxRes = nodeEntry.getValue().indexResult(); + + for (Map.Entry e : idxRes.entrySet()) { + ValidateIndexesPartitionResult res = e.getValue(); + + if (!res.issues().isEmpty()) { + errors = true; + + log("SQL Index " + e.getKey() + " " + e.getValue().toString()); + + for (IndexValidationIssue is : res.issues()) + log(is.toString()); + } + } } if (!errors) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesPartitionResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesPartitionResult.java index 18899607110fc..5d74a57e14ca4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesPartitionResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesPartitionResult.java @@ -29,7 +29,7 @@ import org.apache.ignite.internal.visor.VisorDataTransferObject; /** - * + * Encapsulates intermediate results of validation of SQL index (if {@link #sqlIdxName} is present) or partition. */ public class ValidateIndexesPartitionResult extends VisorDataTransferObject { /** */ @@ -52,6 +52,10 @@ public class ValidateIndexesPartitionResult extends VisorDataTransferObject { @GridToStringExclude private List issues = new ArrayList<>(10); + /** Sql index name. */ + @GridToStringExclude + private String sqlIdxName; + /** * */ @@ -64,12 +68,15 @@ public ValidateIndexesPartitionResult() { * @param size Size. * @param isPrimary Is primary. * @param consistentId Consistent id. + * @param sqlIdxName Sql index name (optional). */ - public ValidateIndexesPartitionResult(long updateCntr, long size, boolean isPrimary, Object consistentId) { + public ValidateIndexesPartitionResult(long updateCntr, long size, boolean isPrimary, Object consistentId, + String sqlIdxName) { this.updateCntr = updateCntr; this.size = size; this.isPrimary = isPrimary; this.consistentId = consistentId; + this.sqlIdxName = sqlIdxName; } /** @@ -107,6 +114,13 @@ public List issues() { return issues; } + /** + * @return null for partition validation result, SQL index name for index validation result + */ + public String sqlIndexName() { + return sqlIdxName; + } + /** * @param t Issue. * @return True if there are already enough issues. @@ -120,6 +134,11 @@ public boolean reportIssue(IndexValidationIssue t) { return false; } + /** {@inheritDoc} */ + @Override public byte getProtocolVersion() { + return V2; + } + /** {@inheritDoc} */ @Override protected void writeExternalData(ObjectOutput out) throws IOException { out.writeLong(updateCntr); @@ -127,6 +146,7 @@ public boolean reportIssue(IndexValidationIssue t) { out.writeBoolean(isPrimary); out.writeObject(consistentId); U.writeCollection(out, issues); + U.writeString(out, sqlIdxName); } /** {@inheritDoc} */ @@ -136,10 +156,15 @@ public boolean reportIssue(IndexValidationIssue t) { isPrimary = in.readBoolean(); consistentId = in.readObject(); issues = U.readList(in); + + if (protoVer >= V2) + sqlIdxName = U.readString(in); } /** {@inheritDoc} */ @Override public String toString() { - return S.toString(ValidateIndexesPartitionResult.class, this); + return sqlIdxName == null ? S.toString(ValidateIndexesPartitionResult.class, this) : + ValidateIndexesPartitionResult.class.getSimpleName() + " [consistentId=" + consistentId + + ", sqlIdxName=" + sqlIdxName + "]"; } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesJobResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesJobResult.java index 25c97b651a4d9..aa74323898bd8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesJobResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesJobResult.java @@ -34,13 +34,19 @@ public class VisorValidateIndexesJobResult extends VisorDataTransferObject { private static final long serialVersionUID = 0L; /** Results of indexes validation from node. */ - private Map res; + private Map partRes; + + /** Results of reverse indexes validation from node. */ + private Map idxRes; /** - * @param res Results of indexes validation from node. + * @param partRes Results of indexes validation from node. + * @param idxRes Results of reverse indexes validation from node. */ - public VisorValidateIndexesJobResult(Map res) { - this.res = res; + public VisorValidateIndexesJobResult(Map partRes, + Map idxRes) { + this.partRes = partRes; + this.idxRes = idxRes; } /** @@ -49,21 +55,37 @@ public VisorValidateIndexesJobResult(Map response() { - return res; + public Map partitionResult() { + return partRes; + } + + /** + * @return Results of reverse indexes validation from node. + */ + public Map indexResult() { + return idxRes; } /** {@inheritDoc} */ @Override protected void writeExternalData(ObjectOutput out) throws IOException { - U.writeMap(out, res); + U.writeMap(out, partRes); + U.writeMap(out, idxRes); } /** {@inheritDoc} */ @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { - res = U.readMap(in); + partRes = U.readMap(in); + + if (protoVer >= V2) + idxRes = U.readMap(in); } /** {@inheritDoc} */ diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java index 373bd15dad315..e0eff612fad2a 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java @@ -31,9 +31,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteInterruptedException; @@ -51,12 +50,15 @@ import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.verify.PartitionKey; import org.apache.ignite.internal.processors.query.GridQueryProcessor; +import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor; import org.apache.ignite.internal.processors.query.QueryTypeDescriptorImpl; import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing; 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.GridIterator; +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.IgniteCallable; import org.apache.ignite.resources.IgniteInstanceResource; @@ -66,9 +68,13 @@ import org.h2.index.Index; /** - * + * Closure that locally validates indexes of given caches. + * Validation consists of three checks: + * 1. If entry is present in cache data tree, it's reachable from all cache SQL indexes + * 2. If entry is present in cache SQL index, it can be dereferenced with link from index + * 3. If entry is present in cache SQL index, it's present in cache data tree */ -public class ValidateIndexesClosure implements IgniteCallable> { +public class ValidateIndexesClosure implements IgniteCallable { /** */ private static final long serialVersionUID = 0L; @@ -84,7 +90,19 @@ public class ValidateIndexesClosure implements IgniteCallable cacheNames; /** Counter of processed partitions. */ - private final AtomicInteger completionCntr = new AtomicInteger(0); + private final AtomicInteger processedPartitions = new AtomicInteger(0); + + /** Total partitions. */ + private volatile int totalPartitions; + + /** Counter of processed indexes. */ + private final AtomicInteger processedIndexes = new AtomicInteger(0); + + /** Total partitions. */ + private volatile int totalIndexes; + + /** Last progress print timestamp. */ + private final AtomicLong lastProgressPrintTs = new AtomicLong(0); /** Calculation executor. */ private volatile ExecutorService calcExecutor; @@ -97,7 +115,7 @@ public ValidateIndexesClosure(Set cacheNames) { } /** {@inheritDoc} */ - @Override public Map call() throws Exception { + @Override public VisorValidateIndexesJobResult call() throws Exception { calcExecutor = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors()); try { @@ -111,7 +129,7 @@ public ValidateIndexesClosure(Set cacheNames) { /** * */ - private Map call0() throws Exception { + private VisorValidateIndexesJobResult call0() throws Exception { Set grpIds = new HashSet<>(); Set missingCaches = new HashSet<>(); @@ -150,8 +168,9 @@ private Map call0() throws Excepti } List>> procPartFutures = new ArrayList<>(); - - completionCntr.set(0); + List>> procIdxFutures = new ArrayList<>(); + List> partArgs = new ArrayList<>(); + List> idxArgs = new ArrayList<>(); for (Integer grpId : grpIds) { CacheGroupContext grpCtx = ignite.context().cache().cacheGroup(grpId); @@ -162,45 +181,82 @@ private Map call0() throws Excepti List parts = grpCtx.topology().localPartitions(); for (GridDhtLocalPartition part : parts) - procPartFutures.add(processPartitionAsync(grpCtx, part)); - } + partArgs.add(new T2<>(grpCtx, part)); - Map res = new HashMap<>(); + GridQueryProcessor qry = ignite.context().query(); - long lastProgressLogTs = U.currentTimeMillis(); + IgniteH2Indexing indexing = (IgniteH2Indexing)qry.getIndexing(); - for (int i = 0; i < procPartFutures.size(); ) { - Future> fut = procPartFutures.get(i); + for (GridCacheContext ctx : grpCtx.caches()) { + Collection types = qry.types(ctx.name()); - try { - Map partRes = fut.get(1, TimeUnit.SECONDS); + if (!F.isEmpty(types)) { + for (GridQueryTypeDescriptor type : types) { + GridH2Table gridH2Tbl = indexing.dataTable(ctx.name(), type.tableName()); + + if (gridH2Tbl == null) + continue; - res.putAll(partRes); + ArrayList indexes = gridH2Tbl.getIndexes(); - i++; + for (Index idx : indexes) + idxArgs.add(new T2<>(ctx, idx)); + } + } } - catch (InterruptedException | ExecutionException e) { - for (int j = i + 1; j < procPartFutures.size(); j++) - procPartFutures.get(j).cancel(false); - - if (e instanceof InterruptedException) - throw new IgniteInterruptedException((InterruptedException)e); - else if (e.getCause() instanceof IgniteException) - throw (IgniteException)e.getCause(); - else - throw new IgniteException(e.getCause()); + } + + // To decrease contention on same indexes. + Collections.shuffle(partArgs); + Collections.shuffle(idxArgs); + + for (T2 t2 : partArgs) + procPartFutures.add(processPartitionAsync(t2.get1(), t2.get2())); + + for (T2 t2 : idxArgs) + procIdxFutures.add(processIndexAsync(t2.get1(), t2.get2())); + + totalPartitions = procPartFutures.size(); + totalIndexes = procIdxFutures.size(); + + Map partResults = new HashMap<>(); + Map idxResults = new HashMap<>(); + + int curPart = 0; + int curIdx = 0; + try { + for (; curPart < procPartFutures.size(); curPart++) { + Future> fut = procPartFutures.get(curPart); + + Map partRes = fut.get(); + + partResults.putAll(partRes); } - catch (TimeoutException ignored) { - if (U.currentTimeMillis() - lastProgressLogTs > 60 * 1000L) { - lastProgressLogTs = U.currentTimeMillis(); - log.warning("ValidateIndexesClosure is still running, processed " + completionCntr.get() + " of " + - procPartFutures.size() + " local partitions"); - } + for (; curIdx < procIdxFutures.size(); curIdx++) { + Future> fut = procIdxFutures.get(curIdx); + + Map idxRes = fut.get(); + + idxResults.putAll(idxRes); } } + catch (InterruptedException | ExecutionException e) { + for (int j = curPart; j < procPartFutures.size(); j++) + procPartFutures.get(j).cancel(false); + + for (int j = curIdx; j < procIdxFutures.size(); j++) + procIdxFutures.get(j).cancel(false); + + if (e instanceof InterruptedException) + throw new IgniteInterruptedException((InterruptedException)e); + else if (e.getCause() instanceof IgniteException) + throw (IgniteException)e.getCause(); + else + throw new IgniteException(e.getCause()); + } - return res; + return new VisorValidateIndexesJobResult(partResults, idxResults); } /** @@ -245,12 +301,24 @@ private Map processPartition( boolean isPrimary = part.primary(grpCtx.topology().readyTopologyVersion()); - partRes = new ValidateIndexesPartitionResult(updateCntrBefore, partSize, isPrimary, consId); + partRes = new ValidateIndexesPartitionResult(updateCntrBefore, partSize, isPrimary, consId, null); boolean enoughIssues = false; - long keysProcessed = 0; - long lastProgressLog = U.currentTimeMillis(); + GridQueryProcessor qryProcessor = ignite.context().query(); + + Method m; + try { + m = GridQueryProcessor.class.getDeclaredMethod("typeByValue", String.class, + CacheObjectContext.class, KeyCacheObject.class, CacheObject.class, boolean.class); + } + catch (NoSuchMethodException e) { + log.error("Failed to invoke typeByValue", e); + + throw new IgniteException(e); + } + + m.setAccessible(true); while (it.hasNextX()) { if (enoughIssues) @@ -266,14 +334,7 @@ private Map processPartition( if (cacheCtx == null) throw new IgniteException("Unknown cacheId of CacheDataRow: " + cacheId); - GridQueryProcessor qryProcessor = ignite.context().query(); - try { - Method m = GridQueryProcessor.class.getDeclaredMethod("typeByValue", String.class, - CacheObjectContext.class, KeyCacheObject.class, CacheObject.class, boolean.class); - - m.setAccessible(true); - QueryTypeDescriptorImpl res = (QueryTypeDescriptorImpl)m.invoke( qryProcessor, cacheCtx.name(), cacheCtx.cacheObjectContext(), row.key(), row.value(), true); @@ -298,7 +359,7 @@ private Map processPartition( Cursor cursor = idx.find((Session) null, h2Row, h2Row); if (cursor == null || !cursor.next()) - throw new IgniteCheckedException("Key not found."); + throw new IgniteCheckedException("Key is present in CacheDataTree, but can't be found in SQL index."); } catch (Throwable t) { Object o = CacheObjectUtils.unwrapBinaryIfNeeded( @@ -313,7 +374,7 @@ private Map processPartition( } } } - catch (IllegalAccessException | NoSuchMethodException e) { + catch (IllegalAccessException e) { log.error("Failed to invoke typeByValue", e); throw new IgniteException(e); @@ -325,16 +386,6 @@ private Map processPartition( throw new IgniteException(target); } - finally { - keysProcessed++; - - if (U.currentTimeMillis() - lastProgressLog >= 60_000 && partSize > 0) { - log.warning("Processing partition " + part.id() + " (" + (keysProcessed * 100 / partSize) + - "% " + keysProcessed + "/" + partSize + ")"); - - lastProgressLog = U.currentTimeMillis(); - } - } } } catch (IgniteCheckedException e) { @@ -345,12 +396,107 @@ private Map processPartition( } finally { part.release(); + + printProgressIfNeeded(); } PartitionKey partKey = new PartitionKey(grpCtx.groupId(), part.id(), grpCtx.cacheOrGroupName()); - completionCntr.incrementAndGet(); + processedPartitions.incrementAndGet(); return Collections.singletonMap(partKey, partRes); } + + /** + * + */ + private void printProgressIfNeeded() { + long curTs = U.currentTimeMillis(); + + long lastTs = lastProgressPrintTs.get(); + + if (curTs - lastTs >= 60_000 && lastProgressPrintTs.compareAndSet(lastTs, curTs)) { + log.warning("Current progress of ValidateIndexesClosure: processed " + + processedPartitions.get() + " of " + totalPartitions + " partitions, " + + processedIndexes.get() + " of " + totalIndexes + " SQL indexes"); + } + } + + /** + * @param ctx Context. + * @param idx Index. + */ + private Future> processIndexAsync(GridCacheContext ctx, Index idx) { + return calcExecutor.submit(new Callable>() { + @Override public Map call() throws Exception { + return processIndex(ctx, idx); + } + }); + } + + /** + * @param ctx Context. + * @param idx Index. + */ + private Map processIndex(GridCacheContext ctx, Index idx) { + Object consId = ignite.context().discovery().localNode().consistentId(); + + ValidateIndexesPartitionResult idxValidationRes = new ValidateIndexesPartitionResult( + -1, -1, true, consId, idx.getName()); + + boolean enoughIssues = false; + + Cursor cursor = null; + + try { + cursor = idx.find((Session)null, null, null); + + if (cursor == null) + throw new IgniteCheckedException("Can't iterate through index: " + idx); + } + catch (Throwable t) { + IndexValidationIssue is = new IndexValidationIssue(null, ctx.name(), idx.getName(), t); + + log.error("Find in index failed: " + is.toString()); + + enoughIssues = true; + } + + while (!enoughIssues) { + KeyCacheObject h2key = null; + + try { + if (!cursor.next()) + break; + + GridH2Row h2Row = (GridH2Row)cursor.get(); + + h2key = h2Row.key(); + + CacheDataRow cacheDataStoreRow = ctx.group().offheap().read(ctx, h2key); + + if (cacheDataStoreRow == null) + throw new IgniteCheckedException("Key is present in SQL index, but can't be found in CacheDataTree."); + } + catch (Throwable t) { + Object o = CacheObjectUtils.unwrapBinaryIfNeeded( + ctx.cacheObjectContext(), h2key, true, true); + + IndexValidationIssue is = new IndexValidationIssue( + String.valueOf(o), ctx.name(), idx.getName(), t); + + log.error("Failed to lookup key: " + is.toString()); + + enoughIssues |= idxValidationRes.reportIssue(is); + } + } + + String uniqueIdxName = "[cache=" + ctx.name() + ", idx=" + idx.getName() + "]"; + + processedIndexes.incrementAndGet(); + + printProgressIfNeeded(); + + return Collections.singletonMap(uniqueIdxName, idxValidationRes); + } } diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTask.java b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTask.java index 1a89c2cadb57e..52b48a58c0340 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTask.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTask.java @@ -23,7 +23,6 @@ import java.util.UUID; import org.apache.ignite.IgniteException; import org.apache.ignite.compute.ComputeJobResult; -import org.apache.ignite.internal.processors.cache.verify.PartitionKey; import org.apache.ignite.internal.processors.task.GridInternal; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.visor.VisorJob; @@ -81,10 +80,7 @@ protected VisorValidateIndexesJob(@Nullable VisorValidateIndexesTaskArg arg, boo ignite.context().resource().injectGeneric(clo); - Map res = clo.call(); - - return new VisorValidateIndexesJobResult(res); - + return clo.call(); } catch (Exception e) { throw new IgniteException(e); diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java index bc99981372491..c89673626e83a 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java @@ -39,6 +39,7 @@ import org.apache.ignite.internal.processors.cache.ttl.CacheTtlTransactionalPartitionedSelfTest; import org.apache.ignite.internal.processors.client.IgniteDataStreamerTest; import org.apache.ignite.internal.processors.query.h2.database.InlineIndexHelperTest; +import org.apache.ignite.util.GridCommandHandlerIndexingTest; /** * Cache tests using indexing. @@ -81,6 +82,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteDataStreamerTest.class); + suite.addTestSuite(GridCommandHandlerIndexingTest.class); + return suite; } } diff --git a/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java b/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java index 9e9c7770a644c..62d3fc02bdbaa 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java @@ -19,17 +19,32 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import java.util.concurrent.ThreadLocalRandom; +import javax.cache.Cache; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cache.QueryEntity; import org.apache.ignite.cache.QueryIndex; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.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.distributed.dht.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.tree.SearchRow; +import org.apache.ignite.internal.processors.query.GridQueryProcessor; +import org.apache.ignite.internal.util.lang.GridIterator; 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 static org.apache.ignite.internal.commandline.CommandHandler.EXIT_CODE_OK; @@ -38,35 +53,201 @@ */ public class GridCommandHandlerIndexingTest extends GridCommandHandlerTest { /** - * + * Tests that validation doesn't fail if nothing is broken. */ - public void testValidateIndexes() throws Exception { + public void testValidateIndexesNoErrors() throws Exception { Ignite ignite = startGrids(2); ignite.cluster().active(true); Ignite client = startGrid("client"); - IgniteCache personCache = client.getOrCreateCache(new CacheConfiguration() - .setName("persons-cache-vi") - .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC) - .setAtomicityMode(CacheAtomicityMode.ATOMIC) - .setBackups(1) - .setQueryEntities(F.asList(personEntity(true, true))) - .setAffinity(new RendezvousAffinityFunction(false, 32))); + String cacheName = "persons-cache-vi"; + + IgniteCache personCache = createPersonCache(client, cacheName); ThreadLocalRandom rand = ThreadLocalRandom.current(); - for (int i = 0; i < 1000; i++) + for (int i = 0; i < 10_000; i++) personCache.put(i, new Person(rand.nextInt(), String.valueOf(rand.nextLong()))); injectTestSystemOut(); - assertEquals(EXIT_CODE_OK, execute("--cache", "validate_indexes", "persons-cache-vi")); + assertEquals(EXIT_CODE_OK, execute("--cache", "validate_indexes", cacheName)); assertTrue(testOut.toString().contains("validate_indexes has finished, no issues found")); } + /** + * Tests that missing rows in CacheDataTree are detected. + */ + public void testBrokenCacheDataTreeShouldFailValidation() throws Exception { + Ignite ignite = startGrids(2); + + ignite.cluster().active(true); + + Ignite client = startGrid("client"); + + String cacheName = "persons-cache-vi"; + + IgniteCache personCache = createPersonCache(client, cacheName); + + ThreadLocalRandom rand = ThreadLocalRandom.current(); + + for (int i = 0; i < 10_000; i++) + personCache.put(i, new Person(rand.nextInt(), String.valueOf(rand.nextLong()))); + + breakCacheDataTree(ignite, cacheName, 1); + + injectTestSystemOut(); + + assertEquals(EXIT_CODE_OK, execute("--cache", "validate_indexes", cacheName)); + + assertTrue(testOut.toString().contains("validate_indexes has finished with errors")); + } + + /** + * Tests that missing rows in H2 indexes are detected. + */ + public void testBrokenSqlIndexShouldFailValidation() throws Exception { + Ignite ignite = startGrids(2); + + ignite.cluster().active(true); + + Ignite client = startGrid("client"); + + String cacheName = "persons-cache-vi"; + + IgniteCache personCache = createPersonCache(client, cacheName); + + ThreadLocalRandom rand = ThreadLocalRandom.current(); + + for (int i = 0; i < 10_000; i++) + personCache.put(i, new Person(rand.nextInt(), String.valueOf(rand.nextLong()))); + + breakSqlIndex(ignite, cacheName); + + injectTestSystemOut(); + + assertEquals(EXIT_CODE_OK, execute("--cache", "validate_indexes", cacheName)); + + assertTrue(testOut.toString().contains("validate_indexes has finished with errors")); + } + + /** + * Removes some entries from a partition skipping index update. This effectively breaks the index. + */ + private void breakCacheDataTree(Ignite ig, String cacheName, int partId) { + IgniteEx ig0 = (IgniteEx)ig; + int cacheId = CU.cacheId(cacheName); + + ScanQuery scanQry = new ScanQuery(partId); + + GridCacheContext ctx = ig0.context().cache().context().cacheContext(cacheId); + + // Get current update counter + String grpName = ig0.context().cache().context().cacheContext(cacheId).config().getGroupName(); + int cacheGrpId = grpName == null ? cacheName.hashCode() : grpName.hashCode(); + + GridDhtLocalPartition locPart = ctx.dht().topology().localPartition(partId); + IgniteCacheOffheapManager.CacheDataStore dataStore = ig0.context().cache().context().cache().cacheGroup(cacheGrpId).offheap().dataStore(locPart); + + Iterator it = ig.cache(cacheName).withKeepBinary().query(scanQry).iterator(); + + for (int i = 0; i < 5_000; i++) { + if (it.hasNext()) { + Cache.Entry entry = it.next(); + + if (i % 5 == 0) { + // Do update + GridCacheDatabaseSharedManager db = (GridCacheDatabaseSharedManager)ig0.context().cache().context().database(); + + db.checkpointReadLock(); + + try { + IgniteCacheOffheapManager.CacheDataStore innerStore = U.field(dataStore, "delegate"); + + // IgniteCacheOffheapManagerImpl.CacheDataRowStore + Object rowStore = U.field(innerStore, "rowStore"); + + // IgniteCacheOffheapManagerImpl.CacheDataTree + Object dataTree = U.field(innerStore, "dataTree"); + + CacheDataRow oldRow = U.invoke( + dataTree.getClass(), + dataTree, + "remove", + new SearchRow(cacheId, ctx.toCacheKeyObject(entry.getKey()))); + + if (oldRow != null) + U.invoke(rowStore.getClass(), rowStore, "removeRow", oldRow.link()); + } + catch (IgniteCheckedException e) { + System.out.println("Failed to remove key skipping indexes: " + entry); + + e.printStackTrace(); + } + finally { + db.checkpointReadUnlock(); + } + } + } + else { + System.out.println("Early exit for index corruption, keys processed: " + i); + + break; + } + } + } + + /** + * Removes some entries from H2 trees skipping partition updates. This effectively breaks the index. + */ + private void breakSqlIndex(Ignite ig, String cacheName) throws Exception { + GridQueryProcessor qry = ((IgniteEx)ig).context().query(); + + GridCacheContext ctx = ((IgniteEx)ig).cachex(cacheName).context(); + + GridDhtLocalPartition locPart = ctx.topology().localPartitions().get(0); + + GridIterator it = ctx.group().offheap().partitionIterator(locPart.id()); + + for (int i = 0; i < 500; i++) { + if (!it.hasNextX()) { + System.out.println("Early exit for index corruption, keys processed: " + i); + + break; + } + + CacheDataRow row = it.nextX(); + + ctx.shared().database().checkpointReadLock(); + + try { + qry.remove(ctx, row); + } + finally { + ctx.shared().database().checkpointReadUnlock(); + } + } + } + + /** + * Dynamically creates cache with SQL indexes. + * + * @param ig Client. + * @param cacheName Cache name. + */ + private IgniteCache createPersonCache(Ignite ig, String cacheName) { + return ig.getOrCreateCache(new CacheConfiguration() + .setName(cacheName) + .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC) + .setAtomicityMode(CacheAtomicityMode.ATOMIC) + .setBackups(1) + .setQueryEntities(F.asList(personEntity(true, true))) + .setAffinity(new RendezvousAffinityFunction(false, 32))); + } + /** * @param idxName Index name. * @param idxOrgId Index org id. From 942dae0d8051d5d36c074e85bc09f344f5e648c0 Mon Sep 17 00:00:00 2001 From: Ilya Borisov Date: Wed, 16 May 2018 22:22:17 +0700 Subject: [PATCH 0178/1463] IGNITE-8479 Web Console: Fixed 'ignite-ace' delay on queries notebook page paragraph unfold. (cherry picked from commit d8c4fc8) --- .../page-queries/components/queries-notebook/template.tpl.pug | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/template.tpl.pug b/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/template.tpl.pug index 96c704d8d731d..1fef75c34ff71 100644 --- a/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/template.tpl.pug +++ b/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/template.tpl.pug @@ -304,7 +304,7 @@ mixin paragraph-scan mixin paragraph-query .row.panel-heading(bs-collapse-toggle) +paragraph-rename - .panel-collapse(role='tabpanel' bs-collapse-target) + .panel-collapse.ng-animate-disabled(role='tabpanel' bs-collapse-target) .col-sm-12 .col-xs-8.col-sm-9(style='border-right: 1px solid #eee') .sql-editor(ignite-ace='{onLoad: aceInit(paragraph), theme: "chrome", mode: "sql", require: ["ace/ext/language_tools"],' + From db62b1bd42d0fae6e15779671e8730b714fdd494 Mon Sep 17 00:00:00 2001 From: Andrey Novikov Date: Wed, 16 May 2018 22:52:09 +0700 Subject: [PATCH 0179/1463] IGNITE-8489 Web Console: Removed retina display support. (cherry picked from commit 771e861) --- modules/web-console/frontend/app/app.js | 1 - .../app/modules/branding/header-logo.pug | 2 +- .../app/modules/branding/powered-by-apache.pug | 2 +- modules/web-console/frontend/app/vendor.js | 1 - modules/web-console/frontend/package.json | 1 - .../frontend/public/images/pb-ignite@2x.png | Bin 8558 -> 0 bytes 6 files changed, 2 insertions(+), 5 deletions(-) delete mode 100644 modules/web-console/frontend/public/images/pb-ignite@2x.png diff --git a/modules/web-console/frontend/app/app.js b/modules/web-console/frontend/app/app.js index 350005feb5b01..b4e71faae063f 100644 --- a/modules/web-console/frontend/app/app.js +++ b/modules/web-console/frontend/app/app.js @@ -168,7 +168,6 @@ angular.module('ignite-console', [ 'dndLists', 'gridster', 'mgcrea.ngStrap', - 'ngRetina', 'nvd3', 'pascalprecht.translate', 'smart-table', diff --git a/modules/web-console/frontend/app/modules/branding/header-logo.pug b/modules/web-console/frontend/app/modules/branding/header-logo.pug index b58f6706e48bf..f0453d39da8ab 100644 --- a/modules/web-console/frontend/app/modules/branding/header-logo.pug +++ b/modules/web-console/frontend/app/modules/branding/header-logo.pug @@ -15,4 +15,4 @@ limitations under the License. a(ui-sref='signin') - img.navbar-brand(ng-src='{{logo.url}}' height='40') + img.navbar-brand(ng-src='{{logo.url}}') diff --git a/modules/web-console/frontend/app/modules/branding/powered-by-apache.pug b/modules/web-console/frontend/app/modules/branding/powered-by-apache.pug index af9aadfdb1e29..6031235968c8a 100644 --- a/modules/web-console/frontend/app/modules/branding/powered-by-apache.pug +++ b/modules/web-console/frontend/app/modules/branding/powered-by-apache.pug @@ -15,4 +15,4 @@ limitations under the License. a(ng-if='poweredBy.show' href='//ignite.apache.org' target='_blank') - img(ng-src='/images/pb-ignite.png' height='65') + img(ng-src='/images/pb-ignite.png') diff --git a/modules/web-console/frontend/app/vendor.js b/modules/web-console/frontend/app/vendor.js index 58b1edefaadc0..84dea92ee812c 100644 --- a/modules/web-console/frontend/app/vendor.js +++ b/modules/web-console/frontend/app/vendor.js @@ -24,7 +24,6 @@ import 'angular-sanitize'; import 'angular-strap'; import 'angular-strap/dist/angular-strap.tpl'; import 'angular-socket-io'; -import 'angular-retina'; import 'angular-messages'; import '@uirouter/angularjs'; diff --git a/modules/web-console/frontend/package.json b/modules/web-console/frontend/package.json index 87a516f584d78..f9d0b489bc5b8 100644 --- a/modules/web-console/frontend/package.json +++ b/modules/web-console/frontend/package.json @@ -47,7 +47,6 @@ "angular-messages": "1.6.9", "angular-motion": "0.4.4", "angular-nvd3": "1.0.9", - "angular-retina": "0.4.0", "angular-sanitize": "1.6.6", "angular-smart-table": "2.1.8", "angular-socket-io": "0.7.0", diff --git a/modules/web-console/frontend/public/images/pb-ignite@2x.png b/modules/web-console/frontend/public/images/pb-ignite@2x.png deleted file mode 100644 index ffcff3817b991d73afc3d169ec4d5e97a13a48d3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8558 zcmYj$Wl$V#7wiIyySux)ySqamc#wn;+}&jtcbDLQMZ{4ms z{d6CznfW&}U@Z+rbQBU4004lltR$!Neun-pkl^0AXKnKj00673rKTtUZsFkmmwaae z0s5{(g6N_jmmte*XMvXJ;oNA@OeCr`{VnIy(L< z_4f7#1_o+tYrp5;m-zVjyo>+ZY;0`)>v`wjzkk=&)x8hCgKJuP{ zUR=Bj7#Ns${dRxhHZ*TA0USD6|pl`UixM*l-SXfxd$jEebbkEPv z4*&oY6B8vRWol|FIXO8gDe3+FeSCa8H8u6y>+9|9?e+CF0|P^Pdiv$%<;lrObaZr9 zR@VCZdQDADb8|BY1j@_H`}OPB;NW0kVPRxsq=$zG1OjPoZFP2bK07-L2?>#tlk@lY zpP8A_(9q!Ix@yT;u{!?Poqm#>Xc}4WH&HH@;{M>akbOBKK7w9{R9&?}- z)%8&5kAeZ{_^&Q#8?@v-odNy%zv_QT{onE7Jrbc%A2cX*eR1mDT$$lRPj}vhvOM(m zunao$Ed~l@q=Q03p;DMoVRnl5lBh4a<>d`>*uQ?baM z-8jEqzXkF8Rnqe(R`>h0qyE=FY8M_S})ar zvp~0U_W8XDmTq zbwEx6bADqt4X zYe`@~9y7*>6@;oB-*EPi*p(M|1BUt7C<~KLr(Cbr= zBB>m?2m)1lG84<_xKLm1t2j1ZiD(KotU$4})2OUkAm&r2JgA%fF|8bSQ~9?{=3#ZI z1od#`T0P7Kf{a9R!g|Ju8mEQy3p&u4R|iGo_Xo7Oe5vLj$HTw6a`}Kb1bJ%iN5EF3 z^Zh0SA1@3Yj|v`U`!@Uq1vCrKOl&;8V=nJ}kBim+c^+x{pMYC_@F*iwVUCJrk^kR` zK8VB-T_*<(t$RrPLi94hINh(W%s;M3w2eiTbC%Gg#X=a~NJ97BqU~lnFtsYjR0*%X zMI0RATVCLp7JdsT`i+fiNrL5XQ{;uPU!@{{j*TDsGj2dRZm$?1jlc}L$u1h&eV}6e1F(^1|8+T~e8( zJs^lEI!)`{RyLchR)@miL`v3{uvo}Ni;?!U3?GLjV_)QW#PfyH?^f1cft5Wh55fbU zTSjm<>YREsR&T^l!2Y6~f&e;Si!e9is}RXgU0JmsOL+9ne&A7{F(0Z+UhHlWm}yoP zF#?36OL}^YVBsG#d1tX&sqwyGA@3GFP?H_3 z$B(zJ3{WiYROV0g__TEKh;D2gqmKiK$-ue=kORe9nJ{z;7|eOP65 zU$K}is(IVuDhg>`t<8G#Gmm$iJJ3{^jB3Rt(U=4bnvu8TBmoWgu!C8p+s_oNI=0kXhAtg&LWu9A1N(f+2;GkTq{+m^NsIy z5-_2l{Zb0H(i+Y5YAcl50c$L}!o{T6<#V zztmofRG330HQ+q~YNK;BdY2;T1i<}>BLY6O)W1$)Ew4JR<^hjc-1zjIx))&x?yX?w zQuSaRa|o0J((X3ZTtep~k@A^XmL`xyoF!xU%#GIEN!ueTs+Oj4zXuD>X~+?NW3GvwY=8GW%H)TGwN0<|^gNe8&cyX5J{?o8vC_*GLMs}B0R zmke~}6c zddE%AC_A(>N_gg#;ZyaztV)7}312bcaT0gDAQ3u46~>?XV3~cO44JYk3m&}THA8#~ zebbS9Sg6-voK4T!P5#p;Vjaaf6JhR!Y8S%Gn+VS`FZ+@prX%Wxa$psUY(;NntZkme zO!Dy>y!^#P%X(tc;9>yMPr}Xl9-1)?3e4O>wuQk$@P`RF#XC>t7y3p0x1m#1Q@?%0 z30450F-{s$(W22s5i9&GQJQ+0I|tLblNfRHY2d> z6#a4+^o9AuEw-kSTStk=itJsNsjW^>JbZ1-F7_6aBc{L!kD7>uy@%1l{y&@}MX<0I z8C8#}tHUiC2dKerLs!XI{GxK))83w1nPY#EiC1@|YGELaqqN8}Q6>HvIwF$D!tM`( z!OkmLWcA>3DxelAW^+;m1>Hn!zY`}~zib_Tz81EviMuRBs)F-0)Ymz{6*2Ux6Clt? zlph^~%tRxNu3*iG4Kt2t^gud|>xRvyR~oGm5K&RTJ|#@Vk2NuUSf9 zZo2MwS#OTPtM4^rnPSh8*FrBC0>6fCe=G5UdgHS7ft|3;PDeTM>byXPVduJ_tkL2h z5h*Fk_=92TMT(B7TU%(P8kPDv69??Aasc9T3`%)Z&$iE2)w^}9iay|ZC1=3TgE91! zq%6B1rUjb)eag8G;vH4kY1eN^AK1_n*VmB#MvvvmdZ>LZYgXczG4U zS~4;|p^5O>`{X3JYs@V{6H?E08^NnY?S&8OX0v_>L~%iHn~B<(JnBBYZ(*% z@s}7JVICS@$(A<9R?c&ONy55rhlc3sk2t=w%Gmrc z;=L+;ityBd7ZBi)I(0=d8QaFL5zNqI>{!f#11c@$j%Ftj#Kvs)A5}epO##>pJG~j# zj4W~{ffuC=&R_FsBn<5W7B*{9J_Tqm8jz(_FfwFjgE*3tJ9bLM#&_pqt%YgJe<&@| z$9aN}$+A@k2CK^OwU47A#C9tCm_6s_w(Y5ZNYm2rxMot+!`dAA;D52}BY`xSnd_UF zl`_z#)1*I@y@o*W^ZQcGN;F6~$8bp+k<1=+Y)B=|4$g2W({JiyZ5uXz%|2H-ApDb& zuT*5k7+_x}YGpi6=kTL9R;j2Bkj%p^g(-Tv458cl0OQYW=Kba;Khji967Oy1t2Osj zZy6nTU!#?iAm#kRqlh1&hjh^EWP2S%>>CZQQ{y5_b39vLEMF`GTYW zkY~N1zw*Am=E+e)(e|*`_q7ydMJK(mJRS|PRI8q-FD|a(C=fXu22;4_BSvk?gOJxw zhVO{#2=zVFPs6;y-N?9^@NczOnDlWqCyUff1)Zv5pVo@aE)X|)vlJ6P<6H%C-o{#^ zVwr6H7%A{+yzLn=GX9x?)kONeX=%Oes;McW{`YhApN#XTP7X)8MXM}~QXxCVtxx1D z0avj%%F6k`iAwaLxiR@-?VSWif>*@a+Kuj8$78Dmrri)_QJs7n?AqtuVf>7Zb$6`WU_a1RrNW@BqsA z?{wkb%(wFP`U}@dYM}gYcBhG2(!1rZEF8k23I4TEM9**DurG zozIJ`G3mB1{ozco5Aw9OT`LJMe+S=$_^S?ct6dRM$UXIsAQtULykud zAz0+YaI+Qx@82z(V`(J`pA7y>iH|&{Axkj5o}}@}Sas-Zh^0Y;#1uhb?ruyp`{d<} zQI4r(nT#L8$xVWV2O=$oL2I$_s&3X`yeA`lMV-O`O8A8kXC~o~A;BxBOapHGEG!*F z3^*?mRcG8?rR-wY3ddG`7Y~2`aL|f)w0i!guab!_k^-5nVTHy~(_tOmqq@2;Nvlzihk9Q&E@YU6ss}zT@cwLW z?Y)GhV$189xHLgCEBm&8{FNfgM!{$u)|ktUmV<^v0d0)ucY#czPU7^V$R)0EyZiYU z`OHEjYVH@v%F~S3c?)BY zYZV$z1>8UbO8?}vVIc)Fj7qDdatQiFQeLtcgAsv$A^@S*A#;0w3@Lz zdJ1t5!4W`G=N}U*koz#dVb0_AZ*ONlfK}6RaFWkxYyl0Xc>rhK=QYB~BIbPbz)*hM zV_q-sG#(F!%D$LMwjIDsL>F$%1f5kX{+sk|f=ySfh=c5Gl}d`xB2y%e_G+4CfQz3? z2L(Vx7UW>_v~^1v{vz)lIGr&T|OIwVA5QSJl?AmjiQZnuS_DI%3x= zF-}2GXNdKK46Bx&_ML1PTEAL8ZYXa&>qVmFs(6i_?EbM>lduCV7GH1fxAh$=`_a zdg}6h?HD2FRQBUd#1iWDhAFw0mHExvhU?)pBmq*r~Ej$*L&y z1#@)i3S#S~ zk(9j@Fu_EbnF6#E1Lj&s`Sd8cD-a3IwA5xfUjOSg!nx947E|(IthgOhUApN7M^FKT z+<1{+w7BN z#rXO846jvM+IlmckF=-9s~0pEz3b=@MgJZwqn4lIi#2bN8yL!;pPKUAh3mG!ix4RS z2{J7}<6^BnDP@(%#$!4}qLZ@YFTQh??AvEFnkM3U|KsqC=$xW z<-S+TE-SU6+N@nt#~ArXFqloLO!G-Q;8ozi4h_i)+@i$}b8>OjHY%FRKD4el4G(tg z$Y+L~C84!$xxFdruIqn|m0w`IS)-LtpKM!X8`Wd@?6-%2;*?7hBG;*p3UTS&F6=B3IJ1A`^zu(E z)C+K~$@fggqF@H{7a<)UPkLO@KQDD`J#}D#oX1M5bsnLYzw7PL&dyx!(s@N6?%x@DfyZF1W(hG(Etq@4x zu$RKpJ6b@}6R5-Css(ZBWzo7n__{y6K>8|Vcl5P`)kq>A-2~e%mfAa?stRYgY>%|h zUVNLJ&8D7wcr^qw=zyfaCLMaeF9{}^XhZaz8c1D7HpA7QGMU*vM!U>jM)37uaM0I> zBm*o}-3&SQ#@8zC>kb9>!F0gzPUS0RBDTX(ybtv`j7GZD2%Z*Z)UK2HAl4b;06ixR z%~Y1xS>-<_&FzJ2+(OtCRg6FD^GKrQFSxK{$4oxL*m}at8WnZg)ma7WS}^`*xU!-d zG!}Ga#Ll`|?N=&XeaRW*{#=MGeXrSbQw0E_Z%q7&P;K)OIdB!u2=yGZNgtEOjvZH- z`F!GG+*RTVaqPwz7&)d}&?MWl2Pst$VD3vO{@wRFZCE}W zKS6EH45lH2l?l@5o5Y3%RvxxU$v%rvo)um=U{J>goGJ3jQ{=zJ+J5bbUFd`K^W-`J zGsz+z{<*7kI-T-HvLeL^Cj09PJuUaowInwHn;#6M0!@fKg_^*tex58I!s>dBbvcI$ z-%A>94|B9po#?i_Z<3;Iyg0%J+yPla@0>h58myCLXgO)2kCp1m-E3b2fpjqH zs%co!KUpcRPX(TDQbHmIwP)R!V@1uQBUB@M-q_Oz1ZoJ>)L(PO5&>;KzJ4pL7FXp& zW*UEczr=}K9B%wU$zP`-yT$t_y)CB}{P~Z8ZfLlL7vh&+Cn)jlso62!A%?G~RQ0J3 zB{)=J%^CDDqWy_?ej9X>3p!SYcqF(UF(~2K)D3Lmz+)UPDX8{z8*3R`JE>Sn>KJwA zrn9JzwcA&bP*I4y`tUiI7RamdgGnjR+{epb8B){VmsHEkxD2GSF?v5uizf8^`$Shm z_wBg{7SMDmCP~MauAOq%U=PIz%C~47w&H3`qMjO=A}5x+)enITD*R~@icMOBD8tcI zJ{m<-h(`Z*-Szlv`#atij#+!zqNRlVK~)l0S-o|&oRoe)X#f3N*09`1Kyczv`_h9g zV(|BkNCE$IqpcHLRBci#t_ln*x@I@Zz)4Rvfq)UBeATh4cNtv}7RKrfD&nU<$BUcMMk;94!&QSEY6SC}zh`UY1XGn#_3S}K5xuEN)P^!~XPjRe^QWrP<8A&AY%{CzUh<~&KH#S1qcTR#Y`|_J8 zU!5bAYIE9%>BfCF@yS^D29r02$tKx43I(=?W|Y^J&<@4C(#A_{oQ6pykN_RpQ=oIE z)`IVxDf5~9h|e4C6qu07z~egN1>O|PA;kiOu0pXGfHns0!g7Jg!Gqc%>>m3YP;GkD zf#m${Zi7%R>Ru|gbIc98D;`y$Lo%DzF*(9yK*>6eo$k&qEpo6RN2#$M_KP^0w?j** zNF$+k+*07O1O{Nsd8jeL-*=pRojXfe!dugL6j3s@T4GA???Ttx<+;lFNFD8$e~$Vs z6xRQE42dg_26eM`7JnGm-beWs-B}jvEYJ?Wqv1jSa)by(n-NQ06@{P zG$|!hv{UUHPp20(mG&v$UVCQ+R9zK@v$i$ge*Y9i6!+~*PWt4rXPqa}JRQCdIg2GL z=cgTX-JJfL@Y+Rn7HEBZms-U3&9aCe_2-l1&E{wQ3rq@M3onua%_$a@<-kqs-LtnN zNy+xNWrJ4ZA3RN|Z_j;SVT4=WUa5X}zNURSf1`w=BFIfD7Kk)l$MPA#=Qsox{_khEvb=^|os323{{iPcHWdH> From 201fd8560c57675245eb307effd09484ec439892 Mon Sep 17 00:00:00 2001 From: Evgeny Stanilovskiy Date: Sat, 28 Apr 2018 15:09:17 +0300 Subject: [PATCH 0180/1463] IGNITE-8401: errorneous WalPointers comparation. - Fixes #3931. Signed-off-by: dspavlov --- .../cache/persistence/GridCacheDatabaseSharedManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 12cfc05d72e2e..787d7fec78ba3 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 @@ -747,7 +747,7 @@ private void unRegistrateMetricsMBean() { WALPointer restore = restoreMemory(status); - if (restore == null && status.endPtr != CheckpointStatus.NULL_PTR) { + if (restore == null && !status.endPtr.equals(CheckpointStatus.NULL_PTR)) { throw new StorageException("Restore wal pointer = " + restore + ", while status.endPtr = " + status.endPtr + ". Can't restore memory - critical part of WAL archive is missing."); } From 6948405529d8ead7c6d93434d4d7cfc15ae83c7e Mon Sep 17 00:00:00 2001 From: Ivan Daschinskiy Date: Thu, 10 May 2018 20:00:12 +0300 Subject: [PATCH 0181/1463] IGNITE-7896 FilePageStore truncate now actually remove redundant partition page file. Signed-off-by: Andrey Gura (cherry picked from commit d154eec) --- .../cache/persistence/file/FilePageStore.java | 23 +-- .../IgnitePdsPartitionFilesTruncateTest.java | 153 ++++++++++++++++++ .../testsuites/IgnitePdsTestSuite2.java | 3 + 3 files changed, 170 insertions(+), 9 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesTruncateTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java index fa02f5dcd3348..05f94218a7fc1 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java @@ -260,26 +260,31 @@ public void stop(boolean cleanFile) throws PersistentStorageIOException { public void truncate(int tag) throws PersistentStorageIOException { lock.writeLock().lock(); + long pages = this.pages(); + try { if (!inited) return; this.tag = tag; - fileIO.clear(); - - long newAlloc = initFile(); - - long delta = newAlloc - allocated.getAndSet(newAlloc); - - assert delta % pageSize == 0; - - allocatedTracker.updateTotalAllocatedPages(delta / pageSize); + try { + fileIO.close(); + } + finally { + cfgFile.delete(); + } } catch (IOException e) { throw new PersistentStorageIOException(e); } finally { + inited = false; + + allocated.set(0); + + allocatedTracker.updateTotalAllocatedPages(-1L * pages); + lock.writeLock().unlock(); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesTruncateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesTruncateTest.java new file mode 100644 index 0000000000000..78c2453df08a7 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesTruncateTest.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.persistence; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteDataStreamer; +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.util.typedef.internal.U; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Checks that evicted partitions doesn't leave files in PDS. + */ +public class IgnitePdsPartitionFilesTruncateTest extends GridCommonAbstractTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setConsistentId(gridName) + .setDataStorageConfiguration(new DataStorageConfiguration() + .setPageSize(1024) + .setWalMode(WALMode.LOG_ONLY) + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration() + .setPersistenceEnabled(true))) + .setCacheConfiguration(new CacheConfiguration(DEFAULT_CACHE_NAME) + .setBackups(1) + .setAffinity(new RendezvousAffinityFunction(false, 32))); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** + * @throws Exception If failed. + */ + public void testTruncatingPartitionFilesOnEviction() throws Exception { + Ignite ignite0 = startGrids(3); + + ignite0.cluster().active(true); + + try (IgniteDataStreamer streamer = ignite0.dataStreamer(DEFAULT_CACHE_NAME)) { + for (int i = 0; i < 1_000; i++) + streamer.addData(i, "Value " + i); + } + + assertEquals(1, ignite0.cacheNames().size()); + + awaitPartitionMapExchange(true, true, null); + + checkPartFiles(0); + checkPartFiles(1); + checkPartFiles(2); + + stopGrid(2); + + ignite0.cluster().setBaselineTopology(ignite0.cluster().topologyVersion()); + + awaitPartitionMapExchange(true, true, null); + + checkPartFiles(0); + checkPartFiles(1); + + startGrid(2); + + ignite0.cluster().setBaselineTopology(ignite0.cluster().topologyVersion()); + + awaitPartitionMapExchange(true, true, null); + + checkPartFiles(0); + checkPartFiles(1); + checkPartFiles(2); + } + + /** + * @param idx Node index. + */ + private void checkPartFiles(int idx) throws Exception { + Ignite ignite = grid(idx); + + int[] parts = ignite.affinity(DEFAULT_CACHE_NAME).allPartitions(ignite.cluster().localNode()); + + Path dirPath = Paths.get(U.defaultWorkDirectory(), "db", + U.maskForFileName(ignite.configuration().getIgniteInstanceName()), "cache-" + DEFAULT_CACHE_NAME); + + info("Path: " + dirPath.toString()); + + assertTrue(Files.exists(dirPath)); + + for (Path f : Files.newDirectoryStream(dirPath)) { + if (f.getFileName().toString().startsWith("part-")) + assertTrue("Node_" + idx +" should contains only partitions " + Arrays.toString(parts) + + ", but the file is redundant: " + f.getFileName(), anyMatch(parts, f)); + } + } + + /** */ + private boolean anyMatch(int[] parts, Path f) { + Pattern ptrn = Pattern.compile("part-(\\d+).bin"); + Matcher matcher = ptrn.matcher(f.getFileName().toString()); + + if (!matcher.find()) + throw new IllegalArgumentException("File is not a partition:" + f.getFileName()); + + int part = Integer.parseInt(matcher.group(1)); + + for (int p: parts) { + if (p == part) + return true; + } + + return false; + } +} 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 cbe9720f9a10e..0cd5e62a32938 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 @@ -25,6 +25,7 @@ import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsCorruptedStoreTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsExchangeDuringCheckpointTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsPageSizesTest; +import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsPartitionFilesTruncateTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsRecoveryAfterFileCorruptionTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePersistentStoreDataStructuresTest; import org.apache.ignite.internal.processors.cache.persistence.LocalWalModeChangeDuringRebalancingSelfTest; @@ -101,6 +102,8 @@ private static void addRealPageStoreTestsLongRunning(TestSuite suite) { public static void addRealPageStoreTests(TestSuite suite) { suite.addTestSuite(IgnitePdsPageSizesTest.class); + suite.addTestSuite(IgnitePdsPartitionFilesTruncateTest.class); + // Metrics test. suite.addTestSuite(IgniteDataStorageMetricsSelfTest.class); From 64d8cf4651dd8dde512f31ea0349973d3fead7ff Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Thu, 17 May 2018 12:19:36 +0300 Subject: [PATCH 0182/1463] IGNITE-8320 Partition file can be truncated only after checkpoint - Fixes #3985. Signed-off-by: Alexey Goncharuk (cherry picked from commit 8cb35e1) --- .../dht/GridDhtLocalPartition.java | 55 ++- .../GridCacheDatabaseSharedManager.java | 383 +++++++++++++-- .../persistence/GridCacheOffheapManager.java | 44 +- .../cache/persistence/file/FilePageStore.java | 43 +- .../serializer/RecordDataV2Serializer.java | 2 +- .../IgnitePdsCorruptedIndexTest.java | 341 ++++++++++++++ .../IgnitePdsPartitionFilesDestroyTest.java | 444 ++++++++++++++++++ .../IgnitePdsPartitionFilesTruncateTest.java | 153 ------ .../junits/GridAbstractTest.java | 9 +- .../junits/multijvm/IgniteProcessProxy.java | 23 +- .../ignite/testsuites/IgnitePdsTestSuite.java | 2 + .../testsuites/IgnitePdsTestSuite2.java | 8 +- .../query/h2/database/InlineIndexHelper.java | 1 - .../IgnitePdsWithIndexingCoreTestSuite.java | 3 + 14 files changed, 1274 insertions(+), 237 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedIndexTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java delete mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesTruncateTest.java 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 a199f6cc9ef30..c54a6cb238b52 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 @@ -30,6 +30,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.NodeStoppingException; @@ -88,6 +89,12 @@ public class GridDhtLocalPartition extends GridCacheConcurrentMapImpl implements /** Maximum size for delete queue. */ public static final int MAX_DELETE_QUEUE_SIZE = Integer.getInteger(IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE, 200_000); + /** ONLY FOR TEST PURPOSES: force test checkpoint on partition eviction. */ + private static boolean forceTestCheckpointOnEviction = IgniteSystemProperties.getBoolean("TEST_CHECKPOINT_ON_EVICTION", false); + + /** ONLY FOR TEST PURPOSES: partition id where test checkpoint was enforced during eviction. */ + static volatile Integer partWhereTestCheckpointEnforced; + /** Maximum size for {@link #rmvQueue}. */ private final int rmvQueueMaxSize; @@ -209,6 +216,10 @@ public class GridDhtLocalPartition extends GridCacheConcurrentMapImpl implements try { store = grp.offheap().createCacheDataStore(id); + // Log partition creation for further crash recovery purposes. + if (grp.walEnabled()) + ctx.wal().log(new PartitionMetaStateRecord(grp.groupId(), id, state(), updateCounter())); + // Inject row cache cleaner on store creation // Used in case the cache with enabled SqlOnheapCache is single cache at the cache group if (ctx.kernalContext().query().moduleEnabled()) { @@ -1037,6 +1048,16 @@ private long clearAll() throws NodeStoppingException { ctx.database().checkpointReadUnlock(); } } + + if (forceTestCheckpointOnEviction) { + if (partWhereTestCheckpointEnforced == null && cleared >= fullSize()) { + ctx.database().forceCheckpoint("test").finishFuture().get(); + + log.warning("Forced checkpoint by test reasons for partition: " + this); + + partWhereTestCheckpointEnforced = id; + } + } } catch (NodeStoppingException e) { if (log.isDebugEnabled()) @@ -1331,10 +1352,10 @@ long expireTime() { */ class ClearFuture extends GridFutureAdapter { /** Flag indicates that eviction callback was registered on the current future. */ - private volatile boolean evictionCallbackRegistered; + private volatile boolean evictionCbRegistered; /** Flag indicates that clearing callback was registered on the current future. */ - private volatile boolean clearingCallbackRegistered; + private volatile boolean clearingCbRegistered; /** Flag indicates that future with all callbacks was finished. */ private volatile boolean finished; @@ -1353,25 +1374,29 @@ class ClearFuture extends GridFutureAdapter { * @param updateSeq If {@code true} update topology sequence after successful eviction. */ private void registerEvictionCallback(boolean updateSeq) { - if (evictionCallbackRegistered) + if (evictionCbRegistered) return; synchronized (this) { // Double check - if (evictionCallbackRegistered) + if (evictionCbRegistered) return; - evictionCallbackRegistered = true; + evictionCbRegistered = true; // Initiates partition eviction and destroy. listen(f -> { - if (f.error() != null) { - rent.onDone(f.error()); - } else if (f.isDone()) { + try { + // Check for errors. + f.get(); + finishEviction(updateSeq); } + catch (Exception e) { + rent.onDone(e); + } - evictionCallbackRegistered = false; + evictionCbRegistered = false; }); } } @@ -1380,21 +1405,21 @@ private void registerEvictionCallback(boolean updateSeq) { * Registers clearing callback on the future. */ private void registerClearingCallback() { - if (clearingCallbackRegistered) + if (clearingCbRegistered) return; synchronized (this) { // Double check - if (clearingCallbackRegistered) + if (clearingCbRegistered) return; - clearingCallbackRegistered = true; + clearingCbRegistered = true; // Recreate cache data store in case of allowed fast eviction, and reset clear flag. listen(f -> { clear = false; - clearingCallbackRegistered = false; + clearingCbRegistered = false; }); } } @@ -1447,8 +1472,8 @@ public boolean initialize(boolean updateSeq, boolean evictionRequested) { reset(); finished = false; - evictionCallbackRegistered = false; - clearingCallbackRegistered = false; + evictionCbRegistered = false; + clearingCbRegistered = false; } if (evictionRequested) 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 12cfc05d72e2e..a542da3f9970a 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 @@ -21,7 +21,6 @@ import java.io.FileFilter; import java.io.IOException; import java.io.RandomAccessFile; -import java.io.Serializable; import java.lang.ref.SoftReference; import java.nio.ByteBuffer; import java.nio.ByteOrder; @@ -85,7 +84,6 @@ import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; import org.apache.ignite.internal.mem.DirectMemoryProvider; -import org.apache.ignite.internal.mem.DirectMemoryRegion; import org.apache.ignite.internal.mem.file.MappedFileMemoryProvider; import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider; import org.apache.ignite.internal.pagemem.FullPageId; @@ -296,6 +294,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan } }; + /** Timeout between partition file destroy and checkpoint to handle it. */ + private static final long PARTITION_DESTROY_CHECKPOINT_TIMEOUT = 30 * 1000; // 30 Seconds. + /** Checkpoint thread. Needs to be volatile because it is created in exchange worker. */ private volatile Checkpointer checkpointer; @@ -389,6 +390,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan /** Initially disabled cache groups. */ private Collection initiallyGlobalWalDisabledGrps = new HashSet<>(); + /** Initially local wal disabled groups. */ private Collection initiallyLocalWalDisabledGrps = new HashSet<>(); /** @@ -498,6 +500,8 @@ private DataRegionConfiguration createDataRegionConfiguration(DataStorageConfigu final GridKernalContext kernalCtx = cctx.kernalContext(); if (!kernalCtx.clientNode()) { + checkpointer = new Checkpointer(cctx.igniteInstanceName(), "db-checkpoint-thread", log); + IgnitePageStoreManager store = cctx.pageStore(); assert store instanceof FilePageStoreManager : "Invalid page store manager was created: " + store; @@ -632,6 +636,9 @@ else if (regCfg.getMaxSize() < 8 * GB) registrateMetricsMBean(); } + if (checkpointer == null) + checkpointer = new Checkpointer(cctx.igniteInstanceName(), "db-checkpoint-thread", log); + super.onActivate(ctx); } @@ -1441,8 +1448,6 @@ private void restoreState() throws IgniteCheckedException { snapshotMgr.restoreState(); - checkpointer = new Checkpointer(cctx.igniteInstanceName(), "db-checkpoint-thread", log); - new IgniteThread(cctx.igniteInstanceName(), "db-checkpoint-thread", checkpointer).start(); CheckpointProgressSnapshot chp = checkpointer.wakeupForCheckpoint(0, "node started"); @@ -1920,20 +1925,23 @@ private WALPointer readPointer(File cpMarkerFile, ByteBuffer buf) throws IgniteC * @throws IgniteCheckedException If failed. * @throws StorageException In case I/O error occurred during operations with storage. */ - private @Nullable WALPointer restoreMemory(CheckpointStatus status) throws IgniteCheckedException { + @Nullable private WALPointer restoreMemory(CheckpointStatus status) throws IgniteCheckedException { return restoreMemory(status, false, (PageMemoryEx)metaStorage.pageMemory()); } /** * @param status Checkpoint status. - * @param storeOnly If {@code True} restores Metastorage only. + * @param metastoreOnly If {@code True} restores Metastorage only. * @param storePageMem Metastore page memory. * @throws IgniteCheckedException If failed. * @throws StorageException In case I/O error occurred during operations with storage. */ - private @Nullable WALPointer restoreMemory(CheckpointStatus status, boolean storeOnly, - PageMemoryEx storePageMem) throws IgniteCheckedException { - assert !storeOnly || storePageMem != null; + @Nullable private WALPointer restoreMemory( + CheckpointStatus status, + boolean metastoreOnly, + PageMemoryEx storePageMem + ) throws IgniteCheckedException { + assert !metastoreOnly || storePageMem != null; if (log.isInfoEnabled()) log.info("Checking memory state [lastValidPos=" + status.endPtr + ", lastMarked=" @@ -1954,7 +1962,7 @@ private WALPointer readPointer(File cpMarkerFile, ByteBuffer buf) throws IgniteC int applied = 0; WALPointer lastRead = null; - Collection ignoreGrps = storeOnly ? Collections.emptySet() : + Collection ignoreGrps = metastoreOnly ? Collections.emptySet() : F.concat(false, initiallyGlobalWalDisabledGrps, initiallyLocalWalDisabledGrps); try (WALIterator it = cctx.wal().replay(status.endPtr)) { @@ -1990,7 +1998,7 @@ else if (!F.eq(cpRec.checkpointId(), status.cpEndId)) // several repetitive restarts and the same pages may have changed several times. int grpId = pageRec.fullPageId().groupId(); - if (storeOnly && grpId != METASTORAGE_CACHE_ID) + if (metastoreOnly && grpId != METASTORAGE_CACHE_ID) continue; if (!ignoreGrps.contains(grpId)) { @@ -2020,22 +2028,47 @@ else if (!F.eq(cpRec.checkpointId(), status.cpEndId)) break; + case PART_META_UPDATE_STATE: + PartitionMetaStateRecord metaStateRecord = (PartitionMetaStateRecord)rec; + + { + int grpId = metaStateRecord.groupId(); + + if (metastoreOnly && grpId != METASTORAGE_CACHE_ID) + continue; + + if (ignoreGrps.contains(grpId)) + continue; + + int partId = metaStateRecord.partitionId(); + + GridDhtPartitionState state = GridDhtPartitionState.fromOrdinal(metaStateRecord.state()); + + if (state == null || state == GridDhtPartitionState.EVICTED) + schedulePartitionDestroy(grpId, partId); + else + cancelOrWaitPartitionDestroy(grpId, partId); + } + + break; + case PARTITION_DESTROY: - PartitionDestroyRecord destroyRec = (PartitionDestroyRecord)rec; + PartitionDestroyRecord destroyRecord = (PartitionDestroyRecord)rec; - final int gId = destroyRec.groupId(); + { + int grpId = destroyRecord.groupId(); - if (storeOnly && gId != METASTORAGE_CACHE_ID) - continue; + if (metastoreOnly && grpId != METASTORAGE_CACHE_ID) + continue; + + if (ignoreGrps.contains(grpId)) + continue; - if (!ignoreGrps.contains(gId)) { - final int pId = destroyRec.partitionId(); + PageMemoryEx pageMem = grpId == METASTORAGE_CACHE_ID ? storePageMem : getPageMemoryForCacheGroup(grpId); - PageMemoryEx pageMem = gId == METASTORAGE_CACHE_ID ? storePageMem : getPageMemoryForCacheGroup(gId); + pageMem.invalidate(grpId, destroyRecord.partitionId()); - pageMem.clearAsync( - (grpId, pageId) -> grpId == gId && PageIdUtils.partId(pageId) == pId, - true).get(); + schedulePartitionDestroy(grpId, destroyRecord.partitionId()); } break; @@ -2046,7 +2079,7 @@ else if (!F.eq(cpRec.checkpointId(), status.cpEndId)) int grpId = r.groupId(); - if (storeOnly && grpId != METASTORAGE_CACHE_ID) + if (metastoreOnly && grpId != METASTORAGE_CACHE_ID) continue; if (!ignoreGrps.contains(grpId)) { @@ -2079,7 +2112,7 @@ else if (!F.eq(cpRec.checkpointId(), status.cpEndId)) } } - if (storeOnly) + if (metastoreOnly) return null; if (status.needRestoreMemory()) { @@ -2691,6 +2724,170 @@ private CheckpointEntry createCheckPointEntry( return new CheckpointEntry(cpTs, ptr, cpId, cacheGrpStates); } + /** + * Adds given partition to checkpointer destroy queue. + * + * @param grpId Group ID. + * @param partId Partition ID. + */ + public void schedulePartitionDestroy(int grpId, int partId) { + Checkpointer cp = checkpointer; + + if (cp != null) + cp.schedulePartitionDestroy(cctx.cache().cacheGroup(grpId), grpId, partId); + } + + /** + * Cancels or wait for partition destroy. + * + * @param grpId Group ID. + * @param partId Partition ID. + * @throws IgniteCheckedException If failed. + */ + public void cancelOrWaitPartitionDestroy(int grpId, int partId) throws IgniteCheckedException { + Checkpointer cp = checkpointer; + + if (cp != null) + cp.cancelOrWaitPartitionDestroy(grpId, partId); + } + + /** + * Partition destroy queue. + */ + private static class PartitionDestroyQueue { + /** */ + private final ConcurrentMap, PartitionDestroyRequest> pendingReqs = + new ConcurrentHashMap<>(); + + /** + * @param grpCtx Group context. + * @param partId Partition ID to destroy. + */ + private void addDestroyRequest(@Nullable CacheGroupContext grpCtx, int grpId, int partId) { + PartitionDestroyRequest req = new PartitionDestroyRequest(grpId, partId); + + PartitionDestroyRequest old = pendingReqs.putIfAbsent(new T2<>(grpId, partId), req); + + assert old == null || grpCtx == null : "Must wait for old destroy request to finish before adding a new one " + + "[grpId=" + grpId + + ", grpName=" + grpCtx.cacheOrGroupName() + + ", partId=" + partId + ']'; + } + + /** + * @param destroyId Destroy ID. + * @return Destroy request to complete if was not concurrently cancelled. + */ + private PartitionDestroyRequest beginDestroy(T2 destroyId) { + PartitionDestroyRequest rmvd = pendingReqs.remove(destroyId); + + return rmvd == null ? null : rmvd.beginDestroy() ? rmvd : null; + } + + /** + * @param grpId Group ID. + * @param partId Partition ID. + * @return Destroy request to wait for if destroy has begun. + */ + private PartitionDestroyRequest cancelDestroy(int grpId, int partId) { + PartitionDestroyRequest rmvd = pendingReqs.remove(new T2<>(grpId, partId)); + + return rmvd == null ? null : !rmvd.cancel() ? rmvd : null; + } + } + + /** + * Partition destroy request. + */ + private static class PartitionDestroyRequest { + /** */ + private final int grpId; + + /** */ + private final int partId; + + /** Destroy cancelled flag. */ + private boolean cancelled; + + /** Destroy future. Not null if partition destroy has begun. */ + private GridFutureAdapter destroyFut; + + /** + * @param grpId Group ID. + * @param partId Partition ID. + */ + private PartitionDestroyRequest(int grpId, int partId) { + this.grpId = grpId; + this.partId = partId; + } + + /** + * Cancels partition destroy request. + * + * @return {@code False} if this request needs to be waited for. + */ + private synchronized boolean cancel() { + if (destroyFut != null) { + assert !cancelled; + + return false; + } + + cancelled = true; + + return true; + } + + /** + * Initiates partition destroy. + * + * @return {@code True} if destroy request should be executed, {@code false} otherwise. + */ + private synchronized boolean beginDestroy() { + if (cancelled) { + assert destroyFut == null; + + return false; + } + + if (destroyFut != null) + return false; + + destroyFut = new GridFutureAdapter<>(); + + return true; + } + + /** + * + */ + private synchronized void onDone(Throwable err) { + assert destroyFut != null; + + destroyFut.onDone(err); + } + + /** + * + */ + private void waitCompleted() throws IgniteCheckedException { + GridFutureAdapter fut; + + synchronized (this) { + assert destroyFut != null; + + fut = destroyFut; + } + + fut.get(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "PartitionDestroyRequest [grpId=" + grpId + ", partId=" + partId + ']'; + } + } + /** * Checkpointer object is used for notification on checkpoint begin, predicate is {@link #scheduledCp}.nextCpTs - now * > 0 . Method {@link #wakeupForCheckpoint} uses notify, {@link #waitCheckpointEvent} uses wait @@ -2704,7 +2901,7 @@ public class Checkpointer extends GridWorker { private volatile CheckpointProgress scheduledCp; /** Current checkpoint. This field is updated only by checkpoint thread. */ - private volatile CheckpointProgress curCpProgress; + @Nullable private volatile CheckpointProgress curCpProgress; /** Shutdown now. */ private volatile boolean shutdownNow; @@ -2850,7 +3047,7 @@ private void doCheckpoint() { syncedPagesCntr = new AtomicInteger(); evictedPagesCntr = new AtomicInteger(); - boolean interrupted = true; + boolean success = false; try { if (chp.hasDelta()) { @@ -2937,11 +3134,22 @@ private void doCheckpoint() { snapshotMgr.afterCheckpointPageWritten(); + try { + destroyEvictedPartitions(); + } + catch (IgniteCheckedException e) { + chp.progress.cpFinishFut.onDone(e); + + cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e)); + + return; + } + // Must mark successful checkpoint only if there are no exceptions or interrupts. - interrupted = false; + success = true; } finally { - if (!interrupted) + if (success) markCheckpointEnd(chp); } @@ -2991,6 +3199,122 @@ private void doCheckpoint() { } } + /** + * Processes all evicted partitions scheduled for destroy. + * + * @throws IgniteCheckedException If failed. + */ + private void destroyEvictedPartitions() throws IgniteCheckedException { + PartitionDestroyQueue destroyQueue = curCpProgress.destroyQueue; + + if (destroyQueue.pendingReqs.isEmpty()) + return; + + List reqs = null; + + for (final PartitionDestroyRequest req : destroyQueue.pendingReqs.values()) { + if (!req.beginDestroy()) + continue; + + final int grpId = req.grpId; + final int partId = req.partId; + + CacheGroupContext grp = cctx.cache().cacheGroup(grpId); + + assert grp != null + : "Cache group is not initialized [grpId=" + grpId + "]"; + assert grp.offheap() instanceof GridCacheOffheapManager + : "Destroying partition files when persistence is off " + grp.offheap(); + + final GridCacheOffheapManager offheap = (GridCacheOffheapManager) grp.offheap(); + + Runnable destroyPartTask = () -> { + try { + offheap.destroyPartitionStore(grpId, partId); + + req.onDone(null); + + if (log.isDebugEnabled()) + log.debug("Partition file has destroyed [grpId=" + grpId + ", partId=" + partId + "]"); + } + catch (Exception e) { + req.onDone(new IgniteCheckedException( + "Partition file destroy has failed [grpId=" + grpId + ", partId=" + partId + "]", e)); + } + }; + + if (asyncRunner != null) { + try { + asyncRunner.execute(destroyPartTask); + } + catch (RejectedExecutionException ignore) { + // Run the task synchronously. + destroyPartTask.run(); + } + } + else + destroyPartTask.run(); + + if (reqs == null) + reqs = new ArrayList<>(); + + reqs.add(req); + } + + if (reqs != null) + for (PartitionDestroyRequest req : reqs) + req.waitCompleted(); + + destroyQueue.pendingReqs.clear(); + } + + /** + * @param grpCtx Group context. Can be {@code null} in case of crash recovery. + * @param grpId Group ID. + * @param partId Partition ID. + */ + private void schedulePartitionDestroy(@Nullable CacheGroupContext grpCtx, int grpId, int partId) { + synchronized (this) { + scheduledCp.destroyQueue.addDestroyRequest(grpCtx, grpId, partId); + } + + if (log.isDebugEnabled()) + log.debug("Partition file has been scheduled to destroy [grpId=" + grpId + ", partId=" + partId + "]"); + + if (grpCtx != null) + wakeupForCheckpoint(PARTITION_DESTROY_CHECKPOINT_TIMEOUT, "partition destroy"); + } + + /** + * @param grpId Group ID. + * @param partId Partition ID. + */ + private void cancelOrWaitPartitionDestroy(int grpId, int partId) throws IgniteCheckedException { + PartitionDestroyRequest req; + + synchronized (this) { + req = scheduledCp.destroyQueue.cancelDestroy(grpId, partId); + } + + if (req != null) + req.waitCompleted(); + + CheckpointProgress cur; + + synchronized (this) { + cur = curCpProgress; + + if (cur != null) + req = cur.destroyQueue.cancelDestroy(grpId, partId); + } + + if (req != null) + req.waitCompleted(); + + if (log.isDebugEnabled()) + log.debug("Partition file destroy has cancelled [grpId=" + grpId + ", partId=" + partId + "]"); + } + /** * */ @@ -3136,7 +3460,7 @@ private Checkpoint markCheckpointBegin(CheckpointMetricsTracker tracker) throws } } - if (hasPages) { + if (hasPages || !curr.destroyQueue.pendingReqs.isEmpty()) { assert cpPtr != null; tracker.onWalCpRecordFsyncStart(); @@ -3611,6 +3935,9 @@ private static class CheckpointProgress { /** */ private volatile SnapshotOperation snapshotOperation; + /** Partitions destroy queue. */ + private final PartitionDestroyQueue destroyQueue = new PartitionDestroyQueue(); + /** Wakeup reason. */ private String reason; 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 aec9481a1d0d5..b62d778f975fe 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 @@ -146,8 +146,10 @@ public IndexStorage getIndexStorage() { } /** {@inheritDoc} */ - @Override protected CacheDataStore createCacheDataStore0(final int p) - throws IgniteCheckedException { + @Override protected CacheDataStore createCacheDataStore0(int p) throws IgniteCheckedException { + if (ctx.database() instanceof GridCacheDatabaseSharedManager) + ((GridCacheDatabaseSharedManager) ctx.database()).cancelOrWaitPartitionDestroy(grp.groupId(), p); + boolean exists = ctx.pageStore() != null && ctx.pageStore().exists(grp.groupId(), p); return new GridCacheDataStore(p, exists); @@ -573,25 +575,41 @@ private static boolean addPartition( /** {@inheritDoc} */ @Override protected void destroyCacheDataStore0(CacheDataStore store) throws IgniteCheckedException { + assert ctx.database() instanceof GridCacheDatabaseSharedManager + : "Destroying cache data store when persistence is not enabled: " + ctx.database(); + + int partId = store.partId(); + ctx.database().checkpointReadLock(); try { - int p = store.partId(); - saveStoreMetadata(store, null, false, true); - - PageMemoryEx pageMemory = (PageMemoryEx)grp.dataRegion().pageMemory(); - - int tag = pageMemory.invalidate(grp.groupId(), p); - - if (grp.walEnabled()) - ctx.wal().log(new PartitionDestroyRecord(grp.groupId(), p)); - - ctx.pageStore().onPartitionDestroyed(grp.groupId(), p, tag); } finally { ctx.database().checkpointReadUnlock(); } + + ((GridCacheDatabaseSharedManager)ctx.database()).schedulePartitionDestroy(grp.groupId(), partId); + } + + /** + * Invalidates page memory for given partition. Destroys partition store. + * NOTE: This method can be invoked only within checkpoint lock or checkpointer thread. + * + * @param grpId Group ID. + * @param partId Partition ID. + * + * @throws IgniteCheckedException If destroy has failed. + */ + public void destroyPartitionStore(int grpId, int partId) throws IgniteCheckedException { + PageMemoryEx pageMemory = (PageMemoryEx)grp.dataRegion().pageMemory(); + + int tag = pageMemory.invalidate(grp.groupId(), partId); + + if (grp.walEnabled()) + ctx.wal().log(new PartitionDestroyRecord(grp.groupId(), partId)); + + ctx.pageStore().onPartitionDestroyed(grpId, partId, tag); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java index 05f94218a7fc1..ae4880d22715c 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.nio.file.Files; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -244,7 +245,7 @@ public void stop(boolean cleanFile) throws PersistentStorageIOException { fileIO.close(); if (cleanFile) - cfgFile.delete(); + Files.delete(cfgFile.toPath()); } catch (IOException e) { throw new PersistentStorageIOException(e); @@ -255,35 +256,34 @@ public void stop(boolean cleanFile) throws PersistentStorageIOException { } /** + * Truncates and deletes partition file. * + * @param tag New partition tag. + * @throws PersistentStorageIOException If failed */ public void truncate(int tag) throws PersistentStorageIOException { - lock.writeLock().lock(); + init(); - long pages = this.pages(); + lock.writeLock().lock(); try { - if (!inited) - return; - this.tag = tag; - try { - fileIO.close(); - } - finally { - cfgFile.delete(); - } + fileIO.clear(); + + fileIO.close(); + + Files.delete(cfgFile.toPath()); } catch (IOException e) { - throw new PersistentStorageIOException(e); + throw new PersistentStorageIOException("Failed to delete partition file: " + cfgFile.getPath(), e); } finally { - inited = false; - allocated.set(0); - allocatedTracker.updateTotalAllocatedPages(-1L * pages); + allocatedTracker.updateTotalAllocatedPages(-1L * this.pages()); + + inited = false; lock.writeLock().unlock(); } @@ -325,7 +325,7 @@ public void finishRecover() throws PersistentStorageIOException { recover = false; } catch (IOException e) { - throw new PersistentStorageIOException("Unable to finish recover", e); + throw new PersistentStorageIOException("Failed to finish recover", e); } finally { lock.writeLock().unlock(); @@ -421,9 +421,9 @@ public void finishRecover() throws PersistentStorageIOException { } /** - * @throws IgniteCheckedException If failed to initialize store file. + * @throws PersistentStorageIOException If failed to initialize store file. */ - private void init() throws IgniteCheckedException { + private void init() throws PersistentStorageIOException { if (!inited) { lock.writeLock().lock(); @@ -431,7 +431,7 @@ private void init() throws IgniteCheckedException { if (!inited) { FileIO fileIO = null; - IgniteCheckedException err = null; + PersistentStorageIOException err = null; try { this.fileIO = fileIO = ioFactory.create(cfgFile, CREATE, READ, WRITE); @@ -451,7 +451,8 @@ private void init() throws IgniteCheckedException { inited = true; } catch (IOException e) { - err = new PersistentStorageIOException("Could not initialize file: " + cfgFile.getName(), e); + err = new PersistentStorageIOException( + "Failed to initialize partition file: " + cfgFile.getName(), e); throw err; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java index efba611d312cc..b3a00beeb6aa8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java @@ -92,7 +92,7 @@ public RecordDataV2Serializer(RecordDataV1Serializer delegateSerializer) { return 8 + 1; case EXCHANGE: - return 4 /*type*/ + 8 /*timestamp*/ + 2 /*constId*/; + return 4 /*type*/ + 8 /*timestamp*/ + 2 /*constId*/; case TX_RECORD: return txRecordSerializer.size((TxRecord)rec); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedIndexTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedIndexTest.java new file mode 100644 index 0000000000000..a1065f6b4b618 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedIndexTest.java @@ -0,0 +1,341 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ignite.internal.processors.cache.persistence; + +import java.io.File; +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.file.OpenOption; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import javax.cache.Cache; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cache.query.QueryCursor; +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.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.processors.cache.distributed.dht.GridDhtLocalPartition; +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.util.typedef.internal.S; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.testframework.junits.multijvm.IgniteProcessProxy; +import org.junit.Assert; + +/** + * Test to reproduce corrupted indexes problem after partition file eviction and truncation. + */ +public class IgnitePdsCorruptedIndexTest extends GridCommonAbstractTest { + /** Cache name. */ + private static final String CACHE = "cache"; + + /** Flag indicates that {@link HaltOnTruncateFileIO} should be used. */ + private boolean haltFileIO; + + /** MultiJVM flag. */ + private boolean multiJvm = true; + + /** Additional remote JVM args. */ + private List additionalArgs = Collections.emptyList(); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setConsistentId(igniteInstanceName); + + DataStorageConfiguration dsCfg = new DataStorageConfiguration() + .setWalMode(WALMode.LOG_ONLY) + .setCheckpointFrequency(10 * 60 * 1000) + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration() + .setMaxSize(512 * 1024 * 1024) + .setPersistenceEnabled(true) + ); + + if (haltFileIO) + dsCfg.setFileIOFactory(new HaltOnTruncateFileIOFactory(new RandomAccessFileIOFactory())); + + cfg.setDataStorageConfiguration(dsCfg); + + CacheConfiguration ccfg = new CacheConfiguration<>(CACHE) + .setBackups(1) + .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC) + .setIndexedTypes(Integer.class, IndexedObject.class, Long.class, IndexedObject.class) + .setAffinity(new RendezvousAffinityFunction(false, 32)); + + cfg.setCacheConfiguration(ccfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected boolean isMultiJvm() { + return multiJvm; + } + + /** {@inheritDoc} */ + @Override protected List additionalRemoteJvmArgs() { + return additionalArgs; + } + + /** + * + */ + public void testCorruption() throws Exception { + final String corruptedNodeName = "corrupted"; + + IgniteEx ignite = startGrid(0); + + haltFileIO = true; + + additionalArgs = new ArrayList<>(); + additionalArgs.add("-D" + "TEST_CHECKPOINT_ON_EVICTION=true"); + additionalArgs.add("-D" + "IGNITE_QUIET=false"); + + IgniteEx corruptedNode = (IgniteEx) startGrid(corruptedNodeName); + + additionalArgs.clear(); + + haltFileIO = false; + + startGrid(2); + + ignite.cluster().active(true); + + awaitPartitionMapExchange(); + + final int entityCnt = 3_200; + + try (IgniteDataStreamer streamer = ignite.dataStreamer(CACHE)) { + streamer.allowOverwrite(true); + + for (int i = 0; i < entityCnt; i++) + streamer.addData(i, new IndexedObject(i)); + } + + startGrid(3); + + resetBaselineTopology(); + + // Corrupted node should be halted during partition destroy. + GridTestUtils.waitForCondition(() -> ignite.cluster().nodes().size() == 3, getTestTimeout()); + + // Clear remote JVM instance cache. + IgniteProcessProxy.kill(corruptedNode.name()); + + stopAllGrids(); + + // Disable multi-JVM mode and start coordinator and corrupted node in the same JVM. + multiJvm = false; + + startGrid(0); + + corruptedNode = (IgniteEx) startGrid(corruptedNodeName); + + corruptedNode.cluster().active(true); + + resetBaselineTopology(); + + // If index was corrupted, rebalance or one of the following queries should be failed. + awaitPartitionMapExchange(); + + for (int k = 0; k < entityCnt; k += entityCnt / 4) { + IgniteCache cache1 = corruptedNode.cache(CACHE); + + int l = k; + int r = k + entityCnt / 4 - 1; + + log.info("Check range [" + l + "-" + r + "]"); + + QueryCursor> qry = + cache1.query(new SqlQuery(IndexedObject.class, "lVal between ? and ?") + .setArgs(l * l, r * r)); + + Collection> queried = qry.getAll(); + + log.info("Qry result size = " + queried.size()); + } + } + + /** + * + */ + private static class IndexedObject { + /** Integer indexed value. */ + @QuerySqlField(index = true) + private int iVal; + + /** Long indexed value. */ + @QuerySqlField(index = true) + private long lVal; + + /** */ + private byte[] payload = new byte[1024]; + + /** + * @param iVal Integer value. + */ + private IndexedObject(int iVal) { + this.iVal = iVal; + this.lVal = (long) iVal * iVal; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (!(o instanceof IndexedObject)) + return false; + + IndexedObject that = (IndexedObject)o; + + return iVal == that.iVal; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return iVal; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(IndexedObject.class, this); + } + } + + /** + * File I/O which halts JVM after specified file truncation. + */ + private static class HaltOnTruncateFileIO extends FileIODecorator { + /** File. */ + private final File file; + + /** The overall number of file truncations have done. */ + private static final AtomicInteger truncations = new AtomicInteger(); + + /** + * @param delegate File I/O delegate + */ + public HaltOnTruncateFileIO(FileIO delegate, File file) { + super(delegate); + this.file = file; + } + + /** {@inheritDoc} */ + @Override public void clear() throws IOException { + super.clear(); + + System.err.println("Truncated file: " + file.getAbsolutePath()); + + truncations.incrementAndGet(); + + Integer checkpointedPart = null; + try { + Field field = GridDhtLocalPartition.class.getDeclaredField("partWhereTestCheckpointEnforced"); + + field.setAccessible(true); + + checkpointedPart = (Integer) field.get(null); + } + catch (Exception e) { + e.printStackTrace(); + } + + // Wait while more than one file have truncated and checkpoint on partition eviction has done. + if (truncations.get() > 1 && checkpointedPart != null) { + System.err.println("JVM is going to be crushed for test reasons..."); + + Runtime.getRuntime().halt(0); + } + } + } + + /** + * I/O Factory which creates {@link HaltOnTruncateFileIO} instances for partition files. + */ + private static class HaltOnTruncateFileIOFactory implements FileIOFactory { + /** */ + private static final long serialVersionUID = 0L; + + /** Delegate factory. */ + private final FileIOFactory delegateFactory; + + /** + * @param delegateFactory Delegate factory. + */ + HaltOnTruncateFileIOFactory(FileIOFactory delegateFactory) { + this.delegateFactory = delegateFactory; + } + + /** + * @param file File. + */ + private static boolean isPartitionFile(File file) { + return file.getName().contains("part") && file.getName().endsWith("bin"); + } + + /** {@inheritDoc} */ + @Override public FileIO create(File file) throws IOException { + FileIO delegate = delegateFactory.create(file); + + if (isPartitionFile(file)) + return new HaltOnTruncateFileIO(delegate, file); + + return delegate; + } + + /** {@inheritDoc} */ + @Override public FileIO create(File file, OpenOption... modes) throws IOException { + FileIO delegate = delegateFactory.create(file, modes); + + if (isPartitionFile(file)) + return new HaltOnTruncateFileIO(delegate, file); + + return delegate; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java new file mode 100644 index 0000000000000..b5afddf542889 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java @@ -0,0 +1,444 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 java.io.File; +import java.io.IOException; +import java.nio.file.OpenOption; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteDataStreamer; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +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.failure.FailureHandler; +import org.apache.ignite.failure.StopNodeFailureHandler; +import org.apache.ignite.internal.IgniteEx; +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.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.util.typedef.G; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.Assert; + +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; + +/** + * Test class to check that partition files after eviction are destroyed correctly on next checkpoint or crash recovery. + */ +public class IgnitePdsPartitionFilesDestroyTest extends GridCommonAbstractTest { + /** Cache name. */ + private static final String CACHE = "cache"; + + /** Partitions count. */ + private static final int PARTS_CNT = 32; + + /** Set if I/O exception should be thrown on partition file truncation. */ + private boolean failFileIo; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setConsistentId(igniteInstanceName); + + DataStorageConfiguration dsCfg = new DataStorageConfiguration() + .setWalMode(WALMode.LOG_ONLY) + .setCheckpointFrequency(10 * 60 * 1000) + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration() + .setMaxSize(512 * 1024 * 1024) + .setPersistenceEnabled(true) + ); + + if (failFileIo) + dsCfg.setFileIOFactory(new FailingFileIOFactory(new RandomAccessFileIOFactory())); + + cfg.setDataStorageConfiguration(dsCfg); + + CacheConfiguration ccfg = new CacheConfiguration<>(CACHE) + .setBackups(1) + .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC) + .setAffinity(new RendezvousAffinityFunction(false, PARTS_CNT)); + + cfg.setCacheConfiguration(ccfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + + failFileIo = false; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected FailureHandler getFailureHandler(String igniteInstanceName) { + return new StopNodeFailureHandler(); + } + + /** + * @param ignite Ignite. + * @param keysCnt Keys count. + */ + private void loadData(IgniteEx ignite, int keysCnt, int multiplier) { + log.info("Load data: keys=" + keysCnt); + + try (IgniteDataStreamer streamer = ignite.dataStreamer(CACHE)) { + streamer.allowOverwrite(true); + + for (int k = 0; k < keysCnt; k++) + streamer.addData(k, k * multiplier); + } + } + + /** + * @param ignite Ignite. + * @param keysCnt Keys count. + */ + private void checkData(IgniteEx ignite, int keysCnt, int multiplier) { + log.info("Check data: " + ignite.name() + ", keys=" + keysCnt); + + IgniteCache cache = ignite.cache(CACHE); + + for (int k = 0; k < keysCnt; k++) + Assert.assertEquals("node = " + ignite.name() + ", key = " + k, (Integer) (k * multiplier), cache.get(k)); + } + + /** + * Test that partition files have been deleted correctly on next checkpoint. + * + * @throws Exception If failed. + */ + public void testPartitionFileDestroyAfterCheckpoint() throws Exception { + IgniteEx crd = (IgniteEx) startGrids(2); + + crd.cluster().active(true); + + int keysCnt = 50_000; + + loadData(crd, keysCnt, 1); + + startGridsMultiThreaded(2, 2); + + // Trigger partitions eviction. + resetBaselineTopology(); + + awaitPartitionMapExchange(true, true, null); + + checkPartitionFiles(crd, true); + + // This checkpoint should delete partition files. + forceCheckpoint(); + + checkPartitionFiles(crd, false); + + for (Ignite ignite : G.allGrids()) + checkData((IgniteEx) ignite, keysCnt, 1); + } + + /** + * Test that partition files are reused correctly. + * + * @throws Exception If failed. + */ + public void testPartitionFileDestroyAndRecreate() throws Exception { + IgniteEx crd = startGrid(0); + + IgniteEx node = startGrid(1); + + crd.cluster().active(true); + + int keysCnt = 50_000; + + loadData(crd, keysCnt, 1); + + startGridsMultiThreaded(2, 2); + + // Trigger partitions eviction. + resetBaselineTopology(); + + awaitPartitionMapExchange(true, true, null); + + checkPartitionFiles(node, true); + + // Trigger partitions re-create. + stopGrid(2); + + resetBaselineTopology(); + + awaitPartitionMapExchange(true, true, null); + + checkPartitionFiles(node, true); + + // Rewrite data. + loadData(crd, keysCnt, 2); + + // Force checkpoint on all nodes. + forceCheckpoint(); + + // Check that all unecessary partition files have been deleted. + checkPartitionFiles(node, false); + + for (Ignite ignite : G.allGrids()) + checkData((IgniteEx) ignite, keysCnt, 2); + } + + /** + * Test that partitions files have been deleted correctly during crash recovery. + * + * @throws Exception If failed. + */ + public void testPartitionFileDestroyCrashRecovery1() throws Exception { + IgniteEx crd = startGrid(0); + + failFileIo = true; + + IgniteEx problemNode = startGrid(1); + + failFileIo = false; + + crd.cluster().active(true); + + int keysCnt = 50_000; + + loadData(crd, keysCnt, 1); + + startGridsMultiThreaded(2, 2); + + // Trigger partitions eviction. + resetBaselineTopology(); + + awaitPartitionMapExchange(true, true, null); + + checkPartitionFiles(problemNode, true); + + try { + forceCheckpoint(problemNode); + + Assert.assertTrue("Checkpoint must be failed", false); + } + catch (Exception expected) { + expected.printStackTrace(); + } + + // Problem node should be stopped after failed checkpoint. + waitForTopology(3); + + problemNode = startGrid(1); + + awaitPartitionMapExchange(); + + // After recovery all evicted partition files should be deleted from disk. + checkPartitionFiles(problemNode, false); + + for (Ignite ignite : G.allGrids()) + checkData((IgniteEx) ignite, keysCnt, 1); + } + + /** + * Test that partitions files are not deleted if they were re-created on next time + * and no checkpoint has done during this time. + * + * @throws Exception If failed. + */ + public void testPartitionFileDestroyCrashRecovery2() throws Exception { + IgniteEx crd = startGrid(0); + + failFileIo = true; + + IgniteEx problemNode = startGrid(1); + + failFileIo = false; + + crd.cluster().active(true); + + int keysCnt = 50_000; + + loadData(crd, keysCnt, 1); + + // Trigger partitions eviction. + startGridsMultiThreaded(2, 2); + + resetBaselineTopology(); + + awaitPartitionMapExchange(true, true, null); + + checkPartitionFiles(problemNode, true); + + // Trigger partitions re-create. + stopGrid(2); + + resetBaselineTopology(); + + awaitPartitionMapExchange(true, true, null); + + checkPartitionFiles(problemNode, true); + + try { + forceCheckpoint(problemNode); + + Assert.assertTrue("Checkpoint must be failed", false); + } + catch (Exception expected) { + expected.printStackTrace(); + } + + // Problem node should be stopped after failed checkpoint. + waitForTopology(2); + + problemNode = startGrid(1); + + awaitPartitionMapExchange(); + + // After recovery all evicted partition files should be deleted from disk. + checkPartitionFiles(problemNode, false); + + for (Ignite ignite : G.allGrids()) + checkData((IgniteEx) ignite, keysCnt, 1); + } + + /** + * If {@code exists} is {@code true}, checks that all partition files exist + * if partition has state EVICTED. + * + * If {@code exists} is {@code false}, checks that all partition files don't exist + * if partition is absent or has state EVICTED. + * + * @param ignite Node. + * @param exists If {@code true} method will check that partition file exists, + * in other case will check that file doesn't exist. + * @throws IgniteCheckedException If failed. + */ + private void checkPartitionFiles(IgniteEx ignite, boolean exists) throws IgniteCheckedException { + int evicted = 0; + + GridDhtPartitionTopology top = ignite.cachex(CACHE).context().topology(); + + for (int p = 0; p < PARTS_CNT; p++) { + GridDhtLocalPartition part = top.localPartition(p); + + File partFile = partitionFile(ignite, CACHE, p); + + if (exists) { + if (part != null && part.state() == GridDhtPartitionState.EVICTED) + Assert.assertTrue("Partition file has deleted ahead of time: " + partFile, partFile.exists()); + + evicted++; + } + else { + if (part == null || part.state() == GridDhtPartitionState.EVICTED) + Assert.assertTrue("Partition file has not deleted: " + partFile, !partFile.exists()); + } + } + + if (exists) + Assert.assertTrue("There should be at least 1 eviction", evicted > 0); + } + + /** + * @param ignite Ignite. + * @param cacheName Cache name. + * @param partId Partition id. + */ + private static File partitionFile(Ignite ignite, String cacheName, int partId) throws IgniteCheckedException { + File dbDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false); + + String nodeName = ignite.name().replaceAll("\\.", "_"); + + return new File(dbDir, String.format("%s/cache-%s/part-%d.bin", nodeName, cacheName, partId)); + } + + /** + * + */ + static class FailingFileIO extends FileIODecorator { + /** + * @param delegate File I/O delegate + */ + public FailingFileIO(FileIO delegate) { + super(delegate); + } + + /** {@inheritDoc} */ + @Override public void clear() throws IOException { + throw new IOException("Test"); + } + } + + /** + * + */ + static class FailingFileIOFactory implements FileIOFactory { + /** Delegate factory. */ + private final FileIOFactory delegateFactory; + + /** + * @param delegateFactory Delegate factory. + */ + FailingFileIOFactory(FileIOFactory delegateFactory) { + this.delegateFactory = delegateFactory; + } + + /** + * @param file File. + */ + private static boolean isPartitionFile(File file) { + return file.getName().contains("part") && file.getName().endsWith("bin"); + } + + /** {@inheritDoc} */ + @Override public FileIO create(File file) throws IOException { + FileIO delegate = delegateFactory.create(file); + + if (isPartitionFile(file)) + return new FailingFileIO(delegate); + + return delegate; + } + + /** {@inheritDoc} */ + @Override public FileIO create(File file, OpenOption... modes) throws IOException { + FileIO delegate = delegateFactory.create(file, modes); + + if (isPartitionFile(file)) + return new FailingFileIO(delegate); + + return delegate; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesTruncateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesTruncateTest.java deleted file mode 100644 index 78c2453df08a7..0000000000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesTruncateTest.java +++ /dev/null @@ -1,153 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.processors.cache.persistence; - -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.Arrays; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteDataStreamer; -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.util.typedef.internal.U; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; - -/** - * Checks that evicted partitions doesn't leave files in PDS. - */ -public class IgnitePdsPartitionFilesTruncateTest extends GridCommonAbstractTest { - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - cfg.setConsistentId(gridName) - .setDataStorageConfiguration(new DataStorageConfiguration() - .setPageSize(1024) - .setWalMode(WALMode.LOG_ONLY) - .setDefaultDataRegionConfiguration( - new DataRegionConfiguration() - .setPersistenceEnabled(true))) - .setCacheConfiguration(new CacheConfiguration(DEFAULT_CACHE_NAME) - .setBackups(1) - .setAffinity(new RendezvousAffinityFunction(false, 32))); - - return cfg; - } - - /** {@inheritDoc} */ - @Override protected void beforeTest() throws Exception { - stopAllGrids(); - - cleanPersistenceDir(); - } - - /** {@inheritDoc} */ - @Override protected void afterTest() throws Exception { - stopAllGrids(); - - cleanPersistenceDir(); - } - - /** - * @throws Exception If failed. - */ - public void testTruncatingPartitionFilesOnEviction() throws Exception { - Ignite ignite0 = startGrids(3); - - ignite0.cluster().active(true); - - try (IgniteDataStreamer streamer = ignite0.dataStreamer(DEFAULT_CACHE_NAME)) { - for (int i = 0; i < 1_000; i++) - streamer.addData(i, "Value " + i); - } - - assertEquals(1, ignite0.cacheNames().size()); - - awaitPartitionMapExchange(true, true, null); - - checkPartFiles(0); - checkPartFiles(1); - checkPartFiles(2); - - stopGrid(2); - - ignite0.cluster().setBaselineTopology(ignite0.cluster().topologyVersion()); - - awaitPartitionMapExchange(true, true, null); - - checkPartFiles(0); - checkPartFiles(1); - - startGrid(2); - - ignite0.cluster().setBaselineTopology(ignite0.cluster().topologyVersion()); - - awaitPartitionMapExchange(true, true, null); - - checkPartFiles(0); - checkPartFiles(1); - checkPartFiles(2); - } - - /** - * @param idx Node index. - */ - private void checkPartFiles(int idx) throws Exception { - Ignite ignite = grid(idx); - - int[] parts = ignite.affinity(DEFAULT_CACHE_NAME).allPartitions(ignite.cluster().localNode()); - - Path dirPath = Paths.get(U.defaultWorkDirectory(), "db", - U.maskForFileName(ignite.configuration().getIgniteInstanceName()), "cache-" + DEFAULT_CACHE_NAME); - - info("Path: " + dirPath.toString()); - - assertTrue(Files.exists(dirPath)); - - for (Path f : Files.newDirectoryStream(dirPath)) { - if (f.getFileName().toString().startsWith("part-")) - assertTrue("Node_" + idx +" should contains only partitions " + Arrays.toString(parts) - + ", but the file is redundant: " + f.getFileName(), anyMatch(parts, f)); - } - } - - /** */ - private boolean anyMatch(int[] parts, Path f) { - Pattern ptrn = Pattern.compile("part-(\\d+).bin"); - Matcher matcher = ptrn.matcher(f.getFileName().toString()); - - if (!matcher.find()) - throw new IllegalArgumentException("File is not a partition:" + f.getFileName()); - - int part = Integer.parseInt(matcher.group(1)); - - for (int p: parts) { - if (p == part) - return true; - } - - return false; - } -} 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 f5784ebcdccd2..00929262e69c0 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 @@ -1015,7 +1015,14 @@ protected Ignite startRemoteGrid(String igniteInstanceName, IgniteConfiguration } } - return new IgniteProcessProxy(cfg, log, locNode, resetDiscovery); + return new IgniteProcessProxy(cfg, log, locNode, resetDiscovery, additionalRemoteJvmArgs()); + } + + /** + * @return Additional JVM args for remote instances. + */ + protected List additionalRemoteJvmArgs() { + return Collections.emptyList(); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java index fb59ae2933320..1eb7ddb09deb8 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java @@ -19,6 +19,8 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; +import java.util.List; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -135,6 +137,18 @@ public IgniteProcessProxy(IgniteConfiguration cfg, IgniteLogger log, Ignite locJ this(cfg, log, locJvmGrid, true); } + /** + * @param cfg Configuration. + * @param log Logger. + * @param locJvmGrid Local JVM grid. + * @throws Exception On error. + */ + public IgniteProcessProxy(IgniteConfiguration cfg, IgniteLogger log, Ignite locJvmGrid, boolean discovery) + throws Exception { + this(cfg, log, locJvmGrid, discovery, Collections.emptyList()); + } + + /** * @param cfg Configuration. * @param log Logger. @@ -142,7 +156,13 @@ public IgniteProcessProxy(IgniteConfiguration cfg, IgniteLogger log, Ignite locJ * @param resetDiscovery Reset DiscoverySpi at the configuration. * @throws Exception On error. */ - public IgniteProcessProxy(IgniteConfiguration cfg, IgniteLogger log, Ignite locJvmGrid, boolean resetDiscovery) + public IgniteProcessProxy( + IgniteConfiguration cfg, + IgniteLogger log, + Ignite locJvmGrid, + boolean resetDiscovery, + List additionalArgs + ) throws Exception { this.cfg = cfg; this.locJvmGrid = locJvmGrid; @@ -151,6 +171,7 @@ public IgniteProcessProxy(IgniteConfiguration cfg, IgniteLogger log, Ignite locJ String params = params(cfg, resetDiscovery); Collection filteredJvmArgs = filteredJvmArgs(); + filteredJvmArgs.addAll(additionalArgs); final CountDownLatch rmtNodeStartedLatch = new CountDownLatch(1); 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 0e94874373174..3aec7efc4ea04 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 @@ -92,6 +92,8 @@ public static void addRealPageStoreTestsLongRunning(TestSuite suite) { /** * Fills {@code suite} with PDS test subset, which operates with real page store and does actual disk operations. * + * NOTE: These tests are also executed using I/O plugins. + * * @param suite suite to add tests into. */ public static void addRealPageStoreTests(TestSuite suite) { 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 0cd5e62a32938..e3880dded5c1c 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 @@ -25,7 +25,7 @@ import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsCorruptedStoreTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsExchangeDuringCheckpointTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsPageSizesTest; -import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsPartitionFilesTruncateTest; +import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsPartitionFilesDestroyTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsRecoveryAfterFileCorruptionTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePersistentStoreDataStructuresTest; import org.apache.ignite.internal.processors.cache.persistence.LocalWalModeChangeDuringRebalancingSelfTest; @@ -92,18 +92,20 @@ private static void addRealPageStoreTestsLongRunning(TestSuite suite) { // Integrity test. suite.addTestSuite(IgnitePdsRecoveryAfterFileCorruptionTest.class); + + suite.addTestSuite(IgnitePdsPartitionFilesDestroyTest.class); } /** * Fills {@code suite} with PDS test subset, which operates with real page store and does actual disk operations. * + * NOTE: These tests are also executed using I/O plugins. + * * @param suite suite to add tests into. */ public static void addRealPageStoreTests(TestSuite suite) { suite.addTestSuite(IgnitePdsPageSizesTest.class); - suite.addTestSuite(IgnitePdsPartitionFilesTruncateTest.class); - // Metrics test. suite.addTestSuite(IgniteDataStorageMetricsSelfTest.class); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java index 5c42d0d3097b4..34191273694f6 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java @@ -61,7 +61,6 @@ public class InlineIndexHelper { Value.SHORT, Value.INT, Value.LONG, - Value.LONG, Value.FLOAT, Value.DOUBLE, Value.DATE, 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 943d43ff7dff3..d33b20b6f6cf5 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 @@ -21,6 +21,7 @@ import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsAtomicCacheRebalancingTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsBinaryMetadataOnClusterRestartTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsBinarySortObjectFieldsTest; +import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsCorruptedIndexTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsMarshallerMappingRestoreOnNodeStartTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsTxCacheHistoricalRebalancingTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsTxCacheRebalancingTest; @@ -76,6 +77,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgnitePdsThreadInterruptionTest.class); suite.addTestSuite(IgnitePdsBinarySortObjectFieldsTest.class); + suite.addTestSuite(IgnitePdsCorruptedIndexTest.class); + return suite; } } From 71bff26bfdc22cb1ceea7c124888f65f804ab755 Mon Sep 17 00:00:00 2001 From: Sergey Chugunov Date: Thu, 17 May 2018 13:10:01 +0300 Subject: [PATCH 0183/1463] IGNITE-8474 Fixed WalStateNodeLeaveExchangeTask preventing exchange merge - Fixes #3990. Signed-off-by: Alexey Goncharuk (cherry-picked from commit #ebd669e4c53cfd66708ff18dd59071e4aace38ae) --- .../GridCachePartitionExchangeManager.java | 115 +++++++++++------- .../cache/WalStateNodeLeaveExchangeTask.java | 2 +- .../distributed/CacheExchangeMergeTest.java | 25 +++- .../ignite/testframework/GridTestUtils.java | 11 +- 4 files changed, 108 insertions(+), 45 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index 28d5d207f59c0..c3a0add55dee9 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 @@ -217,6 +217,9 @@ public class GridCachePartitionExchangeManager extends GridCacheSharedMana /** For tests only. */ private volatile AffinityTopologyVersion exchMergeTestWaitVer; + /** For tests only. */ + private volatile List mergedEvtsForTest; + /** Distributed latch manager. */ private ExchangeLatchManager latchMgr; @@ -1879,9 +1882,14 @@ private void dumpDiagnosticInfo(IgniteInternalFuture fut, * For testing only. * * @param exchMergeTestWaitVer Version to wait for. + * @param mergedEvtsForTest List to collect discovery events with merged exchanges. */ - public void mergeExchangesTestWaitVersion(AffinityTopologyVersion exchMergeTestWaitVer) { + public void mergeExchangesTestWaitVersion( + AffinityTopologyVersion exchMergeTestWaitVer, + @Nullable List mergedEvtsForTest + ) { this.exchMergeTestWaitVer = exchMergeTestWaitVer; + this.mergedEvtsForTest = mergedEvtsForTest; } /** @@ -1968,46 +1976,8 @@ public boolean mergeExchangesOnCoordinator(GridDhtPartitionsExchangeFuture curFu AffinityTopologyVersion exchMergeTestWaitVer = this.exchMergeTestWaitVer; - if (exchMergeTestWaitVer != null) { - if (log.isInfoEnabled()) { - log.info("Exchange merge test, waiting for version [exch=" + curFut.initialVersion() + - ", waitVer=" + exchMergeTestWaitVer + ']'); - } - - long end = U.currentTimeMillis() + 10_000; - - while (U.currentTimeMillis() < end) { - boolean found = false; - - for (CachePartitionExchangeWorkerTask task : exchWorker.futQ) { - if (task instanceof GridDhtPartitionsExchangeFuture) { - GridDhtPartitionsExchangeFuture fut = (GridDhtPartitionsExchangeFuture)task; - - if (exchMergeTestWaitVer.equals(fut.initialVersion())) { - if (log.isInfoEnabled()) - log.info("Exchange merge test, found awaited version: " + exchMergeTestWaitVer); - - found = true; - - break; - } - } - } - - if (found) - break; - else { - try { - U.sleep(100); - } - catch (IgniteInterruptedCheckedException e) { - break; - } - } - } - - this.exchMergeTestWaitVer = null; - } + if (exchMergeTestWaitVer != null) + waitForTestVersion(exchMergeTestWaitVer, curFut); synchronized (curFut.mutex()) { int awaited = 0; @@ -2048,6 +2018,8 @@ public boolean mergeExchangesOnCoordinator(GridDhtPartitionsExchangeFuture curFu ", evtNodeClient=" + CU.clientNode(fut.firstEvent().eventNode())+ ']'); } + addDiscoEvtForTest(fut.firstEvent()); + curFut.context().events().addEvent(fut.initialVersion(), fut.firstEvent(), fut.firstEventCache()); @@ -2071,6 +2043,67 @@ public boolean mergeExchangesOnCoordinator(GridDhtPartitionsExchangeFuture curFu } } + + /** + * For testing purposes. Stores discovery events with merged exchanges to enable examining them later. + * + * @param discoEvt Discovery event. + */ + private void addDiscoEvtForTest(DiscoveryEvent discoEvt) { + List mergedEvtsForTest = this.mergedEvtsForTest; + + if (mergedEvtsForTest != null) + mergedEvtsForTest.add(discoEvt); + } + + /** + * For testing purposes. Method allows to wait for an exchange future of specific version + * to appear in exchange worker queue. + * + * @param exchMergeTestWaitVer Topology Version to wait for. + * @param curFut Current Exchange Future. + */ + private void waitForTestVersion(AffinityTopologyVersion exchMergeTestWaitVer, GridDhtPartitionsExchangeFuture curFut) { + if (log.isInfoEnabled()) { + log.info("Exchange merge test, waiting for version [exch=" + curFut.initialVersion() + + ", waitVer=" + exchMergeTestWaitVer + ']'); + } + + long end = U.currentTimeMillis() + 10_000; + + while (U.currentTimeMillis() < end) { + boolean found = false; + + for (CachePartitionExchangeWorkerTask task : exchWorker.futQ) { + if (task instanceof GridDhtPartitionsExchangeFuture) { + GridDhtPartitionsExchangeFuture fut = (GridDhtPartitionsExchangeFuture)task; + + if (exchMergeTestWaitVer.equals(fut.initialVersion())) { + if (log.isInfoEnabled()) + log.info("Exchange merge test, found awaited version: " + exchMergeTestWaitVer); + + found = true; + + break; + } + } + } + + if (found) + break; + else { + try { + U.sleep(100); + } + catch (IgniteInterruptedCheckedException e) { + break; + } + } + } + + this.exchMergeTestWaitVer = null; + } + /** * Exchange future thread. All exchanges happen only by one thread and next * exchange will not start until previous one completes. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateNodeLeaveExchangeTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateNodeLeaveExchangeTask.java index 3ac12fc3b2ee8..77dfc34e32e7c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateNodeLeaveExchangeTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateNodeLeaveExchangeTask.java @@ -47,7 +47,7 @@ public ClusterNode node() { /** {@inheritDoc} */ @Override public boolean skipForExchangeMerge() { - return false; + return true; } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheExchangeMergeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheExchangeMergeTest.java index 6c714b1f69647..53a75d4458150 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheExchangeMergeTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheExchangeMergeTest.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; +import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; @@ -775,17 +776,37 @@ public void testMergeServersFail1_2() throws Exception { * @throws Exception If failed. */ private void mergeServersFail1(boolean waitRebalance) throws Exception { - final Ignite srv0 = startGrids(4); + final Ignite srv0 = startGrids(5); if (waitRebalance) awaitPartitionMapExchange(); - mergeExchangeWaitVersion(srv0, 6); + final List mergedEvts = new ArrayList<>(); + + mergeExchangeWaitVersion(srv0, 8, mergedEvts); + + UUID grid3Id = grid(3).localNode().id(); + UUID grid2Id = grid(2).localNode().id(); + stopGrid(getTestIgniteInstanceName(4), true, false); stopGrid(getTestIgniteInstanceName(3), true, false); stopGrid(getTestIgniteInstanceName(2), true, false); checkCaches(); + + awaitPartitionMapExchange(); + + assertTrue("Unexpected number of merged disco events: " + mergedEvts.size(), mergedEvts.size() == 2); + + for (DiscoveryEvent discoEvt : mergedEvts) { + ClusterNode evtNode = discoEvt.eventNode(); + + assertTrue("eventNode is null for DiscoEvent " + discoEvt, evtNode != null); + + assertTrue("Unexpected eventNode ID: " + + evtNode.id() + " while expecting " + grid2Id + " or " + grid3Id, + evtNode.id().equals(grid2Id) || evtNode.id().equals(grid3Id)); + } } /** 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 e6c6657578c4d..9390d6b36fe69 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 @@ -1949,6 +1949,15 @@ public static String randomString(Random rnd, int maxLen) { */ public static void mergeExchangeWaitVersion(Ignite node, long topVer) { ((IgniteEx)node).context().cache().context().exchange().mergeExchangesTestWaitVersion( - new AffinityTopologyVersion(topVer, 0)); + new AffinityTopologyVersion(topVer, 0), null); + } + + /** + * @param node Node. + * @param topVer Ready exchange version to wait for before trying to merge exchanges. + */ + public static void mergeExchangeWaitVersion(Ignite node, long topVer, List mergedEvts) { + ((IgniteEx)node).context().cache().context().exchange().mergeExchangesTestWaitVersion( + new AffinityTopologyVersion(topVer, 0), mergedEvts); } } From bc0cb996f25018dc77e054f83bcaaec25f0de015 Mon Sep 17 00:00:00 2001 From: Aleksei Scherbakov Date: Thu, 17 May 2018 20:11:41 +0300 Subject: [PATCH 0184/1463] IGNITE-8423 Control utility may block when joining node is watiting for partition map exchange. - Fixes #4018. Signed-off-by: Alexey Goncharuk --- bin/control.bat | 2 +- bin/control.sh | 2 +- .../internal/commandline/CommandHandler.java | 59 ++++--- .../ignite/util/GridCommandHandlerTest.java | 150 +++++++++++------- 4 files changed, 127 insertions(+), 86 deletions(-) diff --git a/bin/control.bat b/bin/control.bat index 8a1e1c84e3c70..15d5e6fcb619b 100644 --- a/bin/control.bat +++ b/bin/control.bat @@ -104,7 +104,7 @@ if "%OS%" == "Windows_NT" set PROG_NAME=%~nx0% :: call "%SCRIPTS_HOME%\include\setenv.bat" call "%SCRIPTS_HOME%\include\build-classpath.bat" -set CP=%IGNITE_LIBS% +set CP=%IGNITE_LIBS%;%IGNITE_HOME%\libs\optional\ignite-zookeeper\* :: :: Process 'restart'. diff --git a/bin/control.sh b/bin/control.sh index ad4b14b05151b..e2b75afbd1a07 100755 --- a/bin/control.sh +++ b/bin/control.sh @@ -54,7 +54,7 @@ fi # . "${SCRIPTS_HOME}"/include/setenv.sh . "${SCRIPTS_HOME}"/include/build-classpath.sh # Will be removed in the binary release. -CP="${IGNITE_LIBS}" +CP="${IGNITE_LIBS}:${IGNITE_HOME}/libs/optional/ignite-zookeeper/*" RANDOM_NUMBER=$("$JAVA" -cp "${CP}" org.apache.ignite.startup.cmdline.CommandLineRandomNumberGenerator) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java index 04578e53f916d..47cc233f61ebc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java @@ -256,6 +256,9 @@ public class CommandHandler { /** */ private Object lastOperationRes; + /** */ + private GridClientConfiguration clientCfg; + /** Check if experimental commands are enabled. Default {@code false}. */ private final boolean enableExperimental = IgniteSystemProperties.getBoolean(IGNITE_ENABLE_EXPERIMENTAL_COMMAND, false); @@ -446,18 +449,6 @@ private void state(GridClient client) throws Throwable { } } - /** - * @param client Client. - * @param arg Task argument. - * @return Task result. - * @throws GridClientException If failed to execute task. - */ - private Map executeTransactionsTask(GridClient client, - VisorTxTaskArg arg) throws GridClientException { - - return executeTask(client, VisorTxTask.class, arg); - } - /** * * @param client Client. @@ -497,8 +488,28 @@ private R executeTaskByNameOnNode(GridClient client, String taskClsName, Obj GridClientNode node = null; - if (nodeId == null) - node = getBalancedNode(compute); + if (nodeId == null) { + Collection nodes = compute.nodes(GridClientNode::connectable); + + // Prefer node from connect string. + String origAddr = clientCfg.getServers().iterator().next(); + + for (GridClientNode clientNode : nodes) { + Iterator it = F.concat(clientNode.tcpAddresses().iterator(), clientNode.tcpHostNames().iterator()); + + while (it.hasNext()) { + if (origAddr.equals(it.next() + ":" + clientNode.tcpPort())) { + node = clientNode; + + break; + } + } + } + + // Otherwise choose random node. + if (node == null) + node = getBalancedNode(compute); + } else { for (GridClientNode n : compute.nodes()) { if (n.connectable() && nodeId.equals(n.nodeId())) { @@ -1677,12 +1688,12 @@ public int execute(List rawArgs) { " delete [consistentId1,consistentId2,....,consistentIdN] [--force]"); } - log("The utility has --cache subcommand to view and control state of caches in cluster."); - log(" More info: control.sh --cache help"); + log(" View caches information in a cluster. For more details type:"); + log(" control.sh --cache help"); nl(); - log("By default commands affecting the cluster require interactive confirmation. "); - log(" --force option can be used to execute commands without prompting for confirmation."); + log("By default commands affecting the cluster require interactive confirmation."); + log("Use --force option to disable it."); nl(); log("Default values:"); @@ -1710,20 +1721,20 @@ public int execute(List rawArgs) { return EXIT_CODE_OK; } - GridClientConfiguration cfg = new GridClientConfiguration(); + clientCfg = new GridClientConfiguration(); - cfg.setPingInterval(args.pingInterval()); + clientCfg.setPingInterval(args.pingInterval()); - cfg.setPingTimeout(args.pingTimeout()); + clientCfg.setPingTimeout(args.pingTimeout()); - cfg.setServers(Collections.singletonList(args.host() + ":" + args.port())); + clientCfg.setServers(Collections.singletonList(args.host() + ":" + args.port())); if (!F.isEmpty(args.user())) { - cfg.setSecurityCredentialsProvider( + clientCfg.setSecurityCredentialsProvider( new SecurityCredentialsBasicProvider(new SecurityCredentials(args.user(), args.password()))); } - try (GridClient client = GridClientFactory.start(cfg)) { + try (GridClient client = GridClientFactory.start(clientCfg)) { switch (args.command()) { case ACTIVATE: activate(client); diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java index 4daa92cdf000c..670c22c313b82 100644 --- a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java @@ -381,69 +381,10 @@ public void testActiveTransactions() throws Exception { for (Ignite ig : G.allGrids()) assertNotNull(ig.cache(DEFAULT_CACHE_NAME)); - AtomicInteger idx = new AtomicInteger(); - CountDownLatch lockLatch = new CountDownLatch(1); CountDownLatch unlockLatch = new CountDownLatch(1); - IgniteInternalFuture fut = multithreadedAsync(new Runnable() { - @Override public void run() { - int id = idx.getAndIncrement(); - - switch (id) { - case 0: - try (Transaction tx = grid(0).transactions().txStart()) { - grid(0).cache(DEFAULT_CACHE_NAME).putAll(generate(0, 100)); - - lockLatch.countDown(); - - U.awaitQuiet(unlockLatch); - - tx.commit(); - - fail("Commit must fail"); - } - catch (Exception e) { - // No-op. - assertTrue(X.hasCause(e, TransactionRollbackException.class)); - } - - break; - case 1: - U.awaitQuiet(lockLatch); - - doSleep(3000); - - try (Transaction tx = grid(0).transactions().withLabel("label1").txStart(PESSIMISTIC, READ_COMMITTED, Integer.MAX_VALUE, 0)) { - grid(0).cache(DEFAULT_CACHE_NAME).putAll(generate(200, 110)); - - grid(0).cache(DEFAULT_CACHE_NAME).put(0, 0); - } - - break; - case 2: - try (Transaction tx = grid(1).transactions().txStart()) { - U.awaitQuiet(lockLatch); - - grid(1).cache(DEFAULT_CACHE_NAME).put(0, 0); - } - - break; - case 3: - try (Transaction tx = client.transactions().withLabel("label2").txStart(OPTIMISTIC, READ_COMMITTED, 0, 0)) { - U.awaitQuiet(lockLatch); - - client.cache(DEFAULT_CACHE_NAME).putAll(generate(100, 10)); - - client.cache(DEFAULT_CACHE_NAME).put(0, 0); - - tx.commit(); - } - - break; - } - } - }, 4, "tx-thread"); + IgniteInternalFuture fut = startTransactions(lockLatch, unlockLatch); U.awaitQuiet(lockLatch); @@ -518,6 +459,23 @@ else if (entry.getKey().equals(node2)) { }, "--tx", "order", "DURATION"); + // Trigger topology change and test connection. + IgniteInternalFuture startFut = multithreadedAsync(new Runnable() { + @Override public void run() { + try { + startGrid(2); + } + catch (Exception e) { + fail(); + } + } + }, 1, "start-node-thread"); + + doSleep(5000); + + assertEquals(EXIT_CODE_OK, execute(h, "--host", "127.0.0.1", "--port", "11211", "--tx")); + assertEquals(EXIT_CODE_OK, execute(h, "--host", "127.0.0.1", "--port", "11212", "--tx")); + // Test kill by xid. validate(h, map -> { assertEquals(1, map.size()); @@ -533,6 +491,8 @@ else if (entry.getKey().equals(node2)) { unlockLatch.countDown(); + startFut.get(); + fut.get(); } @@ -859,4 +819,74 @@ public void testUnusedWalDelete() throws Exception { assertTrue(!testOut.toString().contains("error")); } + + /** + * + * @param lockLatch Lock latch. + * @param unlockLatch Unlock latch. + */ + private IgniteInternalFuture startTransactions(CountDownLatch lockLatch, CountDownLatch unlockLatch) throws Exception { + IgniteEx client = grid("client"); + + AtomicInteger idx = new AtomicInteger(); + + return multithreadedAsync(new Runnable() { + @Override public void run() { + int id = idx.getAndIncrement(); + + switch (id) { + case 0: + try (Transaction tx = grid(0).transactions().txStart()) { + grid(0).cache(DEFAULT_CACHE_NAME).putAll(generate(0, 100)); + + lockLatch.countDown(); + + U.awaitQuiet(unlockLatch); + + tx.commit(); + + fail("Commit must fail"); + } + catch (Exception e) { + // No-op. + assertTrue(X.hasCause(e, TransactionRollbackException.class)); + } + + break; + case 1: + U.awaitQuiet(lockLatch); + + doSleep(3000); + + try (Transaction tx = grid(0).transactions().withLabel("label1").txStart(PESSIMISTIC, READ_COMMITTED, Integer.MAX_VALUE, 0)) { + grid(0).cache(DEFAULT_CACHE_NAME).putAll(generate(200, 110)); + + grid(0).cache(DEFAULT_CACHE_NAME).put(0, 0); + } + + break; + case 2: + try (Transaction tx = grid(1).transactions().txStart()) { + U.awaitQuiet(lockLatch); + + grid(1).cache(DEFAULT_CACHE_NAME).put(0, 0); + } + + break; + case 3: + try (Transaction tx = client.transactions().withLabel("label2").txStart(OPTIMISTIC, READ_COMMITTED, 0, 0)) { + U.awaitQuiet(lockLatch); + + client.cache(DEFAULT_CACHE_NAME).putAll(generate(100, 10)); + + client.cache(DEFAULT_CACHE_NAME).put(0, 0); + + tx.commit(); + } + + break; + } + } + }, 4, "tx-thread"); + } } From 68e37332e0663c327e8faa2183c351d61f33cde2 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Thu, 17 May 2018 12:18:45 +0300 Subject: [PATCH 0185/1463] IGNITE-8498: JDBC Thin: Fixed streamer compatibility check. This closes #4002. --- .../jdbc/thin/JdbcThinConnection.java | 2 +- .../internal/jdbc/thin/JdbcThinTcpIo.java | 31 ++++++++++++++----- 2 files changed, 25 insertions(+), 8 deletions(-) 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 634579b7074e5..633d65753f98e 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 @@ -189,7 +189,7 @@ void executeNative(String sql, SqlCommand cmd) throws SQLException { // Actual ON, if needed. if (newVal) { - if (!cmd0.isOrdered() && !cliIo.igniteVersion().greaterThanEqual(2, 5, 0)) { + if (!cmd0.isOrdered() && !cliIo.isUnorderedStreamSupported()) { throw new SQLException("Streaming without order doesn't supported by server [remoteNodeVer=" + cliIo.igniteVersion() + ']', SqlStateCode.INTERNAL_ERROR); } 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 44c19847b4ddb..2e060954230a5 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 @@ -121,6 +121,9 @@ public class JdbcThinTcpIo { /** Mutex. */ private final Object mux = new Object(); + /** Current protocol version used to connection to Ignite. */ + private ClientListenerProtocolVersion srvProtocolVer; + /** * Constructor. * @@ -342,6 +345,8 @@ public void handshake(ClientListenerProtocolVersion ver) throws IOException, SQL } else igniteVer = new IgniteProductVersion((byte)2, (byte)0, (byte)0, "Unknown", 0L, null); + + srvProtocolVer = ver; } else { short maj = reader.readShort(); @@ -350,12 +355,12 @@ public void handshake(ClientListenerProtocolVersion ver) throws IOException, SQL String err = reader.readString(); - ClientListenerProtocolVersion srvProtocolVer = ClientListenerProtocolVersion.create(maj, min, maintenance); + ClientListenerProtocolVersion srvProtoVer0 = ClientListenerProtocolVersion.create(maj, min, maintenance); - if (srvProtocolVer.compareTo(VER_2_5_0) < 0 && !F.isEmpty(connProps.getUsername())) { - throw new SQLException("Authentication doesn't support by remote server[driverProtocolVer=" + CURRENT_VER + - ", remoteNodeProtocolVer=" + srvProtocolVer + ", err=" + err + ", url=" + connProps.getUrl() + ']', - SqlStateCode.CONNECTION_REJECTED); + if (srvProtoVer0.compareTo(VER_2_5_0) < 0 && !F.isEmpty(connProps.getUsername())) { + throw new SQLException("Authentication doesn't support by remote server[driverProtocolVer=" + + CURRENT_VER + ", remoteNodeProtocolVer=" + srvProtoVer0 + ", err=" + err + + ", url=" + connProps.getUrl() + ']', SqlStateCode.CONNECTION_REJECTED); } if (VER_2_4_0.equals(srvProtocolVer) || VER_2_3_0.equals(srvProtocolVer) || @@ -402,8 +407,11 @@ private void handshake_2_1_0() throws IOException, SQLException { boolean accepted = reader.readBoolean(); - if (accepted) + if (accepted) { igniteVer = new IgniteProductVersion((byte)2, (byte)1, (byte)0, "Unknown", 0L, null); + + srvProtocolVer = VER_2_1_0; + } else { short maj = reader.readShort(); short min = reader.readShort(); @@ -435,7 +443,7 @@ void sendBatchRequestNoWaitResponse(JdbcOrderedBatchExecuteRequest req) throws I } try { - if (!igniteVer.greaterThanEqual(2, 5, 0)) { + if (!isUnorderedStreamSupported()) { throw new SQLException("Streaming without response doesn't supported by server [driverProtocolVer=" + CURRENT_VER + ", remoteNodeVer=" + igniteVer + ']', SqlStateCode.INTERNAL_ERROR); } @@ -619,4 +627,13 @@ public ConnectionProperties connectionProperties() { IgniteProductVersion igniteVersion() { return igniteVer; } + + /** + * @return {@code true} If the unordered streaming supported. + */ + boolean isUnorderedStreamSupported() { + assert srvProtocolVer != null; + + return srvProtocolVer.compareTo(VER_2_5_0) >= 0; + } } \ No newline at end of file From 16a014399932159da99d33fda5d5cb00ec7aa957 Mon Sep 17 00:00:00 2001 From: Anton Kalashnikov Date: Fri, 18 May 2018 11:18:39 +0300 Subject: [PATCH 0186/1463] IGNITE-8464 Fixed race in WALIterator leading to skipped segments - Fixes #3982. Signed-off-by: Alexey Goncharuk (cherry picked from commit fdad641) --- .../wal/AbstractWalRecordsIterator.java | 11 ++- .../wal/FileWriteAheadLogManager.java | 86 ++++--------------- .../FsyncModeFileWriteAheadLogManager.java | 81 ++++------------- .../wal/serializer/RecordV1Serializer.java | 61 +++++++++++++ .../wal/serializer/SegmentHeader.java | 58 +++++++++++++ .../db/wal/reader/IgniteWalReaderTest.java | 46 +++++----- 6 files changed, 185 insertions(+), 158 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/SegmentHeader.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 65f3a20830109..d9312f6888b2f 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 @@ -33,12 +33,15 @@ import org.apache.ignite.internal.processors.cache.persistence.file.UnzipFileIO; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializer; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactory; +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.lang.IgniteBiTuple; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +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 @@ -265,11 +268,9 @@ protected AbstractReadFileHandle initReadHandle( FileIO fileIO = desc.isCompressed() ? new UnzipFileIO(desc.file()) : ioFactory.create(desc.file()); try { - IgniteBiTuple tup = FileWriteAheadLogManager.readSerializerVersionAndCompactedFlag(fileIO); - - int serVer = tup.get1(); + SegmentHeader segmentHeader = readSegmentHeader(fileIO, curWalSegmIdx); - boolean isCompacted = tup.get2(); + boolean isCompacted = segmentHeader.isCompacted(); if (isCompacted) serializerFactory.skipPositionCheck(true); @@ -289,6 +290,8 @@ protected AbstractReadFileHandle initReadHandle( } } + int serVer = segmentHeader.getSerializerVersion(); + return createReadFileHandle(fileIO, desc.idx(), serializerFactory.createSerializer(serVer), in); } catch (SegmentEofException | EOFException ignore) { 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 e69cc8040f7e9..e095f6a26b30a 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 @@ -124,6 +124,8 @@ import static org.apache.ignite.failure.FailureType.SYSTEM_WORKER_TERMINATION; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.SWITCH_SEGMENT_RECORD; import static org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode.DIRECT; +import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.HEADER_RECORD_SIZE; +import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.readSegmentHeader; import static org.apache.ignite.internal.util.IgniteUtils.findField; import static org.apache.ignite.internal.util.IgniteUtils.findNonPublicMethod; @@ -1142,7 +1144,7 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws Ig // If we have existing segment, try to read version from it. if (lastReadPtr != null) { try { - serVer = readSerializerVersionAndCompactedFlag(fileIO).get1(); + serVer = readSegmentHeader(fileIO, absIdx).getSerializerVersion(); } catch (SegmentEofException | EOFException ignore) { serVer = serializerVer; @@ -1328,17 +1330,28 @@ private void checkOrPrepareFiles() throws IgniteCheckedException { } /** - * Clears the file, fills with zeros for Default mode. + * Clears whole the file, fills with zeros for Default mode. * * @param file File to format. * @throws IgniteCheckedException if formatting failed */ private void formatFile(File file) throws IgniteCheckedException { + formatFile(file, dsCfg.getWalSegmentSize()); + } + + /** + * Clears the file, fills with zeros for Default mode. + * + * @param file File to format. + * @param bytesCntToFormat Count of first bytes to format. + * @throws IgniteCheckedException if formatting failed + */ + private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Formatting file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']'); try (FileIO fileIO = ioFactory.create(file, CREATE, READ, WRITE)) { - int left = dsCfg.getWalSegmentSize(); + int left = bytesCntToFormat; if (mode == WALMode.FSYNC) { while (left > 0) { @@ -1584,13 +1597,7 @@ private synchronized boolean locked(long absIdx) { synchronized (this) { while (locked.containsKey(toArchive) && !stopped) wait(); - } - - // Firstly, format working file - if (!stopped) - formatFile(res.getOrigWorkFile()); - synchronized (this) { // Then increase counter to allow rollover on clean working file changeLastArchivedIndexAndNotifyWaiters(toArchive); @@ -1995,15 +2002,13 @@ private void compressSegmentToFile(long nextSegment, File raw, File zip) int segmentSerializerVer; try (FileIO fileIO = ioFactory.create(raw)) { - IgniteBiTuple tup = readSerializerVersionAndCompactedFlag(fileIO); - - segmentSerializerVer = tup.get1(); + segmentSerializerVer = readSegmentHeader(fileIO, nextSegment).getSerializerVersion(); } try (ZipOutputStream zos = new ZipOutputStream(new BufferedOutputStream(new FileOutputStream(zip)))) { zos.putNextEntry(new ZipEntry("")); - ByteBuffer buf = ByteBuffer.allocate(RecordV1Serializer.HEADER_RECORD_SIZE); + ByteBuffer buf = ByteBuffer.allocate(HEADER_RECORD_SIZE); buf.order(ByteOrder.nativeOrder()); zos.write(prepareSerializerVersionBuffer(nextSegment, segmentSerializerVer, true, buf).array()); @@ -2222,59 +2227,6 @@ else if (create) } } - /** - * Reads record serializer version from provided {@code io} along with compacted flag. - * NOTE: Method mutates position of {@code io}. - * - * @param io I/O interface for file. - * @return Serializer version stored in the file. - * @throws IgniteCheckedException If failed to read serializer version. - */ - static IgniteBiTuple readSerializerVersionAndCompactedFlag(FileIO io) - throws IgniteCheckedException, IOException { - try (ByteBufferExpander buf = new ByteBufferExpander(RecordV1Serializer.HEADER_RECORD_SIZE, ByteOrder.nativeOrder())) { - FileInput in = new FileInput(io, buf); - - in.ensure(RecordV1Serializer.HEADER_RECORD_SIZE); - - int recordType = in.readUnsignedByte(); - - if (recordType == WALRecord.RecordType.STOP_ITERATION_RECORD_TYPE) - throw new SegmentEofException("Reached logical end of the segment", null); - - WALRecord.RecordType type = WALRecord.RecordType.fromOrdinal(recordType - 1); - - if (type != WALRecord.RecordType.HEADER_RECORD) - throw new IOException("Can't read serializer version", null); - - // Read file pointer. - FileWALPointer ptr = RecordV1Serializer.readPosition(in); - - assert ptr.fileOffset() == 0 : "Header record should be placed at the beginning of file " + ptr; - - long hdrMagicNum = in.readLong(); - - boolean compacted; - - if (hdrMagicNum == HeaderRecord.REGULAR_MAGIC) - compacted = false; - else if (hdrMagicNum == HeaderRecord.COMPACTED_MAGIC) - compacted = true; - else { - throw new IOException("Magic is corrupted [exp=" + U.hexLong(HeaderRecord.REGULAR_MAGIC) + - ", actual=" + U.hexLong(hdrMagicNum) + ']'); - } - - // Read serializer version. - int ver = in.readInt(); - - // Read and skip CRC. - in.readInt(); - - return new IgniteBiTuple<>(ver, compacted); - } - } - /** * Needs only for WAL compaction. * @@ -2582,7 +2534,7 @@ private FileWriteHandle( * Write serializer version to current handle. */ public void writeHeader() { - SegmentedRingByteBuffer.WriteSegment seg = buf.offer(RecordV1Serializer.HEADER_RECORD_SIZE); + SegmentedRingByteBuffer.WriteSegment seg = buf.offer(HEADER_RECORD_SIZE); assert seg != null && seg.position() > 0; 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 7bb2ce4dfc48c..f582f193f9198 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 @@ -115,6 +115,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_SERIALIZER_VERSION; 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.processors.cache.persistence.wal.serializer.RecordV1Serializer.readSegmentHeader; /** * File WAL manager. @@ -1037,7 +1038,7 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws Ig // If we have existing segment, try to read version from it. if (lastReadPtr != null) { try { - serVer = readSerializerVersionAndCompactedFlag(fileIO).get1(); + serVer = readSegmentHeader(fileIO, absIdx).getSerializerVersion(); } catch (SegmentEofException | EOFException ignore) { serVer = serializerVersion; @@ -1151,16 +1152,28 @@ private void checkOrPrepareFiles() throws IgniteCheckedException { } /** - * Clears the file with zeros. + * Clears whole the file, fills with zeros for Default mode. * * @param file File to format. + * @throws IgniteCheckedException if formatting failed */ private void formatFile(File file) throws IgniteCheckedException { + formatFile(file, dsCfg.getWalSegmentSize()); + } + + /** + * Clears the file, fills with zeros for Default mode. + * + * @param file File to format. + * @param bytesCntToFormat Count of first bytes to format. + * @throws IgniteCheckedException if formatting failed + */ + private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Formatting file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']'); try (FileIO fileIO = ioFactory.create(file, CREATE, READ, WRITE)) { - int left = dsCfg.getWalSegmentSize(); + int left = bytesCntToFormat; if (mode == WALMode.FSYNC) { while (left > 0) { @@ -1424,13 +1437,7 @@ private synchronized void release(long absIdx) { synchronized (this) { while (locked.containsKey(toArchive) && !stopped) wait(); - } - - // Firstly, format working file - if (!stopped) - formatFile(res.getOrigWorkFile()); - synchronized (this) { // Then increase counter to allow rollover on clean working file changeLastArchivedIndexAndWakeupCompressor(toArchive); @@ -1833,9 +1840,7 @@ private void compressSegmentToFile(long nextSegment, File raw, File zip) int segmentSerializerVer; try (FileIO fileIO = ioFactory.create(raw)) { - IgniteBiTuple tup = readSerializerVersionAndCompactedFlag(fileIO); - - segmentSerializerVer = tup.get1(); + segmentSerializerVer = readSegmentHeader(fileIO, nextSegment).getSerializerVersion(); } try (ZipOutputStream zos = new ZipOutputStream(new BufferedOutputStream(new FileOutputStream(zip)))) { @@ -2029,58 +2034,6 @@ else if (create) } } - /** - * Reads record serializer version from provided {@code io} along with compacted flag. - * NOTE: Method mutates position of {@code io}. - * - * @param io I/O interface for file. - * @return Serializer version stored in the file. - * @throws IgniteCheckedException If failed to read serializer version. - */ - public static IgniteBiTuple readSerializerVersionAndCompactedFlag(FileIO io) - throws IgniteCheckedException, IOException { - try (ByteBufferExpander buf = new ByteBufferExpander(RecordV1Serializer.HEADER_RECORD_SIZE, ByteOrder.nativeOrder())) { - FileInput in = new FileInput(io, buf); - - in.ensure(RecordV1Serializer.HEADER_RECORD_SIZE); - - int recordType = in.readUnsignedByte(); - - if (recordType == WALRecord.RecordType.STOP_ITERATION_RECORD_TYPE) - throw new SegmentEofException("Reached logical end of the segment", null); - - WALRecord.RecordType type = WALRecord.RecordType.fromOrdinal(recordType - 1); - - if (type != WALRecord.RecordType.HEADER_RECORD) - throw new IOException("Can't read serializer version", null); - - // Read file pointer. - FileWALPointer ptr = RecordV1Serializer.readPosition(in); - - assert ptr.fileOffset() == 0 : "Header record should be placed at the beginning of file " + ptr; - - long hdrMagicNum = in.readLong(); - - boolean compacted; - if (hdrMagicNum == HeaderRecord.REGULAR_MAGIC) - compacted = false; - else if (hdrMagicNum == HeaderRecord.COMPACTED_MAGIC) - compacted = true; - else { - throw new IOException("Magic is corrupted [exp=" + U.hexLong(HeaderRecord.REGULAR_MAGIC) + - ", actual=" + U.hexLong(hdrMagicNum) + ']'); - } - - // Read serializer version. - int ver = in.readInt(); - - // Read and skip CRC. - in.readInt(); - - return new IgniteBiTuple<>(ver, compacted); - } - } - /** * Writes record serializer version to provided {@code io}. * NOTE: Method mutates position of {@code io}. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java index 5ae269b953095..dd0819c859ce8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java @@ -29,17 +29,21 @@ import org.apache.ignite.internal.pagemem.wal.record.MarshalledRecord; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; import org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.tree.io.CacheVersionIO; import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput; +import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferExpander; 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.SegmentEofException; import org.apache.ignite.internal.processors.cache.persistence.wal.WalSegmentTailReachedException; 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.io.RecordIO; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiPredicate; import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_SKIP_CRC; @@ -218,6 +222,63 @@ public static void putPosition(ByteBuffer buf, FileWALPointer ptr) { buf.putInt(ptr.fileOffset()); } + /** + * Reads stored record from provided {@code io}. + * NOTE: Method mutates position of {@code io}. + * + * @param io I/O interface for file. + * @param expectedIdx Expected WAL segment index for readable record. + * @return Instance of {@link SegmentHeader} extracted from the file. + * @throws IgniteCheckedException If failed to read serializer version. + */ + public static SegmentHeader readSegmentHeader(FileIO io, long expectedIdx) + throws IgniteCheckedException, IOException { + try (ByteBufferExpander buf = new ByteBufferExpander(HEADER_RECORD_SIZE, ByteOrder.nativeOrder())) { + FileInput in = new FileInput(io, buf); + + in.ensure(HEADER_RECORD_SIZE); + + int recordType = in.readUnsignedByte(); + + if (recordType == WALRecord.RecordType.STOP_ITERATION_RECORD_TYPE) + throw new SegmentEofException("Reached logical end of the segment", null); + + WALRecord.RecordType type = WALRecord.RecordType.fromOrdinal(recordType - 1); + + if (type != WALRecord.RecordType.HEADER_RECORD) + throw new IOException("Can't read serializer version", null); + + // Read file pointer. + FileWALPointer ptr = readPosition(in); + + if (expectedIdx != ptr.index()) + throw new SegmentEofException("Reached logical end of the segment by pointer", null); + + assert ptr.fileOffset() == 0 : "Header record should be placed at the beginning of file " + ptr; + + long hdrMagicNum = in.readLong(); + + boolean compacted; + + if (hdrMagicNum == HeaderRecord.REGULAR_MAGIC) + compacted = false; + else if (hdrMagicNum == HeaderRecord.COMPACTED_MAGIC) + compacted = true; + else { + throw new IOException("Magic is corrupted [exp=" + U.hexLong(HeaderRecord.REGULAR_MAGIC) + + ", actual=" + U.hexLong(hdrMagicNum) + ']'); + } + + // Read serializer version. + int ver = in.readInt(); + + // Read and skip CRC. + in.readInt(); + + return new SegmentHeader(ver, compacted); + } + } + /** * @param in Data input to read pointer from. * @return Read file WAL pointer. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/SegmentHeader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/SegmentHeader.java new file mode 100644 index 0000000000000..8f7e738d6f8fa --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/SegmentHeader.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.cache.persistence.wal.serializer; + +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * WAL segment header info. + */ +public class SegmentHeader { + /** Serializer version. */ + private int serializerVersion; + /** Compacted flag. */ + private boolean isCompacted; + + /** + * @param serializerVersion Serializer version. + * @param isCompacted Compacted flag. + */ + public SegmentHeader(int serializerVersion, boolean isCompacted) { + this.serializerVersion = serializerVersion; + this.isCompacted = isCompacted; + } + + /** + * @return Record serializer version. + */ + public int getSerializerVersion() { + return serializerVersion; + } + + /** + * @return Comacted flag. + */ + public boolean isCompacted() { + return isCompacted; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(SegmentHeader.class, this); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java index 857c0d0f2983b..28f049aaef5c9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java @@ -23,6 +23,7 @@ import java.io.ObjectInput; import java.io.ObjectOutput; import java.io.Serializable; +import java.util.Arrays; import java.util.Collection; import java.util.EnumMap; import java.util.HashMap; @@ -213,45 +214,44 @@ public void testFillWalAndReadRecords() throws Exception { final File wal = new File(db, "wal"); final File walArchive = setWalAndArchiveToSameValue ? wal : new File(wal, "archive"); - final MockWalIteratorFactory mockItFactory = new MockWalIteratorFactory(log, PAGE_SIZE, consistentId, subfolderName, WAL_SEGMENTS); - final WALIterator it = mockItFactory.iterator(wal, walArchive); - final int cntUsingMockIter = iterateAndCount(it, false); - - log.info("Total records loaded " + cntUsingMockIter); - assertTrue(cntUsingMockIter > 0); - assertTrue(cntUsingMockIter > cacheObjectsToWrite); + int[] checkKeyIterArr = new int[cacheObjectsToWrite]; final File walArchiveDirWithConsistentId = new File(walArchive, subfolderName); final File walWorkDirWithConsistentId = new File(wal, subfolderName); final IgniteWalIteratorFactory factory = createWalIteratorFactory(workDir, subfolderName); + + //Check iteratorArchiveDirectory and iteratorArchiveFiles are same. final int cntArchiveDir = iterateAndCount(factory.iteratorArchiveDirectory(walArchiveDirWithConsistentId)); log.info("Total records loaded using directory : " + cntArchiveDir); - final int cntArchiveFileByFile = iterateAndCount( - factory.iteratorArchiveFiles( - walArchiveDirWithConsistentId.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER))); + final int cntArchiveFileByFile = iterateAndCount(factory.iteratorArchiveFiles( + walArchiveDirWithConsistentId.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER))); log.info("Total records loaded using archive directory (file-by-file): " + cntArchiveFileByFile); - assertTrue(cntArchiveFileByFile > cacheObjectsToWrite); - assertTrue(cntArchiveDir > cacheObjectsToWrite); assertTrue(cntArchiveDir == cntArchiveFileByFile); - //really count2 may be less because work dir correct loading is not supported yet - assertTrue("Mock based reader loaded " + cntUsingMockIter + " records " + - "but standalone has loaded only " + cntArchiveDir, - cntUsingMockIter >= cntArchiveDir); - final File[] workFiles = walWorkDirWithConsistentId.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER); + //Check iteratorArchiveFiles + iteratorWorkFiles iterate over all entries. + Arrays.fill(checkKeyIterArr, 0); - final int cntWork = iterateAndCount(factory.iteratorWorkFiles(workFiles)); + iterateAndCountDataRecord(factory.iteratorArchiveFiles( + walArchiveDirWithConsistentId.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER)), new IgniteBiInClosure() { + @Override public void apply(Object o, Object o2) { + checkKeyIterArr[(Integer)o]++; + } + }, null); - log.info("Total records loaded from work: " + cntWork); + final File[] workFiles = walWorkDirWithConsistentId.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER); + + iterateAndCountDataRecord(factory.iteratorWorkFiles(workFiles), new IgniteBiInClosure() { + @Override public void apply(Object o, Object o2) { + checkKeyIterArr[(Integer) o]++; + } + }, null).size(); - assertTrue("Work iterator loaded [" + cntWork + "] " + - "Archive iterator loaded [" + cntArchiveFileByFile + "]; " + - "mock iterator [" + cntUsingMockIter + "]", - cntWork + cntArchiveFileByFile == cntUsingMockIter); + for (int i =0 ; i< cacheObjectsToWrite; i++) + assertTrue("Iterator didn't find key="+ i, checkKeyIterArr[i] > 0); } /** From 470e5c13b6fe265c00f4c6318b865f1043acbc6f Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 18 May 2018 19:28:45 +0300 Subject: [PATCH 0187/1463] IGNITE-8531 Fixed NPE if checkpoint has no pages to write, but has partitions to destroy. - Fixes #4026. Signed-off-by: Alexey Goncharuk (cherry picked from commit 5c8d9ff) --- .../GridCacheDatabaseSharedManager.java | 24 ++++++++++------ .../IgnitePdsPartitionFilesDestroyTest.java | 28 +++++++++++++++++++ 2 files changed, 43 insertions(+), 9 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 2f95ed68b84c7..ffa7259669775 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 @@ -3049,6 +3049,8 @@ private void doCheckpoint() { boolean success = false; + int destroyedPartitionsCnt; + try { if (chp.hasDelta()) { // Identity stores set. @@ -3135,7 +3137,7 @@ private void doCheckpoint() { snapshotMgr.afterCheckpointPageWritten(); try { - destroyEvictedPartitions(); + destroyedPartitionsCnt = destroyEvictedPartitions(); } catch (IgniteCheckedException e) { chp.progress.cpFinishFut.onDone(e); @@ -3155,15 +3157,15 @@ private void doCheckpoint() { tracker.onEnd(); - if (chp.hasDelta()) { + if (chp.hasDelta() || destroyedPartitionsCnt > 0) { if (printCheckpointStats) { if (log.isInfoEnabled()) log.info(String.format("Checkpoint finished [cpId=%s, pages=%d, markPos=%s, " + "walSegmentsCleared=%d, markDuration=%dms, pagesWrite=%dms, fsync=%dms, " + "total=%dms]", - chp.cpEntry.checkpointId(), + chp.cpEntry != null ? chp.cpEntry.checkpointId() : "", chp.pagesSize, - chp.cpEntry.checkpointMark(), + chp.cpEntry != null ? chp.cpEntry.checkpointMark() : "", chp.walFilesDeleted, tracker.markDuration(), tracker.pagesWriteDuration(), @@ -3203,12 +3205,14 @@ private void doCheckpoint() { * Processes all evicted partitions scheduled for destroy. * * @throws IgniteCheckedException If failed. + * + * @return The number of destroyed partition files. */ - private void destroyEvictedPartitions() throws IgniteCheckedException { + private int destroyEvictedPartitions() throws IgniteCheckedException { PartitionDestroyQueue destroyQueue = curCpProgress.destroyQueue; if (destroyQueue.pendingReqs.isEmpty()) - return; + return 0; List reqs = null; @@ -3266,6 +3270,8 @@ private void destroyEvictedPartitions() throws IgniteCheckedException { req.waitCompleted(); destroyQueue.pendingReqs.clear(); + + return reqs != null ? reqs.size() : 0; } /** @@ -3434,7 +3440,7 @@ private Checkpoint markCheckpointBegin(CheckpointMetricsTracker tracker) throws hasPages = hasPageForWrite(cpPagesTuple.get1()); - if (hasPages || curr.nextSnapshot) { + if (hasPages || curr.nextSnapshot || !curr.destroyQueue.pendingReqs.isEmpty()) { // No page updates for this checkpoint are allowed from now on. cpPtr = cctx.wal().log(cpRec); @@ -3813,7 +3819,7 @@ private enum CheckpointEntryType { */ private static class Checkpoint { /** Checkpoint entry. */ - private final CheckpointEntry cpEntry; + @Nullable private final CheckpointEntry cpEntry; /** Checkpoint pages. */ private final GridMultiCollectionWrapper cpPages; @@ -3833,7 +3839,7 @@ private static class Checkpoint { * @param progress Checkpoint progress status. */ private Checkpoint( - CheckpointEntry cpEntry, + @Nullable CheckpointEntry cpEntry, @NotNull GridMultiCollectionWrapper cpPages, CheckpointProgress progress ) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java index b5afddf542889..5e0ccc9ce1fc8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsPartitionFilesDestroyTest.java @@ -19,6 +19,7 @@ import java.io.File; import java.io.IOException; import java.nio.file.OpenOption; +import java.util.List; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; @@ -332,6 +333,33 @@ public void testPartitionFileDestroyCrashRecovery2() throws Exception { checkData((IgniteEx) ignite, keysCnt, 1); } + /** + * Test destroy when partition files are empty and there are no pages for checkpoint. + * + * @throws Exception If failed. + */ + public void testDestroyWhenPartitionsAreEmpty() throws Exception { + IgniteEx crd = (IgniteEx) startGrids(2); + + crd.cluster().active(true); + + forceCheckpoint(); + + // Evict arbitrary partition. + List parts = crd.cachex(CACHE).context().topology().localPartitions(); + for (GridDhtLocalPartition part : parts) + if (part.state() != GridDhtPartitionState.EVICTED) { + part.rent(false).get(); + + break; + } + + // This checkpoint has no pages to write, but has one partition file to destroy. + forceCheckpoint(crd); + + checkPartitionFiles(crd, false); + } + /** * If {@code exists} is {@code true}, checks that all partition files exist * if partition has state EVICTED. From 3e006bead895764d4757795b935e90c04d0ca10a Mon Sep 17 00:00:00 2001 From: vd-pyatkov Date: Fri, 18 May 2018 16:59:14 +0300 Subject: [PATCH 0188/1463] IGNITE-8491 Add JMX flag: Is the node in baseline or not - Fixes #4010. Signed-off-by: Ivan Rakov (cherry picked from commit f8ae30d) --- .../apache/ignite/internal/IgniteKernal.java | 13 +++ .../apache/ignite/mxbean/IgniteMXBean.java | 8 ++ .../util/mbeans/GridMBeanBaselineTest.java | 96 +++++++++++++++++++ 3 files changed, 117 insertions(+) create mode 100644 modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanBaselineTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 046c4b8f84339..a3e6ad9beed1f 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 @@ -106,6 +106,7 @@ import org.apache.ignite.internal.cluster.ClusterGroupAdapter; import org.apache.ignite.internal.cluster.IgniteClusterEx; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; import org.apache.ignite.internal.processors.failure.FailureProcessor; import org.apache.ignite.internal.managers.GridManager; import org.apache.ignite.internal.managers.checkpoint.GridCheckpointManager; @@ -524,6 +525,18 @@ public IgniteKernal(@Nullable GridSpringResourceContext rsrcCtx) { .toString(); } + /** {@inheritDoc} */ + @Override public boolean isNodeInBaseline() { + ClusterNode locNode = localNode(); + + if (locNode.isClient() || locNode.isDaemon()) + return false; + + DiscoveryDataClusterState clusterState = ctx.state().clusterState(); + + return clusterState.hasBaselineTopology() && CU.baselineNode(locNode, clusterState); + } + /** {@inheritDoc} */ @Override public String getCommunicationSpiFormatted() { assert cfg != null; diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java index cc4bc5b65f350..e0a5fb5c5df49 100644 --- a/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java +++ b/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java @@ -438,6 +438,14 @@ public interface IgniteMXBean { @MXBeanDescription("Formatted properties of current coordinator.") public String getCurrentCoordinatorFormatted(); + /** + * Gets a flag whether local node is in baseline. Returns false if baseline topology is not established. + * + * @return Return a baseline flag. + */ + @MXBeanDescription("Baseline node flag.") + public boolean isNodeInBaseline(); + /** * Runs IO latency test against all remote server nodes in cluster. * diff --git a/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanBaselineTest.java b/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanBaselineTest.java new file mode 100644 index 0000000000000..00dce83f1cca6 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanBaselineTest.java @@ -0,0 +1,96 @@ +package org.apache.ignite.util.mbeans; + +import org.apache.ignite.Ignite; +import org.apache.ignite.Ignition; +import org.apache.ignite.cluster.BaselineNode; +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.mxbean.IgniteMXBean; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * + */ +public class GridMBeanBaselineTest extends GridCommonAbstractTest { + /** Client index. */ + private static final int CLIENT_IDX = 33; + + /** Nodes. */ + public static final int NODES = 2; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + return super.getConfiguration(igniteInstanceName) + .setClientMode(igniteInstanceName.equals(getTestIgniteInstanceName(CLIENT_IDX))) + .setDataStorageConfiguration(new DataStorageConfiguration() + .setCheckpointFrequency(2_000) + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration() + .setPersistenceEnabled(true))) + .setDataStorageConfiguration( + new DataStorageConfiguration() + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration() + .setMaxSize(200L * 1024 * 1024) + .setPersistenceEnabled(true))); + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + cleanPersistenceDir(); + } + + /** + * Test ignite kernal node in baseline test. + * + * @throws Exception Thrown if test fails. + */ + public void testIgniteKernalNodeInBaselineTest() throws Exception { + try { + IgniteEx ignite0 = (IgniteEx)startGrids(NODES); + + startGrid(CLIENT_IDX); + + ignite0.cluster().active(true); + + checkBaselineInFromMBean(ignite0); + + startGrid(NODES); + + checkBaselineInFromMBean(ignite0); + + ignite0.cluster().setBaselineTopology(ignite0.cluster().topologyVersion()); + + checkBaselineInFromMBean(ignite0); + } + finally { + stopAllGrids(); + } + } + + /** + * @param ignite Ignite. + */ + private void checkBaselineInFromMBean(IgniteEx ignite) { + Set cIds = ignite.cluster().currentBaselineTopology().stream() + .map(BaselineNode::consistentId) + .collect(Collectors.toSet()); + + for (Ignite ign : Ignition.allGrids()) { + IgniteMXBean igniteMXBean = (IgniteMXBean)ign; + + assertEquals(cIds.contains(ign.cluster().localNode().consistentId()), + igniteMXBean.isNodeInBaseline()); + } + } + +} From ada3e23ae5ca5915af55adf16b70a9a711134123 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Mon, 21 May 2018 15:13:37 +0300 Subject: [PATCH 0189/1463] IGNITE-8521 Do not attempt to unregister continuous query if query ID is null --- .../cache/query/continuous/CacheContinuousQueryManager.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 19225f8ebd984..55c44b489d7a7 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 @@ -703,7 +703,8 @@ private UUID executeQuery0(CacheEntryUpdatedListener locLsnr, catch (IgniteCheckedException e) { log.warning("Failed to start continuous query.", e); - cctx.kernalContext().continuous().stopRoutine(id); + if (id != null) + cctx.kernalContext().continuous().stopRoutine(id); throw new IgniteCheckedException("Failed to start continuous query.", e); } From c44c6d13782c7dbe842f950acb9e0b17e0f53da5 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Mon, 21 May 2018 22:33:50 +0300 Subject: [PATCH 0190/1463] IGNITE-8544 Use exchange result topology version for local wal state management. - Fixes #4039. Signed-off-by: Alexey Goncharuk --- .../GridDhtPartitionsExchangeFuture.java | 2 +- ...alModeChangeDuringRebalancingSelfTest.java | 66 +++++++++++++++++++ 2 files changed, 67 insertions(+), 1 deletion(-) 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 1b79b767daaa8..c62b067013357 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 @@ -1723,7 +1723,7 @@ public void finishMerged() { grp.topology().onExchangeDone(this, grp.affinity().readyAffinity(res), false); } - cctx.walState().changeLocalStatesOnExchangeDone(exchId.topologyVersion()); + cctx.walState().changeLocalStatesOnExchangeDone(res); } if (super.onDone(res, err)) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java index aa2613fa3a1a2..8be819f27b4e8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java @@ -24,10 +24,12 @@ import java.nio.file.OpenOption; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; +import com.sun.org.apache.regexp.internal.RE; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.DataRegionConfiguration; @@ -49,6 +51,7 @@ 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; /** * @@ -63,6 +66,9 @@ public class LocalWalModeChangeDuringRebalancingSelfTest extends GridCommonAbstr /** */ private static final AtomicReference fileIOLatch = new AtomicReference<>(); + /** Replicated cache name. */ + private static final String REPL_CACHE = "cache"; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); @@ -83,7 +89,11 @@ public class LocalWalModeChangeDuringRebalancingSelfTest extends GridCommonAbstr cfg.setCacheConfiguration( new CacheConfiguration(DEFAULT_CACHE_NAME) // Test checks internal state before and after rebalance, so it is configured to be triggered manually + .setRebalanceDelay(-1), + + new CacheConfiguration(REPL_CACHE) .setRebalanceDelay(-1) + .setCacheMode(CacheMode.REPLICATED) ); cfg.setCommunicationSpi(new TcpCommunicationSpi() { @@ -294,6 +304,62 @@ public void testLocalAndGlobalWalStateInterdependence() throws Exception { assertTrue(grpCtx.walEnabled()); } + /** + * Test that local WAL mode changing works well with exchanges merge. + * + * @throws Exception If failed. + */ + public void testWithExchangesMerge() throws Exception { + final int nodeCnt = 5; + final int keyCnt = 10_000; + + Ignite ignite = startGrids(nodeCnt); + + ignite.cluster().active(true); + + IgniteCache cache = ignite.cache(REPL_CACHE); + + for (int k = 0; k < keyCnt; k++) + cache.put(k, k); + + stopGrid(2); + stopGrid(3); + stopGrid(4); + + // Rewrite data to trigger further rebalance. + for (int k = 0; k < keyCnt; k++) + cache.put(k, k * 2); + + // Start several grids in parallel to trigger exchanges merge. + startGridsMultiThreaded(2, 3); + + for (int nodeIdx = 2; nodeIdx < nodeCnt; nodeIdx++) { + CacheGroupContext grpCtx = grid(nodeIdx).cachex(REPL_CACHE).context().group(); + + assertFalse(grpCtx.walEnabled()); + } + + // Invoke rebalance manually. + for (Ignite g : G.allGrids()) + g.cache(REPL_CACHE).rebalance(); + + awaitPartitionMapExchange(); + + for (int nodeIdx = 2; nodeIdx < nodeCnt; nodeIdx++) { + CacheGroupContext grpCtx = grid(nodeIdx).cachex(REPL_CACHE).context().group(); + + assertTrue(grpCtx.walEnabled()); + } + + // Check no data loss. + for (int nodeIdx = 2; nodeIdx < nodeCnt; nodeIdx++) { + IgniteCache cache0 = grid(nodeIdx).cache(REPL_CACHE); + + for (int k = 0; k < keyCnt; k++) + Assert.assertEquals("nodeIdx=" + nodeIdx + ", key=" + k, (Integer) (2 * k), cache0.get(k)); + } + } + /** * @throws Exception If failed. */ From 67d72e6aee444a470e7e0a53f6573e3a5178872c Mon Sep 17 00:00:00 2001 From: Anton Kalashnikov Date: Wed, 23 May 2018 12:24:51 +0300 Subject: [PATCH 0191/1463] IGNITE-8561 SingleSegmentLogicalRecordsIterator is broken - Fixes #4045. (cherry picked from commit 21678bc) --- .../wal/SingleSegmentLogicalRecordsIterator.java | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) 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 d5c10cf72bc4d..36e5b0e21b3a9 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 @@ -47,9 +47,6 @@ public class SingleSegmentLogicalRecordsIterator extends AbstractWalRecordsItera /** Segment initialized flag. */ private boolean segmentInitialized; - /** Archived segment index. */ - private long archivedSegIdx; - /** Archive directory. */ private File archiveDir; @@ -76,7 +73,7 @@ public class SingleSegmentLogicalRecordsIterator extends AbstractWalRecordsItera ) throws IgniteCheckedException { super(log, sharedCtx, initLogicalRecordsSerializerFactory(sharedCtx), ioFactory, bufSize); - this.archivedSegIdx = archivedSegIdx; + curWalSegmIdx = archivedSegIdx; this.archiveDir = archiveDir; this.advanceC = advanceC; @@ -106,7 +103,7 @@ private static RecordSerializerFactory initLogicalRecordsSerializerFactory(GridC segmentInitialized = true; FileWriteAheadLogManager.FileDescriptor fd = new FileWriteAheadLogManager.FileDescriptor( - new File(archiveDir, FileWriteAheadLogManager.FileDescriptor.fileName(archivedSegIdx))); + new File(archiveDir, FileWriteAheadLogManager.FileDescriptor.fileName(curWalSegmIdx))); try { return initReadHandle(fd, null); From 988dabd780547d98e1333a49c3c8e276e8607bdb Mon Sep 17 00:00:00 2001 From: vd-pyatkov Date: Tue, 22 May 2018 12:49:21 +0300 Subject: [PATCH 0192/1463] IGNITE-8491 Add JMX flag: Is the node in baseline or not - fixed license header Signed-off-by: Ivan Rakov (cherry picked from commit e227f22) --- .../util/mbeans/GridMBeanBaselineTest.java | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanBaselineTest.java b/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanBaselineTest.java index 00dce83f1cca6..390eb21386dae 100644 --- a/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanBaselineTest.java +++ b/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanBaselineTest.java @@ -1,3 +1,19 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ package org.apache.ignite.util.mbeans; import org.apache.ignite.Ignite; From cea086c3d0dca8dd023cb1082ddf777ef9d2bc09 Mon Sep 17 00:00:00 2001 From: Dmitriy Sorokin Date: Thu, 24 May 2018 16:03:41 +0300 Subject: [PATCH 0193/1463] IGNITE-8584 Provide ability to terminate any thread with enabled test features. Signed-off-by: Andrey Gura --- .../internal/util/nio/GridNioServer.java | 2 +- .../worker/WorkersControlMXBeanImpl.java | 29 ++++++ .../ignite/mxbean/WorkersControlMXBean.java | 30 ++++++ .../testsuites/IgniteUtilSelfTestSuite.java | 2 + .../util/mbeans/WorkersControlMXBeanTest.java | 98 +++++++++++++++++++ 5 files changed, 160 insertions(+), 1 deletion(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/util/mbeans/WorkersControlMXBeanTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java index 3597a050051f3..da3438e069414 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java @@ -1790,7 +1790,7 @@ else if (!closed) { if (err == null) lsnr.onFailure(SYSTEM_WORKER_TERMINATION, new IllegalStateException("Thread " + name() + " is terminated unexpectedly")); - else if (err instanceof InterruptedException) + else lsnr.onFailure(SYSTEM_WORKER_TERMINATION, err); } else if (err != null) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersControlMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersControlMXBeanImpl.java index 9e427e8f2ab1c..65f872c0162ca 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersControlMXBeanImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersControlMXBeanImpl.java @@ -19,6 +19,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Objects; import org.apache.ignite.internal.util.worker.GridWorker; import org.apache.ignite.mxbean.WorkersControlMXBean; @@ -59,4 +60,32 @@ public WorkersControlMXBeanImpl(WorkersRegistry registry) { return true; } + + /** {@inheritDoc} */ + @Override public boolean stopThreadByUniqueName(String name) { + Thread[] threads = Thread.getAllStackTraces().keySet().stream() + .filter(t -> Objects.equals(t.getName(), name)) + .toArray(Thread[]::new); + + if (threads.length != 1) + return false; + + threads[0].stop(); + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean stopThreadById(long id) { + Thread[] threads = Thread.getAllStackTraces().keySet().stream() + .filter(t -> t.getId() == id) + .toArray(Thread[]::new); + + if (threads.length != 1) + return false; + + threads[0].stop(); + + return true; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/WorkersControlMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/WorkersControlMXBean.java index 0f5419b3b4aac..b999ab7d716de 100644 --- a/modules/core/src/main/java/org/apache/ignite/mxbean/WorkersControlMXBean.java +++ b/modules/core/src/main/java/org/apache/ignite/mxbean/WorkersControlMXBean.java @@ -46,4 +46,34 @@ public interface WorkersControlMXBean { "Name of worker to terminate." ) public boolean terminateWorker(String name); + + /** + * Stops thread by {@code name}, if exists and unique. + * + * @param name Thread name. + * @return {@code True} if thread has been stopped successfully, {@code false} otherwise. + */ + @MXBeanDescription("Stops thread by unique name.") + @MXBeanParametersNames( + "name" + ) + @MXBeanParametersDescriptions( + "Name of thread to stop." + ) + public boolean stopThreadByUniqueName(String name); + + /** + * Stops thread by {@code id}, if exists. + * + * @param id Thread id. + * @return {@code True} if thread has been stopped successfully, {@code false} otherwise. + */ + @MXBeanDescription("Stops thread by id.") + @MXBeanParametersNames( + "id" + ) + @MXBeanParametersDescriptions( + "Id of thread to stop." + ) + public boolean stopThreadById(long id); } 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 124300585b4e7..791621ff598b0 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 @@ -53,6 +53,7 @@ import org.apache.ignite.util.mbeans.GridMBeanDisableSelfTest; import org.apache.ignite.util.mbeans.GridMBeanExoticNamesSelfTest; import org.apache.ignite.util.mbeans.GridMBeanSelfTest; +import org.apache.ignite.util.mbeans.WorkersControlMXBeanTest; /** * Test suite for Ignite utility classes. @@ -92,6 +93,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(GridCacheUtilsSelfTest.class); suite.addTestSuite(IgniteExceptionRegistrySelfTest.class); suite.addTestSuite(GridMessageCollectionTest.class); + suite.addTestSuite(WorkersControlMXBeanTest.class); // Metrics. suite.addTestSuite(ClusterMetricsSnapshotSerializeSelfTest.class); diff --git a/modules/core/src/test/java/org/apache/ignite/util/mbeans/WorkersControlMXBeanTest.java b/modules/core/src/test/java/org/apache/ignite/util/mbeans/WorkersControlMXBeanTest.java new file mode 100644 index 0000000000000..c1c2fdae87726 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/util/mbeans/WorkersControlMXBeanTest.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.util.mbeans; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import org.apache.ignite.internal.worker.WorkersControlMXBeanImpl; +import org.apache.ignite.mxbean.WorkersControlMXBean; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * {@link WorkersControlMXBean} test. + */ +public class WorkersControlMXBeanTest extends GridCommonAbstractTest { + /** Test thread name. */ + private static final String TEST_THREAD_NAME = "test-thread"; + + /** + * @throws Exception Thrown if test fails. + */ + public void testStopThreadByUniqueName() throws Exception { + WorkersControlMXBean workersCtrlMXBean = new WorkersControlMXBeanImpl(null); + + Thread t = startTestThread(); + + assertTrue(workersCtrlMXBean.stopThreadByUniqueName(TEST_THREAD_NAME)); + + t.join(500); + + assertFalse(workersCtrlMXBean.stopThreadByUniqueName(TEST_THREAD_NAME)); + + Thread t1 = startTestThread(); + Thread t2 = startTestThread(); + + assertFalse(workersCtrlMXBean.stopThreadByUniqueName(TEST_THREAD_NAME)); + + t1.stop(); + t2.stop(); + } + + /** + * @throws Exception Thrown if test fails. + */ + public void testStopThreadById() throws Exception { + WorkersControlMXBean workersCtrlMXBean = new WorkersControlMXBeanImpl(null); + + Thread t1 = startTestThread(); + Thread t2 = startTestThread(); + + assertTrue(workersCtrlMXBean.stopThreadById(t1.getId())); + assertTrue(workersCtrlMXBean.stopThreadById(t2.getId())); + + t1.join(500); + t2.join(500); + + assertFalse(workersCtrlMXBean.stopThreadById(t1.getId())); + assertFalse(workersCtrlMXBean.stopThreadById(t2.getId())); + } + + /** + * @return Started thread. + */ + private static Thread startTestThread() throws InterruptedException { + final CountDownLatch latch = new CountDownLatch(1); + + Thread t = new Thread(TEST_THREAD_NAME) { + public void run() { + latch.countDown(); + + for (;;) + ; + } + }; + + t.start(); + + assertTrue(latch.await(500, TimeUnit.MILLISECONDS)); + + assertTrue(t.isAlive()); + + return t; + } +} From 958333b38dcf1bf9164b25c2e6df8941c5099c5f Mon Sep 17 00:00:00 2001 From: Andrey Gura Date: Thu, 24 May 2018 16:56:42 +0300 Subject: [PATCH 0194/1463] IGNITE-8563 Fixed WAL file archiver does not propagate file archiving error to error handler --- .../wal/FileWriteAheadLogManager.java | 28 +++++++------------ .../FsyncModeFileWriteAheadLogManager.java | 28 +++++++------------ 2 files changed, 20 insertions(+), 36 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 e095f6a26b30a..40ebcf04289fd 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 @@ -1591,29 +1591,21 @@ private synchronized boolean locked(long absIdx) { if (stopped) break; - try { - final SegmentArchiveResult res = archiveSegment(toArchive); + final SegmentArchiveResult res = archiveSegment(toArchive); - synchronized (this) { - while (locked.containsKey(toArchive) && !stopped) - wait(); + synchronized (this) { + while (locked.containsKey(toArchive) && !stopped) + wait(); - // Then increase counter to allow rollover on clean working file - changeLastArchivedIndexAndNotifyWaiters(toArchive); + // Then increase counter to allow rollover on clean working file + changeLastArchivedIndexAndNotifyWaiters(toArchive); - notifyAll(); - } - - if (evt.isRecordable(EventType.EVT_WAL_SEGMENT_ARCHIVED)) - evt.record(new WalSegmentArchivedEvent(cctx.discovery().localNode(), - res.getAbsIdx(), res.getDstArchiveFile())); + notifyAll(); } - catch (IgniteCheckedException e) { - synchronized (this) { - cleanErr = e; - notifyAll(); - } + if (evt.isRecordable(EventType.EVT_WAL_SEGMENT_ARCHIVED)) { + evt.record(new WalSegmentArchivedEvent(cctx.discovery().localNode(), + res.getAbsIdx(), res.getDstArchiveFile())); } } } 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 f582f193f9198..79a3e1966dede 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 @@ -1431,29 +1431,21 @@ private synchronized void release(long absIdx) { if (stopped) break; - try { - final SegmentArchiveResult res = archiveSegment(toArchive); + final SegmentArchiveResult res = archiveSegment(toArchive); - synchronized (this) { - while (locked.containsKey(toArchive) && !stopped) - wait(); + synchronized (this) { + while (locked.containsKey(toArchive) && !stopped) + wait(); - // Then increase counter to allow rollover on clean working file - changeLastArchivedIndexAndWakeupCompressor(toArchive); + // Then increase counter to allow rollover on clean working file + changeLastArchivedIndexAndWakeupCompressor(toArchive); - notifyAll(); - } - - if (evt.isRecordable(EventType.EVT_WAL_SEGMENT_ARCHIVED)) - evt.record(new WalSegmentArchivedEvent(cctx.discovery().localNode(), - res.getAbsIdx(), res.getDstArchiveFile())); + notifyAll(); } - catch (IgniteCheckedException e) { - synchronized (this) { - cleanException = e; - notifyAll(); - } + if (evt.isRecordable(EventType.EVT_WAL_SEGMENT_ARCHIVED)) { + evt.record(new WalSegmentArchivedEvent(cctx.discovery().localNode(), + res.getAbsIdx(), res.getDstArchiveFile())); } } } From 55fc1da6036d49aec5e273233c056d84615a247b Mon Sep 17 00:00:00 2001 From: Dmitriy Govorukhin Date: Thu, 24 May 2018 18:04:02 +0300 Subject: [PATCH 0195/1463] IGNITE-8583 DataStorageMetricsMXBean.getOffHeapSize include checkpoint buffer size - Fixes #4054. Signed-off-by: dpavlov (cherry picked from commit 86c1899) --- .../org/apache/ignite/DataRegionMetrics.java | 11 ++- .../org/apache/ignite/DataStorageMetrics.java | 21 ++++ .../persistence/DataRegionMetricsImpl.java | 31 ++++-- .../DataRegionMetricsMXBeanImpl.java | 9 +- .../DataRegionMetricsSnapshot.java | 17 +++- .../persistence/DataStorageMetricsImpl.java | 54 +++++++++++ .../DataStorageMetricsSnapshot.java | 33 ++++++- .../GridCacheDatabaseSharedManager.java | 46 +++++++++ .../cluster/PlatformClusterGroup.java | 4 +- .../visor/cache/VisorMemoryMetrics.java | 4 +- .../mxbean/DataStorageMetricsMXBean.java | 12 +++ .../db/IgnitePdsDataRegionMetricsTest.java | 97 +++++++++++++++++-- .../ApiParity/DataRegionMetricsParityTest.cs | 4 +- .../ApiParity/DataStorageMetricsParityTest.cs | 5 +- 14 files changed, 318 insertions(+), 30 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/DataRegionMetrics.java b/modules/core/src/main/java/org/apache/ignite/DataRegionMetrics.java index ca2fc66818ee1..88dcd168e0104 100644 --- a/modules/core/src/main/java/org/apache/ignite/DataRegionMetrics.java +++ b/modules/core/src/main/java/org/apache/ignite/DataRegionMetrics.java @@ -146,11 +146,18 @@ public interface DataRegionMetrics { public long getPhysicalMemorySize(); /** - * Gets checkpoint buffer size in pages. + * Gets used checkpoint buffer size in pages. * * @return Checkpoint buffer size in pages. */ - public long getCheckpointBufferPages(); + public long getUsedCheckpointBufferPages(); + + /** + * Gets used checkpoint buffer size in bytes. + * + * @return Checkpoint buffer size in bytes. + */ + public long getUsedCheckpointBufferSize(); /** * Gets checkpoint buffer size in bytes. diff --git a/modules/core/src/main/java/org/apache/ignite/DataStorageMetrics.java b/modules/core/src/main/java/org/apache/ignite/DataStorageMetrics.java index 5fb2b1e088700..cdde0aced29d8 100644 --- a/modules/core/src/main/java/org/apache/ignite/DataStorageMetrics.java +++ b/modules/core/src/main/java/org/apache/ignite/DataStorageMetrics.java @@ -191,4 +191,25 @@ public interface DataStorageMetrics { * @return Total size of memory allocated in bytes. */ public long getTotalAllocatedSize(); + + /** + * Gets used checkpoint buffer size in pages. + * + * @return Checkpoint buffer size in pages. + */ + public long getUsedCheckpointBufferPages(); + + /** + * Gets used checkpoint buffer size in bytes. + * + * @return Checkpoint buffer size in bytes. + */ + public long getUsedCheckpointBufferSize(); + + /** + * Checkpoint buffer size in bytes. + * + * @return Checkpoint buffer size in bytes. + */ + public long getCheckpointBufferSize(); } 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 cb13747b775a6..a82f73bd7966f 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 @@ -61,6 +61,9 @@ public class DataRegionMetricsImpl implements DataRegionMetrics, AllocatedPageTr /** */ private final AtomicLong offHeapSize = new AtomicLong(); + /** */ + private final AtomicLong checkpointBufferSize = new AtomicLong(); + /** */ private volatile boolean metricsEnabled; @@ -129,7 +132,7 @@ public DataRegionMetricsImpl(DataRegionConfiguration memPlcCfg, @Nullable Ignite @Override public long getTotalAllocatedSize() { assert pageMem != null; - return getTotalAllocatedPages() * pageMem.pageSize(); + return getTotalAllocatedPages() * (persistenceEnabled ? pageMem.pageSize() : pageMem.systemPageSize()); } /** {@inheritDoc} */ @@ -211,12 +214,12 @@ public DataRegionMetricsImpl(DataRegionConfiguration memPlcCfg, @Nullable Ignite /** {@inheritDoc} */ @Override public long getPhysicalMemorySize() { - return getPhysicalMemoryPages() * pageMem.pageSize(); + return getPhysicalMemoryPages() * pageMem.systemPageSize(); } /** {@inheritDoc} */ - @Override public long getCheckpointBufferPages() { - if (!metricsEnabled) + @Override public long getUsedCheckpointBufferPages() { + if (!metricsEnabled || !persistenceEnabled) return 0; assert pageMem != null; @@ -224,9 +227,17 @@ public DataRegionMetricsImpl(DataRegionConfiguration memPlcCfg, @Nullable Ignite return pageMem.checkpointBufferPagesCount(); } + /** {@inheritDoc} */ + @Override public long getUsedCheckpointBufferSize() { + return getUsedCheckpointBufferPages() * pageMem.systemPageSize(); + } + /** {@inheritDoc} */ @Override public long getCheckpointBufferSize() { - return getCheckpointBufferPages() * pageMem.pageSize(); + if (!metricsEnabled || !persistenceEnabled) + return 0; + + return checkpointBufferSize.get(); } /** {@inheritDoc} */ @@ -276,17 +287,23 @@ public DataRegionMetricsImpl(DataRegionConfiguration memPlcCfg, @Nullable Ignite if (!metricsEnabled) return 0; - return pageMem.loadedPages() * pageMem.pageSize(); + return pageMem.loadedPages() * pageMem.systemPageSize(); } /** - * * @param size Region size. */ public void updateOffHeapSize(long size) { this.offHeapSize.addAndGet(size); } + /** + * @param size Checkpoint buffer size. + */ + public void updateCheckpointBufferSize(long size) { + this.checkpointBufferSize.addAndGet(size); + } + /** * Updates pageReplaceRate metric. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsMXBeanImpl.java index 9ab682487512c..f83716816e53b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsMXBeanImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsMXBeanImpl.java @@ -97,8 +97,13 @@ class DataRegionMetricsMXBeanImpl implements DataRegionMetricsMXBean { } /** {@inheritDoc} */ - @Override public long getCheckpointBufferPages() { - return memMetrics.getCheckpointBufferPages(); + @Override public long getUsedCheckpointBufferPages() { + return memMetrics.getUsedCheckpointBufferPages(); + } + + /** {@inheritDoc} */ + @Override public long getUsedCheckpointBufferSize() { + return memMetrics.getUsedCheckpointBufferSize(); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsSnapshot.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsSnapshot.java index 6126c4baa5c1f..f119419ef8eaa 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsSnapshot.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsSnapshot.java @@ -60,7 +60,10 @@ public class DataRegionMetricsSnapshot implements DataRegionMetrics { private long physicalMemorySize; /** */ - private long checkpointBufferPages; + private long usedCheckpointBufferPages; + + /** */ + private long usedCheckpointBufferSize; /** */ private long checkpointBufferSize; @@ -99,7 +102,8 @@ public DataRegionMetricsSnapshot(DataRegionMetrics metrics) { pageReplaceAge = metrics.getPagesReplaceAge(); physicalMemoryPages = metrics.getPhysicalMemoryPages(); physicalMemorySize = metrics.getPhysicalMemorySize(); - checkpointBufferPages = metrics.getCheckpointBufferPages(); + usedCheckpointBufferPages = metrics.getUsedCheckpointBufferPages(); + usedCheckpointBufferSize = metrics.getUsedCheckpointBufferSize(); checkpointBufferSize = metrics.getCheckpointBufferSize(); pageSize = metrics.getPageSize(); readPages = metrics.getPagesRead(); @@ -170,8 +174,13 @@ public DataRegionMetricsSnapshot(DataRegionMetrics metrics) { } /** {@inheritDoc} */ - @Override public long getCheckpointBufferPages() { - return checkpointBufferPages; + @Override public long getUsedCheckpointBufferPages() { + return usedCheckpointBufferPages; + } + + /** {@inheritDoc} */ + @Override public long getUsedCheckpointBufferSize() { + return usedCheckpointBufferSize; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsImpl.java index b7f99e5038924..03955a4b799cf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsImpl.java @@ -396,6 +396,60 @@ public DataStorageMetricsImpl( return totalAllocatedSize; } + /** {@inheritDoc} */ + @Override public long getUsedCheckpointBufferPages() { + if (!metricsEnabled) + return 0; + + Collection regionMetrics0 = regionMetrics; + + if (F.isEmpty(regionMetrics0)) + return 0; + + long usedCheckpointBufferPages = 0L; + + for (DataRegionMetrics rm : regionMetrics0) + usedCheckpointBufferPages += rm.getUsedCheckpointBufferPages(); + + return usedCheckpointBufferPages; + } + + /** {@inheritDoc} */ + @Override public long getUsedCheckpointBufferSize() { + if (!metricsEnabled) + return 0; + + Collection regionMetrics0 = regionMetrics; + + if (F.isEmpty(regionMetrics0)) + return 0; + + long usedCheckpointBufferSize = 0L; + + for (DataRegionMetrics rm : regionMetrics0) + usedCheckpointBufferSize += rm.getUsedCheckpointBufferSize(); + + return usedCheckpointBufferSize; + } + + /** {@inheritDoc} */ + @Override public long getCheckpointBufferSize(){ + if (!metricsEnabled) + return 0; + + Collection regionMetrics0 = regionMetrics; + + if (F.isEmpty(regionMetrics0)) + return 0; + + long checkpointBufferSize = 0L; + + for (DataRegionMetrics rm : regionMetrics0) + checkpointBufferSize += rm.getCheckpointBufferSize(); + + return checkpointBufferSize; + } + /** * @param wal Write-ahead log manager. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsSnapshot.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsSnapshot.java index cb522791652c2..c3bcd5b9d6df9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsSnapshot.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsSnapshot.java @@ -69,7 +69,16 @@ public class DataStorageMetricsSnapshot implements DataStorageMetrics { private long walLastRollOverTime; /** */ - private long checkpointTotalSize; + private long checkpointTotalTime; + + /** */ + private long usedCheckpointBufferSize; + + /** */ + private long usedCheckpointBufferPages; + + /** */ + private long checkpointBufferSize; /** */ private long dirtyPages; @@ -111,7 +120,10 @@ public DataStorageMetricsSnapshot(DataStorageMetrics metrics) { lastCpCowPages = metrics.getLastCheckpointCopiedOnWritePagesNumber(); walTotalSize = metrics.getWalTotalSize(); walLastRollOverTime = metrics.getWalLastRollOverTime(); - checkpointTotalSize = metrics.getCheckpointTotalTime(); + checkpointTotalTime = metrics.getCheckpointTotalTime(); + usedCheckpointBufferSize = metrics.getUsedCheckpointBufferSize(); + usedCheckpointBufferPages = metrics.getUsedCheckpointBufferPages(); + checkpointBufferSize = metrics.getCheckpointBufferSize(); dirtyPages = metrics.getDirtyPages(); readPages = metrics.getPagesRead(); writtenPages = metrics.getPagesWritten(); @@ -198,7 +210,7 @@ public DataStorageMetricsSnapshot(DataStorageMetrics metrics) { /** {@inheritDoc} */ @Override public long getCheckpointTotalTime() { - return checkpointTotalSize; + return checkpointTotalTime; } /** {@inheritDoc} */ @@ -236,6 +248,21 @@ public DataStorageMetricsSnapshot(DataStorageMetrics metrics) { return totalAllocatedSize; } + /** {@inheritDoc} */ + @Override public long getUsedCheckpointBufferPages() { + return usedCheckpointBufferPages; + } + + /** {@inheritDoc} */ + @Override public long getUsedCheckpointBufferSize() { + return usedCheckpointBufferSize; + } + + /** {@inheritDoc} */ + @Override public long getCheckpointBufferSize(){ + return checkpointBufferSize; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(DataStorageMetricsSnapshot.class, this); 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 ffa7259669775..6ce743d8d9632 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 @@ -84,6 +84,7 @@ import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; import org.apache.ignite.internal.mem.DirectMemoryProvider; +import org.apache.ignite.internal.mem.DirectMemoryRegion; import org.apache.ignite.internal.mem.file.MappedFileMemoryProvider; import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider; import org.apache.ignite.internal.pagemem.FullPageId; @@ -1019,6 +1020,51 @@ private long[] calculateFragmentSizes(int concLvl, long cacheSize, long chpBufSi return pageMem; } + /** + * @param memoryProvider0 Memory provider. + * @param memMetrics Memory metrics. + * @return Wrapped memory provider. + */ + @Override protected DirectMemoryProvider wrapMetricsMemoryProvider( + final DirectMemoryProvider memoryProvider0, + final DataRegionMetricsImpl memMetrics + ) { + return new DirectMemoryProvider() { + private AtomicInteger checkPointBufferIdxCnt = new AtomicInteger(); + + private final DirectMemoryProvider memProvider = memoryProvider0; + + @Override public void initialize(long[] chunkSizes) { + memProvider.initialize(chunkSizes); + + checkPointBufferIdxCnt.set(chunkSizes.length); + } + + @Override public void shutdown() { + memProvider.shutdown(); + } + + @Override public DirectMemoryRegion nextRegion() { + DirectMemoryRegion nextMemoryRegion = memProvider.nextRegion(); + + if (nextMemoryRegion == null) + return null; + + int idx = checkPointBufferIdxCnt.decrementAndGet(); + + long chunkSize = nextMemoryRegion.size(); + + // Checkpoint chunk last in the long[] chunkSizes. + if (idx != 0) + memMetrics.updateOffHeapSize(chunkSize); + else + memMetrics.updateCheckpointBufferSize(chunkSize); + + return nextMemoryRegion; + } + }; + } + /** * Resolves throttling policy according to the settings. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java index f95e69b985e1c..d7a6b565580c4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java @@ -560,8 +560,8 @@ private static void writeDataRegionMetrics(BinaryRawWriter writer, DataRegionMet writer.writeFloat(metrics.getPagesReplaceAge()); writer.writeLong(metrics.getPhysicalMemoryPages()); writer.writeLong(metrics.getPhysicalMemorySize()); - writer.writeLong(metrics.getCheckpointBufferPages()); - writer.writeLong(metrics.getCheckpointBufferSize()); + writer.writeLong(metrics.getUsedCheckpointBufferPages()); + writer.writeLong(metrics.getUsedCheckpointBufferSize()); writer.writeInt(metrics.getPageSize()); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorMemoryMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorMemoryMetrics.java index c19fd3607afd8..5b46220d8f7df 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorMemoryMetrics.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorMemoryMetrics.java @@ -96,8 +96,8 @@ public VisorMemoryMetrics(DataRegionMetrics m) { physicalMemoryPages = m.getPhysicalMemoryPages(); totalAllocatedSz = m.getTotalAllocatedSize(); physicalMemSz = m.getPhysicalMemorySize(); - cpBufPages = m.getCheckpointBufferPages(); - cpBufSz = m.getCheckpointBufferSize(); + cpBufPages = m.getUsedCheckpointBufferPages(); + cpBufSz = m.getUsedCheckpointBufferSize(); pageSize = m.getPageSize(); } diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/DataStorageMetricsMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/DataStorageMetricsMXBean.java index 2450874e64df9..2069099957747 100644 --- a/modules/core/src/main/java/org/apache/ignite/mxbean/DataStorageMetricsMXBean.java +++ b/modules/core/src/main/java/org/apache/ignite/mxbean/DataStorageMetricsMXBean.java @@ -56,6 +56,18 @@ public interface DataStorageMetricsMXBean extends DataStorageMetrics { @MXBeanDescription("Total checkpoint time from last restart.") @Override long getCheckpointTotalTime(); + /** {@inheritDoc} */ + @MXBeanDescription("Used checkpoint buffer size in pages.") + @Override long getUsedCheckpointBufferPages(); + + /** {@inheritDoc} */ + @MXBeanDescription("Used checkpoint buffer size in bytes.") + @Override long getUsedCheckpointBufferSize(); + + /** {@inheritDoc} */ + @MXBeanDescription("Total size in bytes for checkpoint buffer.") + @Override long getCheckpointBufferSize(); + /** {@inheritDoc} */ @MXBeanDescription("Duration of the last checkpoint in milliseconds.") @Override long getLastCheckpointDuration(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java index 4b7d6aeb8b7c1..18a47814755d1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java @@ -17,10 +17,11 @@ package org.apache.ignite.internal.processors.cache.persistence.db; -import java.util.HashMap; import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.Random; +import java.util.HashMap; import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; import org.apache.ignite.DataRegionMetrics; @@ -32,14 +33,20 @@ 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.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.persistence.DataRegion; +import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl; +import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; +import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.PA; +import org.apache.ignite.internal.util.typedef.T2; 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.junit.Assert; import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_DATA_REG_DEFAULT_NAME; @@ -53,6 +60,9 @@ public class IgnitePdsDataRegionMetricsTest extends GridCommonAbstractTest { /** */ private static final long INIT_REGION_SIZE = 10 << 20; + /** */ + private static final long MAX_REGION_SIZE = INIT_REGION_SIZE * 10; + /** */ private static final int ITERATIONS = 3; @@ -72,11 +82,13 @@ public class IgnitePdsDataRegionMetricsTest extends GridCommonAbstractTest { ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER); DataStorageConfiguration memCfg = new DataStorageConfiguration() - .setDefaultDataRegionConfiguration( - new DataRegionConfiguration() - .setInitialSize(INIT_REGION_SIZE) - .setPersistenceEnabled(true) - .setMetricsEnabled(true)); + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration() + .setInitialSize(INIT_REGION_SIZE) + .setMaxSize(MAX_REGION_SIZE) + .setPersistenceEnabled(true) + .setMetricsEnabled(true)) + .setCheckpointFrequency(1000); cfg.setDataStorageConfiguration(memCfg); @@ -194,6 +206,79 @@ public void testMemoryUsageMultipleNodes() throws Exception { checkMetricsConsistency(node2, grpIds); } + /** + * Test for check checkpoint size metric. + * + * @throws Exception If failed. + */ + public void testCheckpointBufferSize() throws Exception { + IgniteEx ig = startGrid(0); + + ig.cluster().active(true); + + DataRegionMetricsImpl regionMetrics = ig.cachex(DEFAULT_CACHE_NAME) + .context().group().dataRegion().memoryMetrics(); + + Assert.assertTrue(regionMetrics.getCheckpointBufferSize() != 0); + Assert.assertTrue(regionMetrics.getCheckpointBufferSize() <= MAX_REGION_SIZE); + } + + /** + * Test for check used checkpoint size metric. + * + * @throws Exception If failed. + */ + public void testUsedCheckpointBuffer() throws Exception { + IgniteEx ig = startGrid(0); + + ig.cluster().active(true); + + final DataRegionMetricsImpl regionMetrics = ig.cachex(DEFAULT_CACHE_NAME) + .context().group().dataRegion().memoryMetrics(); + + Assert.assertEquals(0, regionMetrics.getUsedCheckpointBufferPages()); + Assert.assertEquals(0, regionMetrics.getUsedCheckpointBufferSize()); + + load(ig); + + GridCacheDatabaseSharedManager psMgr = (GridCacheDatabaseSharedManager) ig.context().cache().context().database(); + + GridFutureAdapter> metricsResult = new GridFutureAdapter<>(); + + IgniteInternalFuture chpBeginFut = psMgr.wakeupForCheckpoint(null); + + chpBeginFut.listen((f) -> { + load(ig); + + metricsResult.onDone(new T2<>( + regionMetrics.getUsedCheckpointBufferPages(), + regionMetrics.getUsedCheckpointBufferSize() + )); + }); + + metricsResult.get(); + + Assert.assertTrue(metricsResult.get().get1() > 0); + Assert.assertTrue(metricsResult.get().get2() > 0); + } + + /** + * @param ig Ignite. + */ + private void load(Ignite ig){ + IgniteCache cache = ig.cache(DEFAULT_CACHE_NAME); + + Random rnd = new Random(); + + for (int i = 0; i < 1000; i++) { + byte[] payload = new byte[128]; + + rnd.nextBytes(payload); + + cache.put(i, payload); + } + } + /** */ private static DataRegionMetrics getDfltRegionMetrics(Ignite node) { for (DataRegionMetrics m : node.dataRegionMetrics()) diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/DataRegionMetricsParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/DataRegionMetricsParityTest.cs index 22b8986f812f8..463eaa4b8b204 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/DataRegionMetricsParityTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/DataRegionMetricsParityTest.cs @@ -39,7 +39,9 @@ public class DataRegionMetricsParityTest "PagesWritten", "PagesReplaced", "OffHeapSize", - "OffheapUsedSize" + "OffheapUsedSize", + "UsedCheckpointBufferPages", + "UsedCheckpointBufferSize" }; /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/DataStorageMetricsParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/DataStorageMetricsParityTest.cs index 300ab25c3f433..58c974ec57ea1 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/DataStorageMetricsParityTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/DataStorageMetricsParityTest.cs @@ -38,7 +38,10 @@ public class DataStorageMetricsParityTest "PagesReplaced", "OffHeapSize", "OffheapUsedSize", - "TotalAllocatedSize" + "TotalAllocatedSize", + "UsedCheckpointBufferPages", + "UsedCheckpointBufferSize", + "CheckpointBufferSize" }; /// From c18552f9665fd8b571153a265d3da5634a7e2f04 Mon Sep 17 00:00:00 2001 From: Sergey Chugunov Date: Thu, 24 May 2018 18:11:47 +0300 Subject: [PATCH 0196/1463] IGNITE-8560 Update index validation utility to use statistical check approach - Fixes #4051. Signed-off-by: Ivan Rakov (cherry-picked from commit #76e1fe754d7a8bd059d7ab64d17cbefa4913a702) --- .../internal/commandline/CommandHandler.java | 92 +++++++++++++++---- .../commandline/cache/CacheArguments.java | 34 +++++++ .../verify/VisorValidateIndexesTaskArg.java | 40 +++++++- .../CommandHandlerParsingTest.java | 87 ++++++++++++++++++ .../visor/verify/ValidateIndexesClosure.java | 66 ++++++++++++- .../verify/VisorValidateIndexesTask.java | 2 +- .../util/GridCommandHandlerIndexingTest.java | 8 +- 7 files changed, 305 insertions(+), 24 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java index 47cc233f61ebc..c59e348f768ae 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java @@ -181,6 +181,12 @@ public class CommandHandler { /** */ private static final String BASELINE_SET_VERSION = "version"; + /** Parameter name for validate_indexes command. */ + static final String VI_CHECK_FIRST = "checkFirst"; + + /** Parameter name for validate_indexes command. */ + static final String VI_CHECK_THROUGH = "checkThrough"; + /** */ static final String WAL_PRINT = "print"; @@ -583,10 +589,12 @@ private void printCacheHelp() { usage(" Show information about caches, groups or sequences that match a regex:", CACHE, " list regexPattern [groups|seq] [nodeId]"); usage(" Show hot keys that are point of contention for multiple transactions:", CACHE, " contention minQueueSize [nodeId] [maxPrint]"); usage(" Verify partition counters and hashes between primary and backups on idle cluster:", CACHE, " idle_verify [cache1,...,cacheN]"); - usage(" Validate custom indexes on idle cluster:", CACHE, " validate_indexes [cache1,...,cacheN] [nodeId]"); + usage(" Validate custom indexes on idle cluster:", CACHE, " validate_indexes [cache1,...,cacheN] [nodeId] [checkFirst|checkThrough]"); - log(" If [nodeId] is not specified, cont and validate_indexes commands will be broadcasted to all server nodes."); + log(" If [nodeId] is not specified, contention and validate_indexes commands will be broadcasted to all server nodes."); log(" Another commands where [nodeId] is optional will run on a random server node."); + log(" checkFirst numeric parameter for validate_indexes specifies number of first K keys to be validated."); + log(" checkThrough numeric parameter for validate_indexes allows to check each Kth key."); nl(); } @@ -624,7 +632,11 @@ private void cacheContention(GridClient client, CacheArguments cacheArgs) throws * @param cacheArgs Cache args. */ private void cacheValidateIndexes(GridClient client, CacheArguments cacheArgs) throws GridClientException { - VisorValidateIndexesTaskArg taskArg = new VisorValidateIndexesTaskArg(cacheArgs.caches()); + VisorValidateIndexesTaskArg taskArg = new VisorValidateIndexesTaskArg( + cacheArgs.caches(), + cacheArgs.checkFirst(), + cacheArgs.checkThrough() + ); UUID nodeId = cacheArgs.nodeId() == null ? BROADCAST_UUID : cacheArgs.nodeId(); @@ -1407,7 +1419,8 @@ private CacheArguments parseAndValidateCacheArgs() { break; case IDLE_VERIFY: - parseCacheNamesIfPresent(cacheArgs); + if (hasNextCacheArg()) + parseCacheNames(nextArg(""), cacheArgs); break; @@ -1425,10 +1438,53 @@ private CacheArguments parseAndValidateCacheArgs() { break; case VALIDATE_INDEXES: - parseCacheNamesIfPresent(cacheArgs); + int argsCnt = 0; - if (hasNextCacheArg()) - cacheArgs.nodeId(UUID.fromString(nextArg(""))); + while (hasNextCacheArg() && argsCnt++ < 4) { + String arg = nextArg(""); + + if (VI_CHECK_FIRST.equals(arg) || VI_CHECK_THROUGH.equals(arg)) { + if (!hasNextCacheArg()) + throw new IllegalArgumentException("Numeric value for '" + arg + "' parameter expected."); + + int numVal; + + String numStr = nextArg(""); + + try { + numVal = Integer.parseInt(numStr); + } + catch (IllegalArgumentException e) { + throw new IllegalArgumentException( + "Not numeric value was passed for '" + + arg + + "' parameter: " + + numStr + ); + } + + if (numVal <= 0) + throw new IllegalArgumentException("Value for '" + arg + "' property should be positive."); + + if (VI_CHECK_FIRST.equals(arg)) + cacheArgs.checkFirst(numVal); + else + cacheArgs.checkThrough(numVal); + + continue; + } + + try { + cacheArgs.nodeId(UUID.fromString(arg)); + + continue; + } + catch (IllegalArgumentException ignored) { + //No-op. + } + + parseCacheNames(arg, cacheArgs); + } break; @@ -1473,22 +1529,18 @@ private boolean hasNextCacheArg() { /** * @param cacheArgs Cache args. */ - private void parseCacheNamesIfPresent(CacheArguments cacheArgs) { - if (hasNextCacheArg()) { - String cacheNames = nextArg(""); - - String[] cacheNamesArr = cacheNames.split(","); - Set cacheNamesSet = new HashSet<>(); + private void parseCacheNames(String cacheNames, CacheArguments cacheArgs) { + String[] cacheNamesArr = cacheNames.split(","); + Set cacheNamesSet = new HashSet<>(); - for (String cacheName : cacheNamesArr) { - if (F.isEmpty(cacheName)) - throw new IllegalArgumentException("Non-empty cache names expected."); + for (String cacheName : cacheNamesArr) { + if (F.isEmpty(cacheName)) + throw new IllegalArgumentException("Non-empty cache names expected."); - cacheNamesSet.add(cacheName.trim()); - } - - cacheArgs.caches(cacheNamesSet); + cacheNamesSet.add(cacheName.trim()); } + + cacheArgs.caches(cacheNamesSet); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheArguments.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheArguments.java index 6f315efe6a732..1411b2a2352ce 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheArguments.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/cache/CacheArguments.java @@ -46,6 +46,12 @@ public class CacheArguments { /** Max print. */ private int maxPrint; + /** validate_indexes 'checkFirst' argument */ + private int checkFirst = -1; + + /** validate_indexes 'checkThrough' argument */ + private int checkThrough = -1; + /** Cache view command. */ private @Nullable VisorViewCacheCmd cacheCmd; @@ -160,4 +166,32 @@ public int maxPrint() { public void maxPrint(int maxPrint) { this.maxPrint = maxPrint; } + + /** + * @return Max number of entries to be checked. + */ + public int checkFirst() { + return checkFirst; + } + + /** + * @param checkFirst Max number of entries to be checked. + */ + public void checkFirst(int checkFirst) { + this.checkFirst = checkFirst; + } + + /** + * @return Number of entries to check through. + */ + public int checkThrough() { + return checkThrough; + } + + /** + * @param checkThrough Number of entries to check through. + */ + public void checkThrough(int checkThrough) { + this.checkThrough = checkThrough; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTaskArg.java index cf9aff5ac2033..aa49977c7a6fa 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTaskArg.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTaskArg.java @@ -35,6 +35,12 @@ public class VisorValidateIndexesTaskArg extends VisorDataTransferObject { /** Caches. */ private Set caches; + /** Check first K elements. */ + private int checkFirst; + + /** Check through K element (skip K-1, check Kth). */ + private int checkThrough; + /** * Default constructor. */ @@ -45,8 +51,10 @@ public VisorValidateIndexesTaskArg() { /** * @param caches Caches. */ - public VisorValidateIndexesTaskArg(Set caches) { + public VisorValidateIndexesTaskArg(Set caches, int checkFirst, int checkThrough) { this.caches = caches; + this.checkFirst = checkFirst; + this.checkThrough = checkThrough; } @@ -57,14 +65,44 @@ public Set getCaches() { return caches; } + /** + * @return checkFirst. + */ + public int getCheckFirst() { + return checkFirst; + } + + /** + * @return checkThrough. + */ + public int getCheckThrough() { + return checkThrough; + } + /** {@inheritDoc} */ @Override protected void writeExternalData(ObjectOutput out) throws IOException { U.writeCollection(out, caches); + out.writeInt(checkFirst); + out.writeInt(checkThrough); } /** {@inheritDoc} */ @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { caches = U.readSet(in); + + if (protoVer > V1) { + checkFirst = in.readInt(); + checkThrough = in.readInt(); + } + else { + checkFirst = -1; + checkThrough = -1; + } + } + + /** {@inheritDoc} */ + @Override public byte getProtocolVersion() { + return V2; } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java index 2fc40ca167944..737c0c736f7ae 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/commandline/CommandHandlerParsingTest.java @@ -21,15 +21,20 @@ import java.util.Collections; import java.util.UUID; import junit.framework.TestCase; +import org.apache.ignite.internal.commandline.cache.CacheArguments; +import org.apache.ignite.internal.commandline.cache.CacheCommand; import org.apache.ignite.internal.visor.tx.VisorTxProjection; import org.apache.ignite.internal.visor.tx.VisorTxSortOrder; import org.apache.ignite.internal.visor.tx.VisorTxTaskArg; import static java.util.Arrays.asList; import static org.apache.ignite.IgniteSystemProperties.IGNITE_ENABLE_EXPERIMENTAL_COMMAND; +import static org.apache.ignite.internal.commandline.Command.CACHE; import static org.apache.ignite.internal.commandline.Command.WAL; import static org.apache.ignite.internal.commandline.CommandHandler.DFLT_HOST; import static org.apache.ignite.internal.commandline.CommandHandler.DFLT_PORT; +import static org.apache.ignite.internal.commandline.CommandHandler.VI_CHECK_FIRST; +import static org.apache.ignite.internal.commandline.CommandHandler.VI_CHECK_THROUGH; import static org.apache.ignite.internal.commandline.CommandHandler.WAL_DELETE; import static org.apache.ignite.internal.commandline.CommandHandler.WAL_PRINT; @@ -51,6 +56,88 @@ public class CommandHandlerParsingTest extends TestCase { super.tearDown(); } + /** + * validate_indexes command arguments parsing and validation + */ + public void testValidateIndexArguments() { + CommandHandler hnd = new CommandHandler(); + + //happy case for all parameters + try { + int expectedCheckFirst = 10; + int expectedCheckThrough = 11; + UUID nodeId = UUID.randomUUID(); + + CacheArguments args = hnd.parseAndValidate( + Arrays.asList( + CACHE.text(), + CacheCommand.VALIDATE_INDEXES.text(), + "cache1, cache2", + nodeId.toString(), + VI_CHECK_FIRST, + Integer.toString(expectedCheckFirst), + VI_CHECK_THROUGH, + Integer.toString(expectedCheckThrough) + ) + ).cacheArgs(); + + assertEquals("nodeId parameter unexpected value", nodeId, args.nodeId()); + assertEquals("checkFirst parameter unexpected value", expectedCheckFirst, args.checkFirst()); + assertEquals("checkThrough parameter unexpected value", expectedCheckThrough, args.checkThrough()); + } + catch (IllegalArgumentException e) { + fail("Unexpected exception: " + e); + } + + try { + int expectedParam = 11; + UUID nodeId = UUID.randomUUID(); + + CacheArguments args = hnd.parseAndValidate( + Arrays.asList( + CACHE.text(), + CacheCommand.VALIDATE_INDEXES.text(), + nodeId.toString(), + VI_CHECK_THROUGH, + Integer.toString(expectedParam) + ) + ).cacheArgs(); + + assertNull("caches weren't specified, null value expected", args.caches()); + assertEquals("nodeId parameter unexpected value", nodeId, args.nodeId()); + assertEquals("checkFirst parameter unexpected value", -1, args.checkFirst()); + assertEquals("checkThrough parameter unexpected value", expectedParam, args.checkThrough()); + } + catch (IllegalArgumentException e) { + e.printStackTrace(); + } + + try { + hnd.parseAndValidate( + Arrays.asList( + CACHE.text(), + CacheCommand.VALIDATE_INDEXES.text(), + VI_CHECK_FIRST, + "0" + ) + ); + + fail("Expected exception hasn't been thrown"); + } + catch (IllegalArgumentException e) { + e.printStackTrace(); + } + + try { + hnd.parseAndValidate(Arrays.asList(CACHE.text(), CacheCommand.VALIDATE_INDEXES.text(), VI_CHECK_THROUGH)); + + fail("Expected exception hasn't been thrown"); + } + catch (IllegalArgumentException e) { + e.printStackTrace(); + } + } + /** * Test that experimental command (i.e. WAL command) is disabled by default. */ diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java index e0eff612fad2a..e01dca2c92d8f 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java @@ -89,6 +89,12 @@ public class ValidateIndexesClosure implements IgniteCallable cacheNames; + /** If provided only first K elements will be validated. */ + private final int checkFirst; + + /** If provided only each Kth element will be validated. */ + private final int checkThrough; + /** Counter of processed partitions. */ private final AtomicInteger processedPartitions = new AtomicInteger(0); @@ -109,9 +115,13 @@ public class ValidateIndexesClosure implements IgniteCallable cacheNames) { + public ValidateIndexesClosure(Set cacheNames, int checkFirst, int checkThrough) { this.cacheNames = cacheNames; + this.checkFirst = checkFirst; + this.checkThrough = checkThrough; } /** {@inheritDoc} */ @@ -320,12 +330,39 @@ private Map processPartition( m.setAccessible(true); + final boolean skipConditions = checkFirst > 0 || checkThrough > 0; + final boolean bothSkipConditions = checkFirst > 0 && checkThrough > 0; + + long current = 0; + long processedNumber = 0; + while (it.hasNextX()) { if (enoughIssues) break; CacheDataRow row = it.nextX(); + if (skipConditions) { + if (bothSkipConditions) { + if (processedNumber > checkFirst) + break; + else if (current++ % checkThrough > 0) + continue; + else + processedNumber++; + } + else { + if (checkFirst > 0) { + if (current++ > checkFirst) + break; + } + else { + if (current++ % checkThrough > 0) + continue; + } + } + } + int cacheId = row.cacheId() == 0 ? grpCtx.groupId() : row.cacheId(); GridCacheContext cacheCtx = row.cacheId() == 0 ? @@ -462,6 +499,12 @@ private Map processIndex(GridCacheContex enoughIssues = true; } + final boolean skipConditions = checkFirst > 0 || checkThrough > 0; + final boolean bothSkipConditions = checkFirst > 0 && checkThrough > 0; + + long current = 0; + long processedNumber = 0; + while (!enoughIssues) { KeyCacheObject h2key = null; @@ -471,6 +514,27 @@ private Map processIndex(GridCacheContex GridH2Row h2Row = (GridH2Row)cursor.get(); + if (skipConditions) { + if (bothSkipConditions) { + if (processedNumber > checkFirst) + break; + else if (current++ % checkThrough > 0) + continue; + else + processedNumber++; + } + else { + if (checkFirst > 0) { + if (current++ > checkFirst) + break; + } + else { + if (current++ % checkThrough > 0) + continue; + } + } + } + h2key = h2Row.key(); CacheDataRow cacheDataStoreRow = ctx.group().offheap().read(ctx, h2key); diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTask.java b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTask.java index 52b48a58c0340..abb7f7ee55ec1 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTask.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/VisorValidateIndexesTask.java @@ -76,7 +76,7 @@ protected VisorValidateIndexesJob(@Nullable VisorValidateIndexesTaskArg arg, boo /** {@inheritDoc} */ @Override protected VisorValidateIndexesJobResult run(@Nullable VisorValidateIndexesTaskArg arg) throws IgniteException { try { - ValidateIndexesClosure clo = new ValidateIndexesClosure(arg.getCaches()); + ValidateIndexesClosure clo = new ValidateIndexesClosure(arg.getCaches(), arg.getCheckFirst(), arg.getCheckThrough()); ignite.context().resource().injectGeneric(clo); diff --git a/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java b/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java index 62d3fc02bdbaa..ca9aa5372f552 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java @@ -101,7 +101,13 @@ public void testBrokenCacheDataTreeShouldFailValidation() throws Exception { injectTestSystemOut(); - assertEquals(EXIT_CODE_OK, execute("--cache", "validate_indexes", cacheName)); + assertEquals(EXIT_CODE_OK, + execute( + "--cache", + "validate_indexes", + cacheName, + "checkFirst", "10000", + "checkThrough", "10")); assertTrue(testOut.toString().contains("validate_indexes has finished with errors")); } From 2f2560e26ce884a09cbafafdc5bda2601822c3f3 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 6 Apr 2018 10:35:17 +0300 Subject: [PATCH 0197/1463] IGNITE-7933 Checkpoing file markers should be written atomically - Fixes #3633. Signed-off-by: Alexey Goncharuk (cherry-picked from commit #4a0695ceae2f99c4841e8382e723daff4580ea3d) --- .../apache/ignite/internal/IgnitionEx.java | 2 +- .../GridCacheDatabaseSharedManager.java | 145 +++++++---- .../cache/persistence/file/AsyncFileIO.java | 9 +- .../cache/persistence/file/FileIO.java | 20 +- .../persistence/file/FileIODecorator.java | 9 +- .../persistence/file/RandomAccessFileIO.java | 13 +- .../cache/persistence/file/UnzipFileIO.java | 7 +- .../IgnitePdsDiskErrorsRecoveringTest.java | 231 ++++++++++++++---- .../db/wal/IgniteWalFlushFailoverTest.java | 4 +- ...lushMultiNodeFailoverAbstractSelfTest.java | 4 +- .../pagemem/PagesWriteThrottleSmokeTest.java | 4 +- .../file/AlignedBuffersDirectFileIO.java | 7 +- 12 files changed, 353 insertions(+), 102 deletions(-) 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 e140609aac497..b3c3ee8b5a9b1 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 @@ -414,7 +414,7 @@ public void run() { " milliseconds. Killing node..."); // We are not able to kill only one grid so whole JVM will be stopped. - System.exit(Ignition.KILL_EXIT_CODE); + Runtime.getRuntime().halt(Ignition.KILL_EXIT_CODE); } } }, timeoutMs, TimeUnit.MILLISECONDS); 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 6ce743d8d9632..4096a8d7ba13f 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 @@ -27,6 +27,7 @@ import java.nio.channels.FileChannel; import java.nio.channels.FileLock; import java.nio.channels.OverlappingFileLockException; +import java.nio.file.DirectoryStream; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -118,8 +119,10 @@ 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.persistence.file.FileIO; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStore; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; +import org.apache.ignite.internal.processors.cache.persistence.file.PersistentStorageIOException; import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage; import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetastorageLifecycleListener; import org.apache.ignite.internal.processors.cache.persistence.pagemem.CheckpointMetricsTracker; @@ -215,11 +218,14 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan /** Checkpoint file name pattern. */ private 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 patter. */ private static final Pattern NODE_STARTED_FILE_NAME_PATTERN = Pattern.compile("(\\d+)-node-started\\.bin"); /** Node started file suffix. */ - private static final String NODE_STARTED_FILE_NAME_SUFFIX = "-node-started.bin"; + public static final String NODE_STARTED_FILE_NAME_SUFFIX = "-node-started.bin"; /** */ private static final FileFilter CP_FILE_FILTER = new FileFilter() { @@ -394,6 +400,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan /** Initially local wal disabled groups. */ private Collection initiallyLocalWalDisabledGrps = new HashSet<>(); + /** File I/O factory for writing checkpoint markers. */ + private final FileIOFactory ioFactory; + /** * @param ctx Kernal context. */ @@ -418,6 +427,8 @@ public GridCacheDatabaseSharedManager(GridKernalContext ctx) { maxCpHistMemSize = Math.min(persistenceCfg.getWalHistorySize(), IgniteSystemProperties.getInteger(IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE, 100)); + + ioFactory = persistenceCfg.getFileIOFactory(); } /** */ @@ -514,6 +525,8 @@ private DataRegionConfiguration createDataRegionConfiguration(DataStorageConfigu if (!U.mkdirs(cpDir)) throw new IgniteCheckedException("Could not create directory for checkpoint metadata: " + cpDir); + cleanupCheckpointDirectory(); + final FileLockHolder preLocked = kernalCtx.pdsFolderResolver() .resolveFolders() .getLockedFileLockHolder(); @@ -527,6 +540,26 @@ private DataRegionConfiguration createDataRegionConfiguration(DataStorageConfigu } } + /** + * Cleanup checkpoint directory from all temporary files {@link #FILE_TMP_SUFFIX}. + */ + private void cleanupCheckpointDirectory() throws IgniteCheckedException { + try { + try (DirectoryStream files = Files.newDirectoryStream(cpDir.toPath(), new DirectoryStream.Filter() { + @Override + public boolean accept(Path path) throws IOException { + return path.endsWith(FILE_TMP_SUFFIX); + } + })) { + for (Path path : files) + Files.delete(path); + } + } + catch (IOException e) { + throw new IgniteCheckedException("Failed to cleanup checkpoint directory: " + cpDir, e); + } + } + /** * */ @@ -777,7 +810,7 @@ private void unRegistrateMetricsMBean() { notifyMetastorageReadyForReadWrite(); } - catch (StorageException e) { + catch (StorageException | PersistentStorageIOException e) { cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e)); throw e; @@ -788,41 +821,52 @@ private void unRegistrateMetricsMBean() { } /** + * Creates file with current timestamp and specific "node-started.bin" suffix + * and writes into memory recovery pointer. + * * @param ptr Memory recovery wal pointer. */ private void nodeStart(WALPointer ptr) throws IgniteCheckedException { FileWALPointer p = (FileWALPointer)ptr; - String fileName = U.currentTimeMillis() + "-node-started.bin"; + String fileName = U.currentTimeMillis() + NODE_STARTED_FILE_NAME_SUFFIX; + String tmpFileName = fileName + FILE_TMP_SUFFIX; ByteBuffer buf = ByteBuffer.allocate(20); buf.order(ByteOrder.nativeOrder()); - try (FileChannel ch = FileChannel.open( - Paths.get(cpDir.getAbsolutePath(), fileName), - StandardOpenOption.CREATE_NEW, StandardOpenOption.APPEND) - ) { - buf.putLong(p.index()); + try { + try (FileIO io = ioFactory.create(Paths.get(cpDir.getAbsolutePath(), tmpFileName).toFile(), + StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE)) { + buf.putLong(p.index()); - buf.putInt(p.fileOffset()); + buf.putInt(p.fileOffset()); - buf.putInt(p.length()); + buf.putInt(p.length()); - buf.flip(); + buf.flip(); - ch.write(buf); + io.write(buf); - buf.clear(); + buf.clear(); + + io.force(true); + } - ch.force(true); + Files.move(Paths.get(cpDir.getAbsolutePath(), tmpFileName), Paths.get(cpDir.getAbsolutePath(), fileName)); } catch (IOException e) { - throw new IgniteCheckedException(e); + throw new PersistentStorageIOException("Failed to write node start marker: " + ptr, e); } } /** + * Collects memory recovery pointers from node started files. See {@link #nodeStart(WALPointer)}. + * Each pointer associated with timestamp extracted from file. + * Tuples are sorted by timestamp. * + * @return Sorted list of tuples (node started timestamp, memory recovery pointer). + * @throws IgniteCheckedException */ public List> nodeStartedPointers() throws IgniteCheckedException { List> res = new ArrayList<>(); @@ -834,15 +878,10 @@ public List> nodeStartedPointers() throws IgniteCheckedExce String n1 = o1.getName(); String n2 = o2.getName(); - Long ts1 = Long.valueOf(n1.substring(0, n1.length() - NODE_STARTED_FILE_NAME_SUFFIX.length())); - Long ts2 = Long.valueOf(n2.substring(0, n2.length() - NODE_STARTED_FILE_NAME_SUFFIX.length())); + long ts1 = Long.valueOf(n1.substring(0, n1.length() - NODE_STARTED_FILE_NAME_SUFFIX.length())); + long ts2 = Long.valueOf(n2.substring(0, n2.length() - NODE_STARTED_FILE_NAME_SUFFIX.length())); - if (ts1 == ts2) - return 0; - else if (ts1 < ts2) - return -1; - else - return 1; + return Long.compare(ts1, ts2); } }); @@ -854,8 +893,8 @@ else if (ts1 < ts2) Long ts = Long.valueOf(name.substring(0, name.length() - NODE_STARTED_FILE_NAME_SUFFIX.length())); - try (FileChannel ch = FileChannel.open(f.toPath(), READ)) { - ch.read(buf); + try (FileIO io = ioFactory.create(f, READ)) { + io.read(buf); buf.flip(); @@ -1953,8 +1992,8 @@ else if (type == CheckpointEntryType.END && ts > lastEndTs) { private WALPointer readPointer(File cpMarkerFile, ByteBuffer buf) throws IgniteCheckedException { buf.position(0); - try (FileChannel ch = FileChannel.open(cpMarkerFile.toPath(), READ)) { - ch.read(buf); + try (FileIO io = ioFactory.create(cpMarkerFile, READ)) { + io.read(buf); buf.flip(); @@ -2652,6 +2691,8 @@ private void finalizeCheckpointOnRecovery(long cpTs, UUID cpId, WALPointer walPt } /** + * Writes into specified file checkpoint entry containing WAL pointer to checkpoint record. + * * @param cpId Checkpoint ID. * @param ptr Wal pointer of current checkpoint. */ @@ -2668,31 +2709,40 @@ private CheckpointEntry writeCheckpointEntry( FileWALPointer filePtr = (FileWALPointer)ptr; String fileName = checkpointFileName(cpTs, cpId, type); + String tmpFileName = fileName + FILE_TMP_SUFFIX; - try (FileChannel ch = FileChannel.open(Paths.get(cpDir.getAbsolutePath(), fileName), - StandardOpenOption.CREATE_NEW, StandardOpenOption.APPEND)) { + try { + try (FileIO io = ioFactory.create(Paths.get(cpDir.getAbsolutePath(), skipSync ? fileName : tmpFileName).toFile(), + StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE)) { - tmpWriteBuf.rewind(); + tmpWriteBuf.rewind(); - tmpWriteBuf.putLong(filePtr.index()); + tmpWriteBuf.putLong(filePtr.index()); - tmpWriteBuf.putInt(filePtr.fileOffset()); + tmpWriteBuf.putInt(filePtr.fileOffset()); - tmpWriteBuf.putInt(filePtr.length()); + tmpWriteBuf.putInt(filePtr.length()); - tmpWriteBuf.flip(); + tmpWriteBuf.flip(); - ch.write(tmpWriteBuf); + io.write(tmpWriteBuf); - tmpWriteBuf.clear(); + tmpWriteBuf.clear(); + + if (!skipSync) + io.force(true); + } if (!skipSync) - ch.force(true); + Files.move(Paths.get(cpDir.getAbsolutePath(), tmpFileName), Paths.get(cpDir.getAbsolutePath(), fileName)); return createCheckPointEntry(cpTs, ptr, cpId, rec, type); } catch (IOException e) { - throw new IgniteCheckedException(e); + throw new PersistentStorageIOException("Failed to write checkpoint entry [ptr=" + filePtr + + ", cpTs=" + cpTs + + ", cpId=" + cpId + + ", type=" + type + "]", e); } } @@ -2759,8 +2809,6 @@ private CheckpointEntry createCheckPointEntry( if (type != CheckpointEntryType.START) return null; - CheckpointEntry entry; - Map cacheGrpStates = null; // Create lazy checkpoint entry. @@ -3085,7 +3133,20 @@ private void doCheckpoint() { try { CheckpointMetricsTracker tracker = new CheckpointMetricsTracker(); - Checkpoint chp = markCheckpointBegin(tracker); + Checkpoint chp; + + try { + chp = markCheckpointBegin(tracker); + } + catch (IgniteCheckedException e) { + if (curCpProgress != null) + curCpProgress.cpFinishFut.onDone(e); + + // In case of checkpoint initialization error node should be invalidated and stopped. + cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e)); + + return; + } currCheckpointPagesCnt = chp.pagesSize; @@ -3145,7 +3206,7 @@ private void doCheckpoint() { } catch (IgniteCheckedException e) { chp.progress.cpFinishFut.onDone(e); - // In case of writing error node should be invalidated and stopped. + // In case of checkpoint writing error node should be invalidated and stopped. cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e)); return; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java index b1db79d706a2e..799a78cb99e60 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java @@ -169,13 +169,18 @@ public AsyncFileIO(File file, ThreadLocal holder, OpenOption... } /** {@inheritDoc} */ - @Override public MappedByteBuffer map(int maxWalSegmentSize) throws IOException { + @Override public MappedByteBuffer map(int sizeBytes) throws IOException { throw new UnsupportedOperationException("AsynchronousFileChannel doesn't support mmap."); } /** {@inheritDoc} */ @Override public void force() throws IOException { - ch.force(false); + force(false); + } + + /** {@inheritDoc} */ + @Override public void force(boolean withMetadata) throws IOException { + ch.force(withMetadata); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java index 73e44b0d9af1f..822bd66413dc3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java @@ -124,7 +124,16 @@ public interface FileIO extends AutoCloseable { */ public void write(byte[] buf, int off, int len) throws IOException; - public MappedByteBuffer map(int maxWalSegmentSize) throws IOException; + /** + * Allocates memory mapped buffer for this file with given size. + * + * @param sizeBytes Size of buffer. + * + * @return Instance of mapped byte buffer. + * + * @throws IOException If some I/O error occurs. + */ + public MappedByteBuffer map(int sizeBytes) throws IOException; /** * Forces any updates of this file to be written to the storage @@ -134,6 +143,15 @@ public interface FileIO extends AutoCloseable { */ public void force() throws IOException; + /** + * Forces any updates of this file to be written to the storage + * device that contains it. + * + * @param withMetadata If {@code true} force also file metadata. + * @throws IOException If some I/O error occurs. + */ + public void force(boolean withMetadata) throws IOException; + /** * Returns current file size in bytes. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java index dd563f2d68197..683845bc268ba 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java @@ -77,8 +77,8 @@ public FileIODecorator(FileIO delegate) { } /** {@inheritDoc} */ - @Override public MappedByteBuffer map(int maxWalSegmentSize) throws IOException { - return delegate.map(maxWalSegmentSize); + @Override public MappedByteBuffer map(int sizeBytes) throws IOException { + return delegate.map(sizeBytes); } /** {@inheritDoc} */ @@ -86,6 +86,11 @@ public FileIODecorator(FileIO delegate) { delegate.force(); } + /** {@inheritDoc} */ + @Override public void force(boolean withMetadata) throws IOException { + delegate.force(withMetadata); + } + /** {@inheritDoc} */ @Override public long size() throws IOException { return delegate.size(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java index 23d6ebfeead76..8f7454dcae617 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java @@ -84,8 +84,8 @@ public RandomAccessFileIO(File file, OpenOption... modes) throws IOException { } /** {@inheritDoc} */ - @Override public void force() throws IOException { - ch.force(false); + @Override public void force(boolean withMetadata) throws IOException { + ch.force(withMetadata); } /** {@inheritDoc} */ @@ -104,7 +104,12 @@ public RandomAccessFileIO(File file, OpenOption... modes) throws IOException { } /** {@inheritDoc} */ - @Override public MappedByteBuffer map(int maxWalSegmentSize) throws IOException { - return ch.map(FileChannel.MapMode.READ_WRITE, 0, maxWalSegmentSize); + @Override public MappedByteBuffer map(int sizeBytes) throws IOException { + return ch.map(FileChannel.MapMode.READ_WRITE, 0, sizeBytes); + } + + /** {@inheritDoc} */ + @Override public void force() throws IOException { + force(false); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/UnzipFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/UnzipFileIO.java index 83ff91ba99da5..469cf3eb7ebbc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/UnzipFileIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/UnzipFileIO.java @@ -116,6 +116,11 @@ public UnzipFileIO(File zip) throws IOException { /** {@inheritDoc} */ @Override public void force() throws IOException { + force(false); + } + + /** {@inheritDoc} */ + @Override public void force(boolean withMetadata) throws IOException { throw new UnsupportedOperationException(); } @@ -130,7 +135,7 @@ public UnzipFileIO(File zip) throws IOException { } /** {@inheritDoc} */ - @Override public MappedByteBuffer map(int maxWalSegmentSize) throws IOException { + @Override public MappedByteBuffer map(int sizeBytes) throws IOException { throw new UnsupportedOperationException(); } 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 3e85c7760478c..c902879cbaae2 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 @@ -25,6 +25,7 @@ import java.util.Arrays; import java.util.concurrent.atomic.AtomicLong; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheRebalanceMode; import org.apache.ignite.cache.CacheWriteSynchronizationMode; @@ -39,15 +40,20 @@ import org.apache.ignite.internal.GridKernalState; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInterruptedCheckedException; +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.FileIODecorator; import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; +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.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_WAL_MMAP; /** @@ -60,20 +66,14 @@ public class IgnitePdsDiskErrorsRecoveringTest extends GridCommonAbstractTest { /** */ private static final int WAL_SEGMENT_SIZE = 1024 * PAGE_SIZE; - /** */ - private static final long DFLT_DISK_SPACE_BYTES = Long.MAX_VALUE; - /** */ private static final long STOP_TIMEOUT_MS = 30 * 1000; /** */ private static final String CACHE_NAME = "cache"; - /** */ - private boolean failPageStoreDiskOperations = false; - - /** */ - private long diskSpaceBytes = DFLT_DISK_SPACE_BYTES; + /** Specified i/o factory for particular test. */ + private FileIOFactory ioFactory; /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { @@ -88,8 +88,7 @@ public class IgnitePdsDiskErrorsRecoveringTest extends GridCommonAbstractTest { cleanPersistenceDir(); - failPageStoreDiskOperations = false; - diskSpaceBytes = DFLT_DISK_SPACE_BYTES; + ioFactory = null; System.clearProperty(IGNITE_WAL_MMAP); } @@ -103,10 +102,11 @@ public class IgnitePdsDiskErrorsRecoveringTest extends GridCommonAbstractTest { .setWalMode(WALMode.LOG_ONLY) .setWalCompactionEnabled(false) .setWalSegmentSize(WAL_SEGMENT_SIZE) + .setCheckpointFrequency(240 * 60 * 1000) .setConcurrencyLevel(Runtime.getRuntime().availableProcessors() * 4); - if (failPageStoreDiskOperations) - dsCfg.setFileIOFactory(new LimitedSizeFileIOFactory(new RandomAccessFileIOFactory(), diskSpaceBytes)); + if (ioFactory != null) + dsCfg.setFileIOFactory(ioFactory); cfg.setDataStorageConfiguration(dsCfg); @@ -122,19 +122,17 @@ public class IgnitePdsDiskErrorsRecoveringTest extends GridCommonAbstractTest { } /** - * + * Test node stopping & recovering on cache initialization fail. */ - public void testRecoveringOnCacheInitError() throws Exception { - failPageStoreDiskOperations = true; - - // Two pages is enough to initialize MetaStorage. - diskSpaceBytes = 2 * PAGE_SIZE; + public void testRecoveringOnCacheInitFail() throws Exception { + // Fail to initialize page store. 2 extra pages is needed for MetaStorage. + ioFactory = new FilteringFileIOFactory(".bin", new LimitedSizeFileIOFactory(new RandomAccessFileIOFactory(), 2 * PAGE_SIZE)); final IgniteEx grid = startGrid(0); boolean failed = false; try { - grid.active(true); + grid.cluster().active(true); } catch (Exception expected) { log.warning("Expected cache error", expected); @@ -147,21 +145,128 @@ public void testRecoveringOnCacheInitError() throws Exception { awaitStop(grid); // Grid should be successfully recovered after stopping. - failPageStoreDiskOperations = false; + ioFactory = null; IgniteEx recoveredGrid = startGrid(0); recoveredGrid.active(true); } /** + * Test node stopping & recovering on start marker writing fail during activation. * + * @throws Exception If test failed. */ - public void testRecoveringOnCheckpointWritingError() throws Exception { - failPageStoreDiskOperations = true; - diskSpaceBytes = 1024 * PAGE_SIZE; + 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)); + + IgniteEx grid = startGrid(0); + grid.cluster().active(true); + + for (int i = 0; i < 1000; i++) { + byte payload = (byte) i; + byte[] data = new byte[2048]; + Arrays.fill(data, payload); + + grid.cache(CACHE_NAME).put(i, data); + } + + stopAllGrids(); + + boolean activationFailed = false; + try { + grid = startGrid(0); + grid.cluster().active(true); + } + catch (IgniteException e) { + log.warning("Activation test exception", e); + + activationFailed = true; + } + + Assert.assertTrue("Activation must be failed", activationFailed); + + // Grid should be automatically stopped after checkpoint fail. + awaitStop(grid); + + // Grid should be successfully recovered after stopping. + ioFactory = null; + + IgniteEx recoveredGrid = startGrid(0); + recoveredGrid.cluster().active(true); + + for (int i = 0; i < 1000; i++) { + byte payload = (byte) i; + byte[] data = new byte[2048]; + Arrays.fill(data, payload); + + byte[] actualData = (byte[]) recoveredGrid.cache(CACHE_NAME).get(i); + Assert.assertArrayEquals(data, actualData); + } + } + + + /** + * Test node stopping & recovering on checkpoint begin fail. + * + * @throws Exception If test failed. + */ + 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)); final IgniteEx grid = startGrid(0); - grid.active(true); + grid.cluster().active(true); + + for (int i = 0; i < 1000; i++) { + byte payload = (byte) i; + byte[] data = new byte[2048]; + Arrays.fill(data, payload); + + grid.cache(CACHE_NAME).put(i, data); + } + + String errMsg = "Failed to write checkpoint entry"; + + boolean checkpointFailed = false; + try { + forceCheckpoint(); + } + catch (IgniteCheckedException e) { + if (e.getMessage().contains(errMsg)) + checkpointFailed = true; + } + + Assert.assertTrue("Checkpoint must be failed by IgniteCheckedException: " + errMsg, checkpointFailed); + + // Grid should be automatically stopped after checkpoint fail. + awaitStop(grid); + + // Grid should be successfully recovered after stopping. + ioFactory = null; + + IgniteEx recoveredGrid = startGrid(0); + recoveredGrid.cluster().active(true); + + for (int i = 0; i < 1000; i++) { + byte payload = (byte) i; + byte[] data = new byte[2048]; + Arrays.fill(data, payload); + + byte[] actualData = (byte[]) recoveredGrid.cache(CACHE_NAME).get(i); + Assert.assertArrayEquals(data, actualData); + } + } + + /** + * Test node stopping & recovering on checkpoint pages write fail. + */ + public void testRecoveringOnCheckpointWriteFail() throws Exception { + // Fail write partition and index files at the second checkpoint. Pass only initial checkpoint. + ioFactory = new FilteringFileIOFactory(".bin", new LimitedSizeFileIOFactory(new RandomAccessFileIOFactory(), 128 * PAGE_SIZE)); + + final IgniteEx grid = startGrid(0); + grid.cluster().active(true); for (int i = 0; i < 1000; i++) { byte payload = (byte) i; @@ -187,10 +292,10 @@ public void testRecoveringOnCheckpointWritingError() throws Exception { awaitStop(grid); // Grid should be successfully recovered after stopping. - failPageStoreDiskOperations = false; + ioFactory = null; IgniteEx recoveredGrid = startGrid(0); - recoveredGrid.active(true); + recoveredGrid.cluster().active(true); for (int i = 0; i < 1000; i++) { byte payload = (byte) i; @@ -203,33 +308,35 @@ public void testRecoveringOnCheckpointWritingError() throws Exception { } /** - * + * Test node stopping & recovering on WAL writing fail with enabled MMAP (Batch allocation for WAL segments). */ - public void testRecoveringOnWALErrorWithMmap() throws Exception { - diskSpaceBytes = WAL_SEGMENT_SIZE; + public void testRecoveringOnWALWritingFail1() throws Exception { + // Allow to allocate only 1 wal segment, fail on write to second. + ioFactory = new FilteringFileIOFactory(".wal", new LimitedSizeFileIOFactory(new RandomAccessFileIOFactory(), WAL_SEGMENT_SIZE)); System.setProperty(IGNITE_WAL_MMAP, "true"); - emulateRecoveringOnWALWritingError(); + doTestRecoveringOnWALWritingFail(); } /** - * + * Test node stopping & recovering on WAL writing fail with disabled MMAP. */ - public void testRecoveringOnWALErrorWithoutMmap() throws Exception { - diskSpaceBytes = 2 * WAL_SEGMENT_SIZE; + public void testRecoveringOnWALWritingFail2() throws Exception { + // Fail somewhere on the second wal segment. + ioFactory = new FilteringFileIOFactory(".wal", new LimitedSizeFileIOFactory(new RandomAccessFileIOFactory(), (long) (1.5 * WAL_SEGMENT_SIZE))); System.setProperty(IGNITE_WAL_MMAP, "false"); - emulateRecoveringOnWALWritingError(); + doTestRecoveringOnWALWritingFail(); } /** - * + * Test node stopping & recovery on WAL writing fail. */ - private void emulateRecoveringOnWALWritingError() throws Exception { + private void doTestRecoveringOnWALWritingFail() throws Exception { final IgniteEx grid = startGrid(0); FileWriteAheadLogManager wal = (FileWriteAheadLogManager)grid.context().cache().context().wal(); - wal.setFileIOFactory(new LimitedSizeFileIOFactory(new RandomAccessFileIOFactory(), diskSpaceBytes)); + wal.setFileIOFactory(ioFactory); - grid.active(true); + grid.cluster().active(true); int failedPosition = -1; @@ -254,9 +361,11 @@ private void emulateRecoveringOnWALWritingError() throws Exception { // Grid should be automatically stopped after WAL fail. awaitStop(grid); + ioFactory = null; + // Grid should be successfully recovered after stopping. IgniteEx recoveredGrid = startGrid(0); - recoveredGrid.active(true); + recoveredGrid.cluster().active(true); for (int i = 0; i < failedPosition; i++) { byte payload = (byte) i; @@ -328,11 +437,49 @@ public LimitedSizeFileIO(FileIO delegate, AtomicLong availableSpaceBytes) { } /** {@inheritDoc} */ - @Override public MappedByteBuffer map(int maxWalSegmentSize) throws IOException { - availableSpaceBytes.addAndGet(-maxWalSegmentSize); + @Override public MappedByteBuffer map(int sizeBytes) throws IOException { + availableSpaceBytes.addAndGet(-sizeBytes); if (availableSpaceBytes.get() < 0) throw new IOException("Not enough space!"); - return super.map(maxWalSegmentSize); + return super.map(sizeBytes); + } + } + + /** + * Factory to provide custom File I/O interfaces only for files with specified suffix. + * For other files {@link RandomAccessFileIO} will be used. + */ + private static class FilteringFileIOFactory implements FileIOFactory { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Delegate. */ + private final FileIOFactory delegate; + + /** File suffix pattern. */ + private final String pattern; + + /** + * Constructor. + * + * @param pattern File suffix pattern. + * @param delegate I/O Factory delegate. + */ + FilteringFileIOFactory(String pattern, FileIOFactory delegate) { + this.delegate = delegate; + this.pattern = pattern; + } + + /** {@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 { + if (file.getName().endsWith(pattern)) + return delegate.create(file, modes); + return new RandomAccessFileIO(file, modes); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java index 946b4e80c52d5..042a447d032eb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java @@ -206,8 +206,8 @@ private static class FailingFileIOFactory implements FileIOFactory { } /** {@inheritDoc} */ - @Override public MappedByteBuffer map(int maxWalSegmentSize) throws IOException { - return delegate.map(maxWalSegmentSize); + @Override public MappedByteBuffer map(int sizeBytes) throws IOException { + return delegate.map(sizeBytes); } }; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java index 1259c3c489ccc..fe1632817e44c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java @@ -250,8 +250,8 @@ private static class FailingFileIOFactory implements FileIOFactory { } /** {@inheritDoc} */ - @Override public MappedByteBuffer map(int maxWalSegmentSize) throws IOException { - return delegate.map(maxWalSegmentSize); + @Override public MappedByteBuffer map(int sizeBytes) throws IOException { + return delegate.map(sizeBytes); } }; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java index 9f1342fd9d3b7..249718b02ec06 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java @@ -319,8 +319,8 @@ private static class SlowCheckpointFileIOFactory implements FileIOFactory { } /** {@inheritDoc} */ - @Override public MappedByteBuffer map(int maxWalSegmentSize) throws IOException { - return delegate.map(maxWalSegmentSize); + @Override public MappedByteBuffer map(int sizeBytes) throws IOException { + return delegate.map(sizeBytes); } }; } diff --git a/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIO.java b/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIO.java index 3cb4886e4613f..681426cd4c18e 100644 --- a/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIO.java +++ b/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIO.java @@ -460,12 +460,17 @@ private static String getLastError() { } /** {@inheritDoc} */ - @Override public MappedByteBuffer map(int maxWalSegmentSize) throws IOException { + @Override public MappedByteBuffer map(int sizeBytes) throws IOException { throw new UnsupportedOperationException("AsynchronousFileChannel doesn't support mmap."); } /** {@inheritDoc} */ @Override public void force() throws IOException { + force(false); + } + + /** {@inheritDoc} */ + @Override public void force(boolean withMetadata) throws IOException { if (IgniteNativeIoLib.fsync(fdCheckOpened()) < 0) throw new IOException(String.format("Error fsync()'ing %s, got %s", file, getLastError())); } From aacb391e1fb4c0b087856f960137d338bdd571bd Mon Sep 17 00:00:00 2001 From: Anton Kalashnikov Date: Fri, 25 May 2018 12:01:10 +0300 Subject: [PATCH 0198/1463] IGNITE-8540 Fast cleanup of PDS when joining node is not in baseline - Fixes #4037. Signed-off-by: Alexey Goncharuk (cherry-picked from commit #3f14d2b35d7818196598e8541590186e06b8edbb) --- .../pagemem/store/IgnitePageStoreManager.java | 8 ++ .../wal/IgniteWriteAheadLogManager.java | 5 + .../processors/cache/GridCacheProcessor.java | 16 ++- .../GridDhtPartitionsExchangeFuture.java | 18 +++ .../GridCacheDatabaseSharedManager.java | 19 +++- .../IgniteCacheDatabaseSharedManager.java | 7 ++ .../file/FilePageStoreManager.java | 23 ++++ .../wal/FileWriteAheadLogManager.java | 25 +++++ .../FsyncModeFileWriteAheadLogManager.java | 25 +++++ ...teAbsentEvictionNodeOutOfBaselineTest.java | 106 ++++++++++++++++++ .../pagemem/NoOpPageStoreManager.java | 6 + .../persistence/pagemem/NoOpWALManager.java | 5 + .../testsuites/IgnitePdsTestSuite2.java | 2 + 13 files changed, 262 insertions(+), 3 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteAbsentEvictionNodeOutOfBaselineTest.java 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 0fc9f94b41978..7dba8aee94c5e 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 @@ -20,6 +20,7 @@ import java.nio.ByteBuffer; import java.util.Map; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; @@ -220,4 +221,11 @@ public void initializeForCache(CacheGroupDescriptor grpDesc, StoredCacheData cac * @return number of pages. */ public long pagesAllocated(int grpId); + + /** + * Cleanup persistent space for cache. + * + * @param cacheConfiguration Cache configuration of cache which should be cleanup. + */ + public void cleanupPersistentSpace(CacheConfiguration cacheConfiguration) throws IgniteCheckedException; } 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 b5c22c92341c4..fd5d53b17e1d4 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 @@ -144,4 +144,9 @@ public interface IgniteWriteAheadLogManager extends GridCacheSharedManager, Igni * @param grpId Group id. */ public boolean disabled(int grpId); + + /** + * Cleanup all directories relating to WAL (e.g. work WAL dir, archive WAL dir). + */ + public void cleanupWalDirectories() throws IgniteCheckedException; } 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 ccccdec536cff..8cd6b881e0614 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 @@ -179,6 +179,7 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_CONSISTENCY_CHECK_SKIPPED; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_TX_CONFIG; import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isNearEnabled; +import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isPersistentCache; /** * Cache processor. @@ -3294,10 +3295,23 @@ public void saveCacheConfiguration(DynamicCacheDescriptor desc) throws IgniteChe assert desc != null; if (sharedCtx.pageStore() != null && !sharedCtx.kernalContext().clientNode() && - CU.isPersistentCache(desc.cacheConfiguration(), sharedCtx.gridConfig().getDataStorageConfiguration())) + isPersistentCache(desc.cacheConfiguration(), sharedCtx.gridConfig().getDataStorageConfiguration())) sharedCtx.pageStore().storeCacheData(desc.toStoredData(), true); } + /** + * Remove all persistent files for all registered caches. + */ + public void cleanupCachesDirectories() throws IgniteCheckedException { + if (sharedCtx.pageStore() == null || sharedCtx.kernalContext().clientNode()) + return; + + for (DynamicCacheDescriptor desc : cacheDescriptors().values()) { + if (isPersistentCache(desc.cacheConfiguration(), sharedCtx.gridConfig().getDataStorageConfiguration())) + sharedCtx.pageStore().cleanupPersistentSpace(desc.cacheConfiguration()); + } + } + /** * @param reqs Requests. * @return Collection of futures. 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 c62b067013357..0acf41ba50fd3 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 @@ -774,6 +774,15 @@ else if (msg instanceof WalStateAbstractMessage) * @throws IgniteCheckedException If failed. */ private void initCachesOnLocalJoin() throws IgniteCheckedException { + if (isLocalNodeNotInBaseline()) { + cctx.cache().cleanupCachesDirectories(); + + cctx.database().cleanupCheckpointDirectory(); + + if (cctx.wal() != null) + cctx.wal().cleanupWalDirectories(); + } + cctx.activate(); LocalJoinCachesContext locJoinCtx = exchActions == null ? null : exchActions.localJoinContext(); @@ -799,6 +808,15 @@ private void initCachesOnLocalJoin() throws IgniteCheckedException { cctx.cache().startCachesOnLocalJoin(locJoinCtx, initialVersion()); } + /** + * @return {@code true} if local node is not in baseline and {@code false} otherwise. + */ + private boolean isLocalNodeNotInBaseline() { + BaselineTopology topology = cctx.discovery().discoCache().state().baselineTopology(); + + return topology!= null && !topology.consistentIds().contains(cctx.localNode().consistentId()); + } + /** * @throws IgniteCheckedException If failed. */ 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 4096a8d7ba13f..6064cfe2775cd 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 @@ -525,7 +525,7 @@ private DataRegionConfiguration createDataRegionConfiguration(DataStorageConfigu if (!U.mkdirs(cpDir)) throw new IgniteCheckedException("Could not create directory for checkpoint metadata: " + cpDir); - cleanupCheckpointDirectory(); + cleanupTempCheckpointDirectory(); final FileLockHolder preLocked = kernalCtx.pdsFolderResolver() .resolveFolders() @@ -543,7 +543,7 @@ private DataRegionConfiguration createDataRegionConfiguration(DataStorageConfigu /** * Cleanup checkpoint directory from all temporary files {@link #FILE_TMP_SUFFIX}. */ - private void cleanupCheckpointDirectory() throws IgniteCheckedException { + private void cleanupTempCheckpointDirectory() throws IgniteCheckedException { try { try (DirectoryStream files = Files.newDirectoryStream(cpDir.toPath(), new DirectoryStream.Filter() { @Override @@ -555,6 +555,21 @@ public boolean accept(Path path) throws IOException { Files.delete(path); } } + catch (IOException e) { + throw new IgniteCheckedException("Failed to cleanup checkpoint directory from temporary files: " + cpDir, e); + } + } + + /** + * Cleanup checkpoint directory. + */ + public void cleanupCheckpointDirectory() throws IgniteCheckedException { + try { + try (DirectoryStream files = Files.newDirectoryStream(cpDir.toPath())) { + for (Path path : files) + Files.delete(path); + } + } catch (IOException e) { throw new IgniteCheckedException("Failed to cleanup checkpoint directory: " + cpDir, e); } 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 a251234683342..5e8f40706e984 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 @@ -732,6 +732,13 @@ public void checkpointReadUnlock() { // No-op. } + /** + * No-op for non-persistent storage. + */ + public void cleanupCheckpointDirectory() throws IgniteCheckedException { + // No-op. + } + /** * */ 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 661694d72101b..d065ff1e1e54b 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 @@ -153,6 +153,29 @@ public FilePageStoreManager(GridKernalContext ctx) { U.ensureDirectory(storeWorkDir, "page store work directory", log); } + /** {@inheritDoc} */ + public void cleanupPersistentSpace(CacheConfiguration cacheConfiguration) throws IgniteCheckedException { + try { + File cacheWorkDir = cacheWorkDir(cacheConfiguration); + + if(!cacheWorkDir.exists()) + return; + + try (DirectoryStream files = Files.newDirectoryStream(cacheWorkDir.toPath(), + new DirectoryStream.Filter() { + @Override public boolean accept(Path entry) throws IOException { + return entry.toFile().getName().endsWith(FILE_SUFFIX); + } + })) { + for (Path path : files) + Files.delete(path); + } + } + catch (IOException e) { + throw new IgniteCheckedException("Failed to cleanup persistent directory: ", e); + } + } + /** {@inheritDoc} */ @Override public void stop0(boolean cancel) { if (log.isDebugEnabled()) 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 40ebcf04289fd..5efd5ee6c44b2 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 @@ -33,8 +33,10 @@ import java.nio.ByteOrder; import java.nio.MappedByteBuffer; import java.nio.channels.ClosedByInterruptException; +import java.nio.file.DirectoryStream; import java.nio.file.FileAlreadyExistsException; import java.nio.file.Files; +import java.nio.file.Path; import java.sql.Time; import java.util.ArrayList; import java.util.Arrays; @@ -1329,6 +1331,29 @@ private void checkOrPrepareFiles() throws IgniteCheckedException { checkFiles(0, false, null, null); } + /** {@inheritDoc} */ + public void cleanupWalDirectories() throws IgniteCheckedException { + try { + try (DirectoryStream files = Files.newDirectoryStream(walWorkDir.toPath())) { + for (Path path : files) + Files.delete(path); + } + } + catch (IOException e) { + throw new IgniteCheckedException("Failed to cleanup wal work directory: " + walWorkDir, e); + } + + try { + try (DirectoryStream files = Files.newDirectoryStream(walArchiveDir.toPath())) { + for (Path path : files) + Files.delete(path); + } + } + catch (IOException e) { + throw new IgniteCheckedException("Failed to cleanup wal archive directory: " + walArchiveDir, e); + } + } + /** * Clears whole the file, fills with zeros for Default mode. * 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 79a3e1966dede..4a642d0193fe3 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 @@ -28,8 +28,10 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.nio.file.DirectoryStream; import java.nio.file.FileAlreadyExistsException; import java.nio.file.Files; +import java.nio.file.Path; import java.sql.Time; import java.util.ArrayList; import java.util.Arrays; @@ -886,6 +888,29 @@ private boolean hasIndex(long absIdx) { return ctx != null && !ctx.walEnabled(); } + /** {@inheritDoc} */ + @Override public void cleanupWalDirectories() throws IgniteCheckedException { + try { + try (DirectoryStream files = Files.newDirectoryStream(walWorkDir.toPath())) { + for (Path path : files) + Files.delete(path); + } + } + catch (IOException e) { + throw new IgniteCheckedException("Failed to cleanup wal work directory: " + walWorkDir, e); + } + + try { + try (DirectoryStream files = Files.newDirectoryStream(walArchiveDir.toPath())) { + for (Path path : files) + Files.delete(path); + } + } + catch (IOException e) { + throw new IgniteCheckedException("Failed to cleanup wal archive directory: " + walArchiveDir, e); + } + } + /** * Lists files in archive directory and returns the index of last archived file. * diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteAbsentEvictionNodeOutOfBaselineTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteAbsentEvictionNodeOutOfBaselineTest.java new file mode 100644 index 0000000000000..0c9fb631b39a2 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/IgniteAbsentEvictionNodeOutOfBaselineTest.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.baseline; + +import java.util.List; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +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.processors.cache.distributed.dht.GridDhtLocalPartition; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Test absenting eviction for joined node if it is out of baseline. + */ +public class IgniteAbsentEvictionNodeOutOfBaselineTest extends GridCommonAbstractTest { + /** */ + private static final String TEST_CACHE_NAME = "test"; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setDataStorageConfiguration(new DataStorageConfiguration() + .setWalSegmentSize(512 * 1024) + .setWalSegments(4) + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration() + .setMaxSize(256 * 1024 * 1024) + .setPersistenceEnabled(true)) + .setWalMode(WALMode.LOG_ONLY)); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** + * Removed partitions if node is out of baseline. + */ + public void testPartitionsRemovedIfJoiningNodeNotInBaseline() throws Exception { + //given: start 3 nodes with data + Ignite ignite0 = startGrids(3); + + ignite0.cluster().active(true); + + IgniteCache cache = ignite0.getOrCreateCache(TEST_CACHE_NAME); + + for(int i = 0; i< 100; i++) + cache.put(i, i); + + //when: stop one node and reset baseline topology + stopGrid(2); + + resetBaselineTopology(); + + awaitPartitionMapExchange(); + + for(int i = 0; i< 200; i++) + cache.put(i, i); + + //then: after returning stopped node to grid its partitions should be removed + IgniteEx ignite2 = startGrid(2); + + List partitions = ignite2.cachex(TEST_CACHE_NAME).context().topology().localPartitions(); + + assertTrue("Should be empty : " + partitions, partitions.isEmpty()); + } +} 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 be40c90f7c15c..ba236af929365 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 @@ -24,6 +24,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.pagemem.FullPageId; import org.apache.ignite.internal.pagemem.PageIdUtils; @@ -215,4 +216,9 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager { @Override public long pagesAllocated(int grpId) { return 0; } + + /** {@inheritDoc} */ + @Override public void cleanupPersistentSpace(CacheConfiguration cacheConfiguration) throws IgniteCheckedException { + // No-op. + } } 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 c95d1f4a61340..0188445463d32 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 @@ -101,6 +101,11 @@ public class NoOpWALManager implements IgniteWriteAheadLogManager { return false; } + /** {@inheritDoc} */ + @Override public void cleanupWalDirectories() throws IgniteCheckedException { + // No-op. + } + /** {@inheritDoc} */ @Override public void start(GridCacheSharedContext cctx) throws IgniteCheckedException { // No-op. 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 e3880dded5c1c..af797407c9e15 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 @@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsRecoveryAfterFileCorruptionTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePersistentStoreDataStructuresTest; 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.IgniteAllBaselineNodesOnlineFullApiSelfTest; import org.apache.ignite.internal.processors.cache.persistence.baseline.IgniteOfflineBaselineNodeFullApiSelfTest; import org.apache.ignite.internal.processors.cache.persistence.baseline.IgniteOnlineNodeOutOfBaselineFullApiSelfTest; @@ -70,6 +71,7 @@ public static TestSuite suite() { suite.addTestSuite(IgniteAllBaselineNodesOnlineFullApiSelfTest.class); suite.addTestSuite(IgniteOfflineBaselineNodeFullApiSelfTest.class); suite.addTestSuite(IgniteOnlineNodeOutOfBaselineFullApiSelfTest.class); + suite.addTestSuite(IgniteAbsentEvictionNodeOutOfBaselineTest.class); return suite; } From d9b93dc7264e8030bd3d40525ead35bb38d316c5 Mon Sep 17 00:00:00 2001 From: Sergey Chugunov Date: Fri, 25 May 2018 13:29:26 +0300 Subject: [PATCH 0199/1463] IGNITE-7933 compilation error fix after cherry-pick --- .../LocalWalModeChangeDuringRebalancingSelfTest.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java index 8be819f27b4e8..ca46a75e09181 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java @@ -614,6 +614,11 @@ private static class TestFileIO implements FileIO { delegate.force(); } + /** {@inheritDoc} */ + @Override public void force(boolean withMetadata) throws IOException { + delegate.force(withMetadata); + } + /** {@inheritDoc} */ @Override public long size() throws IOException { return delegate.size(); From 660e18af579a60cbb9282bad3ce87edb715dd920 Mon Sep 17 00:00:00 2001 From: Stanislav Lukyanov Date: Tue, 22 May 2018 16:43:45 +0300 Subject: [PATCH 0200/1463] IGNITE-8501 Retry on GridServiceNotFoundException in GridServiceProxy needs to be fixed - Fixes #4004. Signed-off-by: Ivan Rakov (cherry picked from commit 7c2fe64) --- .../processors/service/GridServiceProxy.java | 22 +++++-- ...idServiceProxyClientReconnectSelfTest.java | 62 ++++++++++++++++++- 2 files changed, 78 insertions(+), 6 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java index c5a2ceea19f54..eb75a5b0d1c5c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java @@ -206,13 +206,25 @@ else if (U.isToStringMethod(mtd)) throw e; } catch (IgniteCheckedException e) { - // Rethrow original service method exception so that calling user code can handle it correctly. - ServiceProxyException svcProxyE = X.cause(e, ServiceProxyException.class); + // Check if ignorable exceptions are in the cause chain. + Throwable ignorableCause = X.cause(e, GridServiceNotFoundException.class); - if (svcProxyE != null) - throw svcProxyE.getCause(); + if (ignorableCause == null) + ignorableCause = X.cause(e, ClusterTopologyCheckedException.class); - throw U.convertException(e); + if (ignorableCause != null) { + if (log.isDebugEnabled()) + log.debug("Service was not found or topology changed (will retry): " + ignorableCause.getMessage()); + } + else { + // Rethrow original service method exception so that calling user code can handle it correctly. + ServiceProxyException svcProxyE = X.cause(e, ServiceProxyException.class); + + if (svcProxyE != null) + throw svcProxyE.getCause(); + + throw U.convertException(e); + } } catch (Exception e) { throw new IgniteException(e); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProxyClientReconnectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProxyClientReconnectSelfTest.java index d2fda2f0d3325..5fdcdc49e5224 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProxyClientReconnectSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProxyClientReconnectSelfTest.java @@ -82,13 +82,49 @@ public void testClientReconnect() throws Exception { startGrid("server"); - assert latch.await(10, TimeUnit.SECONDS); + assertTrue(latch.await(10, TimeUnit.SECONDS)); client.services().deployClusterSingleton("my-service", new MyServiceImpl()); assertEquals(42, proxy.hello()); } + + /** + * @throws Exception If failed. + */ + public void testClientReconnectLongServiceInit() throws Exception { + startGrid("server"); + + Ignite client = startGrid("client"); + + client.services().deployClusterSingleton("my-service", new MyLongInitServiceImpl()); + + MyService proxy = client.services().serviceProxy("my-service", MyService.class, false); + + assertEquals(9001, proxy.hello()); + + final CountDownLatch latch = new CountDownLatch(1); + + client.events().localListen(new IgnitePredicate() { + @Override public boolean apply(Event event) { + latch.countDown(); + + return true; + } + }, EventType.EVT_CLIENT_NODE_RECONNECTED); + + stopGrid("server"); + + startGrid("server"); + + assertTrue(latch.await(10, TimeUnit.SECONDS)); + + client.services().deployClusterSingleton("my-service", new MyLongInitServiceImpl()); + + assertEquals(9001, proxy.hello()); + } + /** */ private interface MyService extends Service { @@ -121,4 +157,28 @@ private static class MyServiceImpl implements MyService { // No-op. } } + + /** + */ + private static class MyLongInitServiceImpl implements MyService { + /** {@inheritDoc} */ + @Override public int hello() { + return 9001; + } + + /** {@inheritDoc} */ + @Override public void cancel(ServiceContext ctx) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void init(ServiceContext ctx) throws Exception { + Thread.sleep(5_000); + } + + /** {@inheritDoc} */ + @Override public void execute(ServiceContext ctx) throws Exception { + // No-op. + } + } } From 8d5853f2f4460dfc4959f128ed8d8c0d54dceb58 Mon Sep 17 00:00:00 2001 From: Ilya Kasnacheev Date: Tue, 29 May 2018 15:15:23 +0300 Subject: [PATCH 0201/1463] IGNITE-8547 - Use JVM serialization for enum values with OptimizedMarshaller, avoid deadlock. Cherry-picked from 5564a14f7cb. --- .../ignite/internal/binary/BinaryUtils.java | 19 +- .../builder/BinaryBuilderSerializer.java | 3 +- .../OptimizedObjectOutputStream.java | 4 +- .../ignite/internal/util/IgniteUtils.java | 15 ++ .../MarshallerEnumDeadlockMultiJvmTest.java | 189 ++++++++++++++++++ .../IgniteMarshallerSelfTestSuite.java | 2 + 6 files changed, 213 insertions(+), 19 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerEnumDeadlockMultiJvmTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java index 1f167f5f08080..082cc206e3231 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java @@ -600,7 +600,7 @@ public static byte typeByClass(Class cls) { if (type != null) return type; - if (isEnum(cls)) + if (U.isEnum(cls)) return GridBinaryMarshaller.ENUM; if (cls.isArray()) @@ -1141,7 +1141,7 @@ else if (isSpecialCollection(cls)) return BinaryWriteMode.COL; else if (isSpecialMap(cls)) return BinaryWriteMode.MAP; - else if (isEnum(cls)) + else if (U.isEnum(cls)) return BinaryWriteMode.ENUM; else if (cls == BinaryEnumObjectImpl.class) return BinaryWriteMode.BINARY_ENUM; @@ -1174,21 +1174,6 @@ public static boolean isSpecialMap(Class cls) { return HashMap.class.equals(cls) || LinkedHashMap.class.equals(cls); } - /** - * Check if class represents a Enum. - * - * @param cls Class. - * @return {@code True} if this is a Enum class. - */ - public static boolean isEnum(Class cls) { - if (cls.isEnum()) - return true; - - Class sCls = cls.getSuperclass(); - - return sCls != null && sCls.isEnum(); - } - /** * @return Value. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java index 018444c65123a..42f6873bf1548 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java @@ -27,6 +27,7 @@ import org.apache.ignite.internal.binary.BinaryUtils; import org.apache.ignite.internal.binary.BinaryWriterExImpl; import org.apache.ignite.internal.binary.GridBinaryMarshaller; +import org.apache.ignite.internal.util.IgniteUtils; /** * @@ -110,7 +111,7 @@ public void writeValue(BinaryWriterExImpl writer, Object val, boolean forceCol, return; } - if (BinaryUtils.isEnum(val.getClass())) { + if (IgniteUtils.isEnum(val.getClass())) { String clsName = ((Enum)val).getDeclaringClass().getName(); int typeId = writer.context().typeId(clsName); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/marshaller/optimized/OptimizedObjectOutputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/marshaller/optimized/OptimizedObjectOutputStream.java index 66da2da7eb908..fadbec61fad02 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/marshaller/optimized/OptimizedObjectOutputStream.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/marshaller/optimized/OptimizedObjectOutputStream.java @@ -42,6 +42,7 @@ import org.apache.ignite.internal.util.GridHandleTable; import org.apache.ignite.internal.util.io.GridDataOutput; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.marshaller.MarshallerContext; @@ -180,7 +181,8 @@ private void writeObject0(Object obj) throws IOException { if (obj == null) writeByte(NULL); else { - if (obj instanceof Throwable && !(obj instanceof Externalizable)) { + if (obj instanceof Throwable && !(obj instanceof Externalizable) || U.isEnum(obj.getClass())) { + // Avoid problems with differing Enum objects or Enum implementation class deadlocks. writeByte(JDK); try { 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 2defefad4721e..1e34c2d66a93e 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 @@ -6156,6 +6156,21 @@ public static boolean isJdk(Class cls) { return s.startsWith("java.") || s.startsWith("javax."); } + /** + * Check if given class represents a Enum. + * + * @param cls Class to check. + * @return {@code True} if this is a Enum class. + */ + public static boolean isEnum(Class cls) { + if (cls.isEnum()) + return true; + + Class sCls = cls.getSuperclass(); + + return sCls != null && sCls.isEnum(); + } + /** * Converts {@link InterruptedException} to {@link IgniteCheckedException}. * diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerEnumDeadlockMultiJvmTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerEnumDeadlockMultiJvmTest.java new file mode 100644 index 0000000000000..7042c03221d5f --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerEnumDeadlockMultiJvmTest.java @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.marshaller; + +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import javax.cache.configuration.Factory; +import org.apache.ignite.Ignite; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.binary.BinaryMarshaller; +import org.apache.ignite.internal.marshaller.optimized.OptimizedMarshaller; +import org.apache.ignite.lang.IgniteCallable; +import org.apache.ignite.marshaller.jdk.JdkMarshaller; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Contains test of Enum marshalling with various {@link Marshaller}s. See IGNITE-8547 for details. + */ +public class MarshallerEnumDeadlockMultiJvmTest extends GridCommonAbstractTest { + /** */ + private Factory marshFactory; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String instanceName) throws Exception { + return super.getConfiguration(instanceName).setMarshaller(marshFactory.create()); + } + + /** */ + public void testJdkMarshaller() throws Exception { + marshFactory = new JdkMarshallerFactory(); + + runRemoteUnmarshal(); + } + + /** */ + public void testOptimizedMarshaller() throws Exception { + marshFactory = new OptimizedMarshallerFactory(); + + runRemoteUnmarshal(); + } + + /** */ + public void testBinaryMarshaller() throws Exception { + marshFactory = new BinaryMarshallerFactory(); + + runRemoteUnmarshal(); + } + + /** */ + private void runRemoteUnmarshal() throws Exception { + Ignite ignite = startGrid(0); + + byte[] one = ignite.configuration().getMarshaller().marshal(DeclaredBodyEnum.ONE); + byte[] two = ignite.configuration().getMarshaller().marshal(DeclaredBodyEnum.TWO); + + startGrid(1); + + ignite.compute(ignite.cluster().forRemotes()).call(new UnmarshalCallable(one, two)); + } + + /** {@inheritDoc} */ + @Override protected boolean isMultiJvm() { + return true; + } + + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** */ + private static class OptimizedMarshallerFactory implements Factory { + /** {@inheritDoc} */ + @Override public Marshaller create() { + return new OptimizedMarshaller(false); + } + } + + /** */ + private static class BinaryMarshallerFactory implements Factory { + /** {@inheritDoc} */ + @Override public Marshaller create() { + return new BinaryMarshaller(); + } + } + + /** */ + private static class JdkMarshallerFactory implements Factory { + /** {@inheritDoc} */ + @Override public Marshaller create() { + return new JdkMarshaller(); + } + } + + /** + * Attempts to unmarshal both in-built and inner-class enum values at exactly the same time in multiple threads. + */ + private static class UnmarshalCallable implements IgniteCallable { + /** */ + private final byte[] one; + /** */ + private final byte[] two; + /** */ + @IgniteInstanceResource + private Ignite ign; + + /** */ + public UnmarshalCallable(byte[] one, byte[] two) { + this.one = one; + this.two = two; + } + + /** {@inheritDoc} */ + @Override public Object call() throws Exception { + ExecutorService executor = Executors.newFixedThreadPool(2); + + final CyclicBarrier start = new CyclicBarrier(2); + + for (int i = 0; i < 2; i++) { + final int ii = i; + + executor.execute(new Runnable() { + @Override public void run() { + try { + start.await(); + + if (ii == 0) + ign.configuration().getMarshaller().unmarshal(one, null); + else + ign.configuration().getMarshaller().unmarshal(two, null); + } + catch (Exception e) { + e.printStackTrace(); + } + } + }); + } + + try { + executor.shutdown(); + + executor.awaitTermination(5, TimeUnit.SECONDS); + + if (!executor.isTerminated()) + throw new IllegalStateException("Failed to wait for completion"); + } + catch (Exception te) { + throw new IllegalStateException("Failed to wait for completion", te); + } + + return null; + } + } + + /** */ + public enum DeclaredBodyEnum { + ONE, + TWO { + /** {@inheritDoc} */ + @Override public boolean isSupported() { + return false; + } + }; + + /** + * A bogus method. + */ + public boolean isSupported() { + return true; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteMarshallerSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteMarshallerSelfTestSuite.java index 2d7f64b50b74b..0eb50d739ff9e 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteMarshallerSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteMarshallerSelfTestSuite.java @@ -31,6 +31,7 @@ import org.apache.ignite.internal.util.io.GridUnsafeDataInputOutputByteOrderSelfTest; import org.apache.ignite.internal.util.io.GridUnsafeDataOutputArraySizingSelfTest; import org.apache.ignite.marshaller.GridMarshallerMappingConsistencyTest; +import org.apache.ignite.marshaller.MarshallerEnumDeadlockMultiJvmTest; import org.apache.ignite.marshaller.jdk.GridJdkMarshallerSelfTest; import org.apache.ignite.testframework.GridTestUtils; @@ -67,6 +68,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { GridTestUtils.addTestIfNeeded(suite, GridHandleTableSelfTest.class, ignoredTests); GridTestUtils.addTestIfNeeded(suite, OptimizedMarshallerPooledSelfTest.class, ignoredTests); GridTestUtils.addTestIfNeeded(suite, GridMarshallerMappingConsistencyTest.class, ignoredTests); + GridTestUtils.addTestIfNeeded(suite, MarshallerEnumDeadlockMultiJvmTest.class, ignoredTests); return suite; } From bbc6de06ed23cdff96342b1dbdc4359fa4ccddaf Mon Sep 17 00:00:00 2001 From: Sergey Chugunov Date: Tue, 29 May 2018 18:39:02 +0300 Subject: [PATCH 0202/1463] IGNITE-8636 debug logging for client join scenario. - Fixes #4087. Signed-off-by: Dmitriy Pavlov (cherry-picked from commit #79278e074e313e9ca23b5ccc97926e1f5d5cc031) --- .../GridCachePartitionExchangeManager.java | 10 +++++-- .../GridDhtPartitionsExchangeFuture.java | 26 ++++++++++++++++++- 2 files changed, 33 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index c3a0add55dee9..6533bbc5f9f89 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 @@ -1552,8 +1552,14 @@ else if (!grp.isLocal()) if (updated) scheduleResendPartitions(); } - else - exchangeFuture(msg.exchangeId(), null, null, null, null).onReceiveSingleMessage(node, msg); + else { + GridDhtPartitionsExchangeFuture exchFut = exchangeFuture(msg.exchangeId(), null, null, null, null); + + if (log.isDebugEnabled()) + log.debug("Notifying exchange future about single message: " + exchFut); + + exchFut.onReceiveSingleMessage(node, msg); + } } finally { leaveBusy(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java index 0acf41ba50fd3..944dce014115a 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 @@ -2068,6 +2068,9 @@ public void onReceiveSingleMessage(final ClusterNode node, final GridDhtPartitio if (mergedWith0 != null) { mergedWith0.processMergedMessage(node, msg); + if (log.isDebugEnabled()) + log.debug("Merged message processed, message handling finished: " + msg); + return; } @@ -2093,6 +2096,9 @@ public void onReceiveSingleMessage(final ClusterNode node, final GridDhtPartitio * @param msg Client's message. */ public void waitAndReplyToNode(final UUID nodeId, final GridDhtPartitionsSingleMessage msg) { + if (log.isDebugEnabled()) + log.debug("Single message will be handled on completion of exchange future: " + this); + listen(new CI1>() { @Override public void apply(IgniteInternalFuture fut) { if (cctx.kernalContext().isStopping()) @@ -2109,8 +2115,17 @@ public void waitAndReplyToNode(final UUID nodeId, final GridDhtPartitionsSingleM ClusterNode node = cctx.node(nodeId); - if (node == null) + if (node == null) { + if (log.isDebugEnabled()) { + log.debug("No node found for nodeId: " + + nodeId + + ", handling of single message will be stopped: " + + msg + ); + } + return; + } finishState0 = new FinishState(cctx.localNodeId(), initialVersion(), @@ -2177,6 +2192,8 @@ private void processSingleMessage(UUID nodeId, GridDhtPartitionsSingleMessage ms ", allReceived=" + allReceived + ']'); } } + else if (log.isDebugEnabled()) + log.debug("Coordinator received single message it didn't expect to receive: " + msg); break; } @@ -2855,6 +2872,13 @@ private void sendAllPartitionsToNode(FinishState finishState, GridDhtPartitionsS try { cctx.io().send(node, fullMsg, SYSTEM_POOL); + + if (log.isDebugEnabled()) { + log.debug("Full message was sent to node: " + + node + + ", fullMsg: " + fullMsg + ); + } } catch (ClusterTopologyCheckedException e) { if (log.isDebugEnabled()) From 364977fa49a9517cf7798e7414a156c2efd70c1d Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 16 Apr 2018 20:43:36 +0300 Subject: [PATCH 0203/1463] IGNITE-7972 Fixed NPE in TTL manager on unwindEvicts. - Fixes #3810. Signed-off-by: dpavlov (cherry picked from commit 737933e) --- .../internal/processors/cache/GridCacheTtlManager.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java index b6f54a14748ee..d36485ab70a7f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java @@ -40,8 +40,8 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter { /** Entries pending removal. */ private GridConcurrentSkipListSetEx pendingEntries; - /** */ - private boolean eagerTtlEnabled; + /** See {@link CacheConfiguration#isEagerTtl()}. */ + private volatile boolean eagerTtlEnabled; /** */ private GridCacheContext dhtCtx; @@ -166,6 +166,12 @@ public void expire() { * @return {@code True} if unprocessed expired entries remains. */ public boolean expire(int amount) { + // TTL manager is not initialized or eagerTtl disabled for cache. + if (!eagerTtlEnabled) + return false; + + assert cctx != null; + long now = U.currentTimeMillis(); try { From 131d4da5eeef8cc6988c83d92469b974c8d8cbe7 Mon Sep 17 00:00:00 2001 From: Ivan Daschinskiy Date: Tue, 29 May 2018 17:38:38 +0300 Subject: [PATCH 0204/1463] IGNITE-8624 Added reproducer of IGNITE-7972 issue. - Fixes #4077. Signed-off-by: Dmitriy Pavlov (cherry picked from commit d191cef) --- .../transactions/TxOnCachesStartTest.java | 141 ++++++++++++++++++ .../testsuites/IgniteCacheTestSuite6.java | 9 +- 2 files changed, 147 insertions(+), 3 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOnCachesStartTest.java diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOnCachesStartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOnCachesStartTest.java new file mode 100644 index 0000000000000..24044040cac8b --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOnCachesStartTest.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.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 org.apache.ignite.transactions.Transaction; + +/** + * Tests transactions closes correctly while other caches start and stop. + * Tests possible {@link NullPointerException} in {@link TransactionProxyImpl#leave} due to race while + * {@link org.apache.ignite.internal.processors.cache.GridCacheTtlManager} initializes (IGNITE-7972). + */ +public class TxOnCachesStartTest extends GridCommonAbstractTest { + /** */ + private static int NUM_CACHES = 100; + + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testTransactionCloseOnCachesStartAndStop() throws Exception { + Ignite srv = startGrids(5); + + IgniteEx client1 = startGrid(getConfiguration("client-1").setClientMode(true)); + + srv.cluster().active(true); + + CountDownLatch latch = new CountDownLatch(1); + + AtomicReference ex = new AtomicReference<>(null); + + IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(new Runnable() { + @Override public void run() { + for (int i = 0; i < NUM_CACHES; i++) { + IgniteCache cache = client1.getOrCreateCache(testCacheConfiguration(DEFAULT_CACHE_NAME + i)); + + try { + U.sleep(100); + } + catch (Exception e) { + //Ignore. + } + + cache.destroy(); + } + } + }, 1, "tx-thread"); + + GridTestUtils.runMultiThreadedAsync(new Runnable() { + @Override public void run() { + while (true) { + try (Transaction tx = client1.transactions().txStart()) { + /** Empty transaction, just testing {@link TransactionProxyImpl#leave} */ + } + catch (NullPointerException e) { + e.printStackTrace(); + + ex.compareAndSet(null, e); + + latch.countDown(); + + break; + } + } + } + }, 1, "tx-thread"); + + latch.await(5, TimeUnit.SECONDS); + + fut.cancel(); + + assertNull("NullPointerException thrown while closing transaction", ex.get()); + } + + /** + * Get cache configuration for tests. + * + * @param name Name. + */ + private CacheConfiguration testCacheConfiguration(String name) { + return new CacheConfiguration() + .setGroupName("default-group") + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL) + .setName(name); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java index fb0daa9f67396..66c1c488cf447 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java @@ -44,14 +44,15 @@ import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsCacheAssignmentNodeRestartsTest; import org.apache.ignite.internal.processors.cache.transactions.TxLabelTest; import org.apache.ignite.internal.processors.cache.transactions.TxMultiCacheAsyncOpsTest; +import org.apache.ignite.internal.processors.cache.transactions.TxOnCachesStartTest; +import org.apache.ignite.internal.processors.cache.transactions.TxOptimisticOnPartitionExchangeTest; import org.apache.ignite.internal.processors.cache.transactions.TxOptimisticPrepareOnUnstableTopologyTest; +import org.apache.ignite.internal.processors.cache.transactions.TxRollbackAsyncNearCacheTest; +import org.apache.ignite.internal.processors.cache.transactions.TxRollbackAsyncTest; import org.apache.ignite.internal.processors.cache.transactions.TxRollbackAsyncWithPersistenceTest; -import org.apache.ignite.internal.processors.cache.transactions.TxOptimisticOnPartitionExchangeTest; import org.apache.ignite.internal.processors.cache.transactions.TxRollbackOnTimeoutNearCacheTest; import org.apache.ignite.internal.processors.cache.transactions.TxRollbackOnTimeoutNoDeadlockDetectionTest; import org.apache.ignite.internal.processors.cache.transactions.TxRollbackOnTimeoutTest; -import org.apache.ignite.internal.processors.cache.transactions.TxRollbackAsyncNearCacheTest; -import org.apache.ignite.internal.processors.cache.transactions.TxRollbackAsyncTest; import org.apache.ignite.internal.processors.cache.transactions.TxRollbackOnTopologyChangeTest; /** @@ -89,6 +90,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(TxMultiCacheAsyncOpsTest.class); + suite.addTestSuite(TxOnCachesStartTest.class); + suite.addTestSuite(IgnitePdsCacheAssignmentNodeRestartsTest.class); suite.addTestSuite(WalModeChangeSelfTest.class); From 9d1a9c5bf3d28487d2b4cceafea7fb0ab9f36003 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Wed, 30 May 2018 16:55:46 +0700 Subject: [PATCH 0205/1463] IGNITE-8568 Added support for "Collocated" query mode. (cherry picked from commit da90769) --- .../internal/visor/query/VisorQueryTask.java | 1 + .../visor/query/VisorQueryTaskArg.java | 69 +++++++++++++++++-- 2 files changed, 64 insertions(+), 6 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java index 2e322763dc703..b5af1b04b5151 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java @@ -79,6 +79,7 @@ private VisorQueryJob(VisorQueryTaskArg arg, boolean debug) { qry.setPageSize(arg.getPageSize()); qry.setLocal(arg.isLocal()); qry.setDistributedJoins(arg.isDistributedJoins()); + qry.setCollocated(arg.isCollocated()); qry.setEnforceJoinOrder(arg.isEnforceJoinOrder()); qry.setReplicatedOnly(arg.isReplicatedOnly()); qry.setLazy(arg.getLazy()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTaskArg.java index e9428809d1128..5220b02632bb1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTaskArg.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTaskArg.java @@ -55,6 +55,9 @@ public class VisorQueryTaskArg extends VisorDataTransferObject { /** Lazy query execution flag */ private boolean lazy; + /** Collocation flag. */ + private boolean collocated; + /** * Default constructor. */ @@ -71,9 +74,16 @@ public VisorQueryTaskArg() { * @param loc Flag whether to execute query locally. * @param pageSize Result batch size. */ - public VisorQueryTaskArg(String cacheName, String qryTxt, boolean distributedJoins, - boolean enforceJoinOrder, boolean replicatedOnly, boolean loc, int pageSize) { - this(cacheName, qryTxt, distributedJoins, enforceJoinOrder, replicatedOnly, loc, pageSize, false); + public VisorQueryTaskArg( + String cacheName, + String qryTxt, + boolean distributedJoins, + boolean enforceJoinOrder, + boolean replicatedOnly, + boolean loc, + int pageSize + ) { + this(cacheName, qryTxt, distributedJoins, enforceJoinOrder, replicatedOnly, loc, pageSize, false, false); } /** @@ -86,8 +96,41 @@ public VisorQueryTaskArg(String cacheName, String qryTxt, boolean distributedJoi * @param pageSize Result batch size. * @param lazy Lazy query execution flag. */ - public VisorQueryTaskArg(String cacheName, String qryTxt, boolean distributedJoins, - boolean enforceJoinOrder, boolean replicatedOnly, boolean loc, int pageSize, boolean lazy) { + public VisorQueryTaskArg( + String cacheName, + String qryTxt, + boolean distributedJoins, + boolean enforceJoinOrder, + boolean replicatedOnly, + boolean loc, + int pageSize, + boolean lazy + ) { + this(cacheName, qryTxt, distributedJoins, enforceJoinOrder, replicatedOnly, loc, pageSize, lazy, false); + } + + /** + * @param cacheName Cache name for query. + * @param qryTxt Query text. + * @param distributedJoins If {@code true} then distributed joins enabled. + * @param enforceJoinOrder If {@code true} then enforce join order. + * @param replicatedOnly {@code true} then query contains only replicated tables. + * @param loc Flag whether to execute query locally. + * @param pageSize Result batch size. + * @param lazy Lazy query execution flag. + * @param collocated Collocation flag. + */ + public VisorQueryTaskArg( + String cacheName, + String qryTxt, + boolean distributedJoins, + boolean enforceJoinOrder, + boolean replicatedOnly, + boolean loc, + int pageSize, + boolean lazy, + boolean collocated + ) { this.cacheName = cacheName; this.qryTxt = qryTxt; this.distributedJoins = distributedJoins; @@ -96,6 +139,7 @@ public VisorQueryTaskArg(String cacheName, String qryTxt, boolean distributedJoi this.loc = loc; this.pageSize = pageSize; this.lazy = lazy; + this.collocated = collocated; } /** @@ -156,9 +200,18 @@ public boolean getLazy() { return lazy; } + /** + * Flag indicating if this query is collocated. + * + * @return {@code true} If the query is collocated. + */ + public boolean isCollocated() { + return collocated; + } + /** {@inheritDoc} */ @Override public byte getProtocolVersion() { - return V2; + return V3; } /** {@inheritDoc} */ @@ -170,6 +223,7 @@ public boolean getLazy() { out.writeBoolean(loc); out.writeInt(pageSize); out.writeBoolean(lazy); + out.writeBoolean(collocated); } /** {@inheritDoc} */ @@ -183,6 +237,9 @@ public boolean getLazy() { if (protoVer > V1) lazy = in.readBoolean(); + + if (protoVer > V2) + collocated = in.readBoolean(); } /** {@inheritDoc} */ From e9b16df8d03f2a612971cc76f8f50a7ef5cabdf8 Mon Sep 17 00:00:00 2001 From: Vasiliy Sisko Date: Wed, 30 May 2018 16:53:32 +0700 Subject: [PATCH 0206/1463] IGNITE-8568 Web Console: Added support for "Collocated" query mode on "Queries" screen. (cherry picked from commit 4ef96dd) --- modules/web-console/backend/app/schemas.js | 3 ++- .../components/queries-notebook/controller.js | 23 +++++++++++++++---- .../queries-notebook/template.tpl.pug | 8 +++++++ .../queries-notebooks-list/controller.js | 8 +++---- .../app/modules/agent/AgentManager.service.js | 20 ++++++++++------ 5 files changed, 45 insertions(+), 17 deletions(-) diff --git a/modules/web-console/backend/app/schemas.js b/modules/web-console/backend/app/schemas.js index 3f37487827e06..5898ed88cb247 100644 --- a/modules/web-console/backend/app/schemas.js +++ b/modules/web-console/backend/app/schemas.js @@ -1138,7 +1138,8 @@ module.exports.factory = function(mongoose) { qryType: String, nonCollocatedJoins: {type: Boolean, default: false}, enforceJoinOrder: {type: Boolean, default: false}, - lazy: {type: Boolean, default: false} + lazy: {type: Boolean, default: false}, + collocated: Boolean }] }); 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 859d1f737db52..d4fb2ee7b53d2 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,6 +41,8 @@ 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 ENFORCE_JOIN_SINCE = [['1.7.9', '1.8.0'], ['1.8.4', '1.9.0'], '1.9.1']; const LAZY_QUERY_SINCE = [['2.1.4-p1', '2.2.0'], '2.2.1']; @@ -1406,7 +1408,7 @@ export class NotebookCtrl { .then(() => _closeOldQuery(paragraph)) .then(() => args.localNid || _chooseNode(args.cacheName, false)) .then((nid) => agentMgr.querySql(nid, args.cacheName, args.query, args.nonCollocatedJoins, - args.enforceJoinOrder, false, !!args.localNid, args.pageSize, args.lazy)) + args.enforceJoinOrder, false, !!args.localNid, args.pageSize, args.lazy, args.collocated)) .then((res) => _processQueryResult(paragraph, false, res)) .catch((err) => paragraph.setError(err)); }; @@ -1439,6 +1441,15 @@ export class NotebookCtrl { return false; }; + $scope.collocatedJoinsAvailable = (paragraph) => { + const cache = _.find($scope.caches, {name: paragraph.cacheName}); + + if (cache) + return !!_.find(cache.nodes, (node) => Version.since(node.version, ...COLLOCATED_QUERY_SINCE)); + + return false; + }; + $scope.enforceJoinOrderAvailable = (paragraph) => { const cache = _.find($scope.caches, {name: paragraph.cacheName}); @@ -1474,6 +1485,7 @@ export class NotebookCtrl { const nonCollocatedJoins = !!paragraph.nonCollocatedJoins; const enforceJoinOrder = !!paragraph.enforceJoinOrder; const lazy = !!paragraph.lazy; + const collocated = !!paragraph.collocated; $scope.queryAvailable(paragraph) && _chooseNode(paragraph.cacheName, local) .then((nid) => { @@ -1499,14 +1511,15 @@ export class NotebookCtrl { nonCollocatedJoins, enforceJoinOrder, localNid: local ? nid : null, - lazy + lazy, + collocated }; ActivitiesData.post({ action: '/queries/execute' }); const qry = args.maxPages ? addLimit(args.query, args.pageSize * args.maxPages) : query; - return agentMgr.querySql(nid, args.cacheName, qry, nonCollocatedJoins, enforceJoinOrder, false, local, args.pageSize, lazy); + return agentMgr.querySql(nid, args.cacheName, qry, nonCollocatedJoins, enforceJoinOrder, false, local, args.pageSize, lazy, collocated); }) .then((res) => { _processQueryResult(paragraph, true, res); @@ -1559,7 +1572,7 @@ export class NotebookCtrl { ActivitiesData.post({ action: '/queries/explain' }); - return agentMgr.querySql(nid, args.cacheName, args.query, false, !!paragraph.enforceJoinOrder, false, false, args.pageSize, false); + return agentMgr.querySql(nid, args.cacheName, args.query, args.nonCollocatedJoins, !!paragraph.enforceJoinOrder, false, false, args.pageSize, false, args.collocated); }) .then((res) => _processQueryResult(paragraph, true, res)) .catch((err) => { @@ -1753,7 +1766,7 @@ export class NotebookCtrl { return Promise.resolve(args.localNid || _chooseNode(args.cacheName, false)) .then((nid) => args.type === 'SCAN' ? agentMgr.queryScanGetAll(nid, args.cacheName, args.query, !!args.regEx, !!args.caseSensitive, !!args.near, !!args.localNid) - : agentMgr.querySqlGetAll(nid, args.cacheName, args.query, !!args.nonCollocatedJoins, !!args.enforceJoinOrder, false, !!args.localNid, !!args.lazy)) + : agentMgr.querySqlGetAll(nid, args.cacheName, args.query, !!args.nonCollocatedJoins, !!args.enforceJoinOrder, false, !!args.localNid, !!args.lazy, !!args.collocated)) .then((res) => _export(exportFileName(paragraph, true), paragraph.gridOptions.columnDefs, res.columns, res.rows)) .catch(Messages.showError) .then(() => { diff --git a/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/template.tpl.pug b/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/template.tpl.pug index 1fef75c34ff71..4dbb36ecf461f 100644 --- a/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/template.tpl.pug +++ b/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/template.tpl.pug @@ -117,6 +117,14 @@ mixin query-settings NOTE: In some cases it may consume more heap memory or may take a long time than collocated joins.' data-trigger='hover') input(type='checkbox' ng-model='paragraph.nonCollocatedJoins') span Allow non-collocated joins + .row(ng-if='collocatedJoinsAvailable(paragraph)') + label.tipLabel(bs-tooltip data-placement='bottom' data-title='Used For Optimization Purposes Of Queries With GROUP BY Statements.
    \ + NOTE: Whenever Ignite executes a distributed query, it sends sub-queries to individual cluster members.
    \ + If you know in advance that the elements of your query selection are collocated together on the same node\ + and you group by collocated key (primary or affinity key), then Ignite can make significant performance and\ + network optimizations by grouping data on remote nodes.' data-trigger='hover') + input(type='checkbox' ng-model='paragraph.collocated') + span Collocated Query .row(ng-if='enforceJoinOrderAvailable(paragraph)') label.tipLabel(bs-tooltip data-placement='bottom' data-title='Enforce join order of tables in the query.
    \ If set, then query optimizer will not reorder tables within join.
    \ diff --git a/modules/web-console/frontend/app/components/page-queries/components/queries-notebooks-list/controller.js b/modules/web-console/frontend/app/components/page-queries/components/queries-notebooks-list/controller.js index 2e4146ce798c3..ad86a1006d3f8 100644 --- a/modules/web-console/frontend/app/components/page-queries/components/queries-notebooks-list/controller.js +++ b/modules/web-console/frontend/app/components/page-queries/components/queries-notebooks-list/controller.js @@ -33,14 +33,14 @@ export class NotebooksListCtrl { const categories = [ { name: 'Name', visible: true, enableHiding: false }, - { name: 'SQL Query', visible: true, enableHiding: false }, - { name: 'Scan Query', visible: true, enableHiding: false } + { name: 'SQL Queries', visible: true, enableHiding: false }, + { name: 'Scan Queries', visible: true, enableHiding: false } ]; const columnDefs = [ { name: 'name', displayName: 'Notebook name', categoryDisplayName: 'Name', field: 'name', cellTemplate: notebookNameTemplate, pinnedLeft: true, filter: { placeholder: 'Filter by Name...' } }, - { name: 'sqlQueryNum', displayName: 'SQL Query', categoryDisplayName: 'SQL Query', field: 'sqlQueriesParagraphsLength', cellTemplate: sqlQueryTemplate, enableSorting: true, type: 'number', minWidth: 150, width: 150, enableFiltering: false }, - { name: 'scanQueryNum', displayName: 'Scan Query', categoryDisplayName: 'Scan Query', field: 'scanQueriesParagraphsLength', cellTemplate: scanQueryTemplate, enableSorting: true, type: 'number', minWidth: 150, width: 150, enableFiltering: false } + { name: 'sqlQueryNum', displayName: 'SQL Queries', categoryDisplayName: 'SQL Queries', field: 'sqlQueriesParagraphsLength', cellTemplate: sqlQueryTemplate, enableSorting: true, type: 'number', minWidth: 150, width: 150, enableFiltering: false }, + { name: 'scanQueryNum', displayName: 'Scan Queries', categoryDisplayName: 'Scan Queries', field: 'scanQueriesParagraphsLength', cellTemplate: scanQueryTemplate, enableSorting: true, type: 'number', minWidth: 150, width: 150, enableFiltering: false } ]; this.gridOptions = { diff --git a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js index b73b2c15c6866..fd44f9e4188ac 100644 --- a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js +++ b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js @@ -32,6 +32,7 @@ const State = { }; const LAZY_QUERY_SINCE = [['2.1.4-p1', '2.2.0'], '2.2.1']; +const COLLOCATED_QUERY_SINCE = [['2.3.5', '2.4.0'], ['2.4.6', '2.5.0'], '2.5.2']; class ConnectionState { constructor(cluster) { @@ -557,15 +558,19 @@ export default class IgniteAgentManager { * @param {Boolean} local Flag whether to execute query locally. * @param {int} pageSz * @param {Boolean} lazy query flag. + * @param {Boolean} collocated Collocated query. * @returns {Promise} */ - querySql(nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, replicatedOnly, local, pageSz, lazy) { + querySql(nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, replicatedOnly, local, pageSz, lazy, collocated) { if (this.available('2.0.0')) { - const task = this.available(...LAZY_QUERY_SINCE) ? - this.visorTask('querySqlX2', nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, replicatedOnly, local, pageSz, lazy) : - this.visorTask('querySqlX2', nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, replicatedOnly, local, pageSz); + let args = [cacheName, query, nonCollocatedJoins, enforceJoinOrder, replicatedOnly, local, pageSz]; - return task.then(({error, result}) => { + if (this.available(COLLOCATED_QUERY_SINCE)) + args = [...args, lazy, collocated]; + else if (this.available(...LAZY_QUERY_SINCE)) + args = [...args, lazy]; + + return this.visorTask('querySqlX2', nid, ...args).then(({error, result}) => { if (_.isEmpty(error)) return result; @@ -615,9 +620,10 @@ export default class IgniteAgentManager { * @param {Boolean} replicatedOnly Flag whether query contains only replicated tables. * @param {Boolean} local Flag whether to execute query locally. * @param {Boolean} lazy query flag. + * @param {Boolean} collocated Collocated query. * @returns {Promise} */ - querySqlGetAll(nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, replicatedOnly, local, lazy) { + querySqlGetAll(nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, replicatedOnly, local, lazy, collocated) { // Page size for query. const pageSz = 1024; @@ -635,7 +641,7 @@ export default class IgniteAgentManager { }); }; - return this.querySql(nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, replicatedOnly, local, pageSz, lazy) + return this.querySql(nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, replicatedOnly, local, pageSz, lazy, collocated) .then(fetchResult); } From 6a24ab750b600f049a7396478e545e7d872e68f2 Mon Sep 17 00:00:00 2001 From: Aleksei Scherbakov Date: Thu, 31 May 2018 16:38:33 +0300 Subject: [PATCH 0207/1463] IGNITE-8651 VisrTxTask fails when printing transactions having implicit single type - Fixes #4096. Signed-off-by: Alexey Goncharuk --- .../ignite/internal/visor/tx/VisorTxTask.java | 12 ++++- .../ignite/util/GridCommandHandlerTest.java | 47 +++++++++++++------ 2 files changed, 43 insertions(+), 16 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java index 72b174050c72c..b411e29006757 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java @@ -21,6 +21,7 @@ import java.util.Collection; import java.util.Collections; import java.util.Comparator; +import java.util.ConcurrentModificationException; import java.util.List; import java.util.Map; import java.util.TreeMap; @@ -32,6 +33,7 @@ import org.apache.ignite.compute.ComputeJobResult; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxMapping; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; +import org.apache.ignite.internal.processors.cache.distributed.near.IgniteTxMappings; import org.apache.ignite.internal.processors.cache.transactions.TransactionProxyImpl; import org.apache.ignite.internal.processors.task.GridInternal; import org.apache.ignite.internal.util.typedef.F; @@ -179,10 +181,16 @@ private VisorTxJob(VisorTxTaskArg arg, boolean debug) { int size = 0; if (locTx.mappings() != null) { - for (GridDistributedTxMapping mapping : locTx.mappings().mappings()) { + IgniteTxMappings txMappings = locTx.mappings(); + + for (GridDistributedTxMapping mapping : + txMappings.single() ? Collections.singleton(txMappings.singleMapping()) : txMappings.mappings()) { + if (mapping == null) + continue; + mappings.add(mapping.primary().id()); - size += mapping.entries().size(); // Entries are not synchronized so no visibility guaranties. + size += mapping.entries().size(); // Entries are not synchronized so no visibility guaranties for size. } } diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java index 670c22c313b82..385e79b3926fa 100644 --- a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java @@ -22,6 +22,7 @@ import java.io.PrintStream; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.HashSet; import java.util.Map; import java.util.TreeMap; @@ -47,6 +48,7 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.commandline.CommandHandler; import org.apache.ignite.internal.commandline.cache.CacheCommand; +import org.apache.ignite.internal.processors.cache.GridCacheFuture; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.X; @@ -402,7 +404,7 @@ public void testActiveTransactions() throws Exception { for (VisorTxInfo info : res.getInfos()) { if (info.getSize() == 100) { - toKill[0] = info; + toKill[0] = info; // Store for further use. break; } @@ -411,7 +413,7 @@ public void testActiveTransactions() throws Exception { assertEquals(3, map.size()); }, "--tx"); - assertNotNull(toKill); + assertNotNull(toKill[0]); // Test filter by label. validate(h, map -> { @@ -460,21 +462,18 @@ else if (entry.getKey().equals(node2)) { }, "--tx", "order", "DURATION"); // Trigger topology change and test connection. - IgniteInternalFuture startFut = multithreadedAsync(new Runnable() { - @Override public void run() { - try { - startGrid(2); - } - catch (Exception e) { - fail(); - } + IgniteInternalFuture startFut = multithreadedAsync(() -> { + try { + startGrid(2); + } + catch (Exception e) { + fail(); } }, 1, "start-node-thread"); - doSleep(5000); + doSleep(5000); // Give enough time to reach exchange future. - assertEquals(EXIT_CODE_OK, execute(h, "--host", "127.0.0.1", "--port", "11211", "--tx")); - assertEquals(EXIT_CODE_OK, execute(h, "--host", "127.0.0.1", "--port", "11212", "--tx")); + assertEquals(EXIT_CODE_OK, execute(h, "--tx")); // Test kill by xid. validate(h, map -> { @@ -486,7 +485,7 @@ else if (entry.getKey().equals(node2)) { assertEquals(toKill[0].getXid(), info.getXid()); }, "--tx", "kill", - "xid", toKill[0].getXid().toString(), + "xid", toKill[0].getXid().toString(), // Use saved on first run value. "nodes", grid(0).localNode().consistentId().toString()); unlockLatch.countDown(); @@ -494,6 +493,10 @@ else if (entry.getKey().equals(node2)) { startFut.get(); fut.get(); + + awaitPartitionMapExchange(); + + checkFutures(); } /** @@ -889,4 +892,20 @@ private IgniteInternalFuture startTransactions(CountDownLatch lockLatch, Coun } }, 4, "tx-thread"); } + + /** + * Checks if all tx futures are finished. + */ + private void checkFutures() { + for (Ignite ignite : G.allGrids()) { + IgniteEx ig = (IgniteEx)ignite; + + final Collection> futs = ig.context().cache().context().mvcc().activeFutures(); + + for (GridCacheFuture fut : futs) + log.info("Waiting for future: " + fut); + + assertTrue("Expecting no active futures: node=" + ig.localNode().id(), futs.isEmpty()); + } + } } From 0b147cf9b8f8842aeedbb2da936f0ad437b0eff9 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Thu, 31 May 2018 16:56:14 +0300 Subject: [PATCH 0208/1463] IGNITE-8476 AssertionError exception occurs when trying to remove node from baseline under loading (cherry picked from commit 5e6e4e5) --- .../apache/ignite/internal/IgniteKernal.java | 18 +- .../ignite/internal/IgniteNodeAttributes.java | 3 + .../processors/cache/CacheGroupContext.java | 13 +- .../processors/cache/ClusterCachesInfo.java | 58 +- .../processors/cache/GridCacheProcessor.java | 1 + ...entAffinityAssignmentWithBaselineTest.java | 974 ++++++++++++++++++ .../testsuites/IgnitePdsTestSuite2.java | 2 + 7 files changed, 1059 insertions(+), 10 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/ClientAffinityAssignmentWithBaselineTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index a3e6ad9beed1f..0c4e386ab544f 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 @@ -105,9 +105,6 @@ import org.apache.ignite.internal.binary.BinaryUtils; import org.apache.ignite.internal.cluster.ClusterGroupAdapter; import org.apache.ignite.internal.cluster.IgniteClusterEx; -import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; -import org.apache.ignite.internal.processors.failure.FailureProcessor; import org.apache.ignite.internal.managers.GridManager; import org.apache.ignite.internal.managers.checkpoint.GridCheckpointManager; import org.apache.ignite.internal.managers.collision.GridCollisionManager; @@ -121,6 +118,7 @@ import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager; import org.apache.ignite.internal.marshaller.optimized.OptimizedMarshaller; import org.apache.ignite.internal.processors.GridProcessor; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor; import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor; import org.apache.ignite.internal.processors.cache.CacheConfigurationOverride; @@ -136,11 +134,13 @@ import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor; import org.apache.ignite.internal.processors.closure.GridClosureProcessor; import org.apache.ignite.internal.processors.cluster.ClusterProcessor; +import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; import org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor; import org.apache.ignite.internal.processors.cluster.IGridClusterStateProcessor; import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor; import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor; import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor; +import org.apache.ignite.internal.processors.failure.FailureProcessor; import org.apache.ignite.internal.processors.hadoop.Hadoop; import org.apache.ignite.internal.processors.hadoop.HadoopProcessorAdapter; import org.apache.ignite.internal.processors.job.GridJobProcessor; @@ -201,10 +201,10 @@ import org.apache.ignite.mxbean.ClusterMetricsMXBean; import org.apache.ignite.mxbean.IgniteMXBean; import org.apache.ignite.mxbean.StripedExecutorMXBean; -import org.apache.ignite.mxbean.WorkersControlMXBean; import org.apache.ignite.mxbean.ThreadPoolMXBean; -import org.apache.ignite.mxbean.TransactionsMXBean; import org.apache.ignite.mxbean.TransactionMetricsMxBean; +import org.apache.ignite.mxbean.TransactionsMXBean; +import org.apache.ignite.mxbean.WorkersControlMXBean; import org.apache.ignite.plugin.IgnitePlugin; import org.apache.ignite.plugin.PluginNotFoundException; import org.apache.ignite.plugin.PluginProvider; @@ -1582,7 +1582,13 @@ private void fillNodeAttributes(boolean notifyEnabled) throws IgniteCheckedExcep // Stick in network context into attributes. add(ATTR_IPS, (ips.isEmpty() ? "" : ips)); - add(ATTR_MACS, (macs.isEmpty() ? "" : macs)); + + Map userAttrs = configuration().getUserAttributes(); + + if (userAttrs != null && userAttrs.get(IgniteNodeAttributes.ATTR_MACS_OVERRIDE) != null) + add(ATTR_MACS, (Serializable)userAttrs.get(IgniteNodeAttributes.ATTR_MACS_OVERRIDE)); + else + add(ATTR_MACS, (macs.isEmpty() ? "" : macs)); // Stick in some system level attributes add(ATTR_JIT_NAME, U.getCompilerMx() == null ? "" : U.getCompilerMx().getName()); 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 073369fdd2efb..6a4beebac0cba 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 @@ -120,6 +120,9 @@ public final class IgniteNodeAttributes { /** Internal attribute name constant. */ public static final String ATTR_MACS = ATTR_PREFIX + ".macs"; + /** Allows to override {@link #ATTR_MACS} by adding this attribute in the user attributes. */ + public static final String ATTR_MACS_OVERRIDE = "override." + ATTR_MACS; + /** Internal attribute name constant. */ public static final String ATTR_PHY_RAM = ATTR_PREFIX + ".phy.ram"; 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 d1bdbb6342c3f..8a650380f2518 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 @@ -133,6 +133,9 @@ public class CacheGroupContext { /** */ private final DataRegion dataRegion; + /** Persistence enabled flag. */ + private final boolean persistenceEnabled; + /** */ private final CacheObjectContext cacheObjCtx; @@ -158,8 +161,8 @@ public class CacheGroupContext { private volatile boolean globalWalEnabled; /** - * @param grpId Group ID. * @param ctx Context. + * @param grpId Group ID. * @param rcvdFrom Node ID cache group was received from. * @param cacheType Cache type. * @param ccfg Cache configuration. @@ -169,6 +172,7 @@ public class CacheGroupContext { * @param freeList Free list. * @param reuseList Reuse list. * @param locStartVer Topology version when group was started on local node. + * @param persistenceEnabled Persistence enabled flag. * @param walEnabled Wal enabled flag. */ CacheGroupContext( @@ -183,7 +187,9 @@ public class CacheGroupContext { FreeList freeList, ReuseList reuseList, AffinityTopologyVersion locStartVer, - boolean walEnabled) { + boolean persistenceEnabled, + boolean walEnabled + ) { assert ccfg != null; assert dataRegion != null || !affNode; assert grpId != 0 : "Invalid group ID [cache=" + ccfg.getName() + ", grpName=" + ccfg.getGroupName() + ']'; @@ -200,6 +206,7 @@ public class CacheGroupContext { this.locStartVer = locStartVer; this.cacheType = cacheType; this.globalWalEnabled = walEnabled; + this.persistenceEnabled = persistenceEnabled; this.localWalEnabled = true; persistGlobalWalState(walEnabled); @@ -912,7 +919,7 @@ public void start() throws IgniteCheckedException { * @return Persistence enabled flag. */ public boolean persistenceEnabled() { - return dataRegion != null && dataRegion.config().isPersistenceEnabled(); + return persistenceEnabled; } /** 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 975617ee8dc0a..9ff84e7d510cd 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 @@ -38,10 +38,12 @@ import org.apache.ignite.cache.QueryEntity; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.NearCacheConfiguration; import org.apache.ignite.internal.GridCachePluginContext; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.internal.managers.discovery.DiscoCache; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage; @@ -60,6 +62,7 @@ import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.apache.ignite.plugin.CachePluginContext; import org.apache.ignite.plugin.CachePluginProvider; import org.apache.ignite.plugin.PluginProvider; @@ -1752,7 +1755,7 @@ private CacheGroupDescriptor registerCacheGroup( Map caches = Collections.singletonMap(startedCacheCfg.getName(), cacheId); - boolean persistent = CU.isPersistentCache(startedCacheCfg, ctx.config().getDataStorageConfiguration()); + boolean persistent = resolvePersistentFlag(exchActions, startedCacheCfg); CacheGroupDescriptor grpDesc = new CacheGroupDescriptor( startedCacheCfg, @@ -1781,6 +1784,59 @@ private CacheGroupDescriptor registerCacheGroup( return grpDesc; } + /** + * Resolves persistent flag for new cache group descriptor. + * + * @param exchActions Optional exchange actions to update if new group was added. + * @param startedCacheCfg Started cache configuration. + */ + private boolean resolvePersistentFlag(@Nullable ExchangeActions exchActions, CacheConfiguration startedCacheCfg) { + if (!ctx.clientNode()) { + // On server, we always can determine whether cache is persistent by local storage configuration. + return CU.isPersistentCache(startedCacheCfg, ctx.config().getDataStorageConfiguration()); + } + else if (exchActions == null) { + // It's either client local join event or cache is statically configured on another node. + // No need to resolve on client - we'll anyway receive group descriptor from server with correct flag. + return false; + } + else { + // Dynamic cache start. Initiator of the start may not have known whether cache should be persistent. + // On client, we should peek attributes of any affinity server node to get data storage configuration. + Collection aliveSrvNodes = ctx.discovery().aliveServerNodes(); + + assert !aliveSrvNodes.isEmpty() : "No alive server nodes"; + + for (ClusterNode srvNode : aliveSrvNodes) { + if (CU.affinityNode(srvNode, startedCacheCfg.getNodeFilter())) { + Object dsCfgBytes = srvNode.attribute(IgniteNodeAttributes.ATTR_DATA_STORAGE_CONFIG); + + if (dsCfgBytes instanceof byte[]) { + try { + DataStorageConfiguration crdDsCfg = new JdkMarshaller().unmarshal( + (byte[])dsCfgBytes, U.resolveClassLoader(ctx.config())); + + return CU.isPersistentCache(startedCacheCfg, crdDsCfg); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to unmarshal remote data storage configuration [remoteNode=" + + srvNode + ", cacheName=" + startedCacheCfg.getName() + "]", e); + } + } + else { + U.error(log, "Remote marshalled data storage configuration is absent [remoteNode=" + srvNode + + ", cacheName=" + startedCacheCfg.getName() + ", dsCfg=" + dsCfgBytes + "]"); + } + } + } + + U.error(log, "Failed to find affinity server node with data storage configuration for starting cache " + + "[cacheName=" + startedCacheCfg.getName() + ", aliveSrvNodes=" + aliveSrvNodes + "]"); + + return false; + } + } + /** * @param ccfg Cache configuration to start. * @throws IgniteCheckedException If failed. 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 8cd6b881e0614..68b684027f002 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 @@ -2038,6 +2038,7 @@ private CacheGroupContext startCacheGroup( freeList, reuseList, exchTopVer, + desc.persistenceEnabled(), desc.walEnabled() ); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/ClientAffinityAssignmentWithBaselineTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/ClientAffinityAssignmentWithBaselineTest.java new file mode 100644 index 0000000000000..15ec41557dfba --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/ClientAffinityAssignmentWithBaselineTest.java @@ -0,0 +1,974 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.baseline; + +import java.io.File; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.LockSupport; +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.CacheWriteSynchronizationMode; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cluster.BaselineNode; +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.internal.IgniteEx; +import org.apache.ignite.internal.IgniteNodeAttributes; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgnitePredicate; +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 static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; + +/** + * Checks that client affinity assignment cache is calculated correctly regardless of current baseline topology. + */ +public class ClientAffinityAssignmentWithBaselineTest extends GridCommonAbstractTest { + /** Nodes count. */ + private static final int DEFAULT_NODES_COUNT = 5; + + /** Tx cache name. */ + private static final String PARTITIONED_TX_CACHE_NAME = "p-tx-cache"; + + /** Tx cache name with shifted affinity. */ + private static final String PARTITIONED_TX_PRIM_SYNC_CACHE_NAME = "prim-sync"; + + /** Tx cache name from client static configuration. */ + private static final String PARTITIONED_TX_CLIENT_CACHE_NAME = "p-tx-client-cache"; + + /** Atomic cache name. */ + private static final String PARTITIONED_ATOMIC_CACHE_NAME = "p-atomic-cache"; + + /** Tx cache name. */ + private static final String REPLICATED_TX_CACHE_NAME = "r-tx-cache"; + + /** Atomic cache name. */ + private static final String REPLICATED_ATOMIC_CACHE_NAME = "r-atomic-cache"; + + /** Client grid name. */ + private static final String CLIENT_GRID_NAME = "client"; + + /** Flaky node name */ + private static final String FLAKY_NODE_NAME = "flaky"; + + /** Entries. */ + private static final int ENTRIES = 3_000; + + /** Flaky node wal path. */ + public static final String FLAKY_WAL_PATH = "flakywal"; + + /** Flaky node wal archive path. */ + public static final String FLAKY_WAL_ARCHIVE_PATH = "flakywalarchive"; + + /** Flaky node storage path. */ + public static final String FLAKY_STORAGE_PATH = "flakystorage"; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + if (igniteInstanceName.startsWith(CLIENT_GRID_NAME)) { + // Intentionally skipping data storage in client configuration. + cfg.setClientMode(true); + } + else { + cfg.setDataStorageConfiguration( + new DataStorageConfiguration() + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration() + .setPersistenceEnabled(true) + .setMaxSize(200 * 1024 * 1024) + ) + ); + } + + if (igniteInstanceName.contains(FLAKY_NODE_NAME)) { + File store = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false); + + cfg.getDataStorageConfiguration().setWalPath(new File(store, FLAKY_WAL_PATH).getAbsolutePath()); + cfg.getDataStorageConfiguration().setWalArchivePath(new File(store, FLAKY_WAL_ARCHIVE_PATH).getAbsolutePath()); + cfg.getDataStorageConfiguration().setStoragePath(new File(store, FLAKY_STORAGE_PATH).getAbsolutePath()); + } + + cfg.setConsistentId(igniteInstanceName); + + List srvConfigs = new ArrayList<>(); + srvConfigs.add(cacheConfig(PARTITIONED_TX_CACHE_NAME)); + srvConfigs.add(cacheConfig(PARTITIONED_TX_PRIM_SYNC_CACHE_NAME)); + srvConfigs.add(cacheConfig(REPLICATED_ATOMIC_CACHE_NAME)); + + List clientConfigs = new ArrayList<>(srvConfigs); + + // Skip some configs in client static configuration to check that clients receive correct cache descriptors. + srvConfigs.add(cacheConfig(PARTITIONED_ATOMIC_CACHE_NAME)); + srvConfigs.add(cacheConfig(REPLICATED_TX_CACHE_NAME)); + + // Skip config in server static configuration to check that caches received on client join start correctly. + clientConfigs.add(cacheConfig(PARTITIONED_TX_CLIENT_CACHE_NAME)); + + if (igniteInstanceName.startsWith(CLIENT_GRID_NAME)) + cfg.setCacheConfiguration(clientConfigs.toArray(new CacheConfiguration[clientConfigs.size()])); + else + cfg.setCacheConfiguration(srvConfigs.toArray(new CacheConfiguration[srvConfigs.size()])); + + // Enforce different mac adresses to emulate distributed environment by default. + cfg.setUserAttributes(Collections.singletonMap( + IgniteNodeAttributes.ATTR_MACS_OVERRIDE, UUID.randomUUID().toString())); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + cleanPersistenceDir(); + + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** + * @param cacheName Cache name. + */ + private CacheConfiguration cacheConfig(String cacheName) { + CacheConfiguration cfg = new CacheConfiguration<>(); + + if (PARTITIONED_ATOMIC_CACHE_NAME.equals(cacheName)) { + cfg.setName(PARTITIONED_ATOMIC_CACHE_NAME); + cfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); + cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + cfg.setAffinity(new RendezvousAffinityFunction(false, 32)); + cfg.setBackups(2); + } + else if (PARTITIONED_TX_CACHE_NAME.equals(cacheName)) { + cfg.setName(PARTITIONED_TX_CACHE_NAME); + cfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + cfg.setAffinity(new RendezvousAffinityFunction(false, 32)); + cfg.setBackups(2); + } + else if (PARTITIONED_TX_CLIENT_CACHE_NAME.equals(cacheName)) { + cfg.setName(PARTITIONED_TX_CLIENT_CACHE_NAME); + cfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + cfg.setAffinity(new RendezvousAffinityFunction(false, 32)); + cfg.setBackups(2); + } + else if (PARTITIONED_TX_PRIM_SYNC_CACHE_NAME.equals(cacheName)) { + cfg.setName(PARTITIONED_TX_PRIM_SYNC_CACHE_NAME); + cfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC); + cfg.setAffinity(new RendezvousAffinityFunction(false, 41)); // To break collocation. + cfg.setBackups(2); + } + else if (REPLICATED_ATOMIC_CACHE_NAME.equals(cacheName)) { + cfg.setName(REPLICATED_ATOMIC_CACHE_NAME); + cfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); + cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + cfg.setAffinity(new RendezvousAffinityFunction(false, 32)); + cfg.setCacheMode(CacheMode.REPLICATED); + } + else if (REPLICATED_TX_CACHE_NAME.equals(cacheName)) { + cfg.setName(REPLICATED_TX_CACHE_NAME); + cfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); + cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + cfg.setAffinity(new RendezvousAffinityFunction(false, 32)); + cfg.setCacheMode(CacheMode.REPLICATED); + } + else + throw new IllegalArgumentException("Unexpected cache name"); + + return cfg; + } + + /** + * + */ + public void testPartitionedAtomicCache() throws Exception { + testChangingBaselineDown(PARTITIONED_ATOMIC_CACHE_NAME, false); + } + + /** + * + */ + public void testPartitionedTxCache() throws Exception { + testChangingBaselineDown(PARTITIONED_TX_CACHE_NAME, false); + } + + /** + * Test that activation after client join won't break cache. + */ + public void testLateActivation() throws Exception { + testChangingBaselineDown(PARTITIONED_TX_CACHE_NAME, true); + } + + /** + * + */ + public void testReplicatedAtomicCache() throws Exception { + testChangingBaselineDown(REPLICATED_ATOMIC_CACHE_NAME, false); + } + + /** + * + */ + public void testReplicatedTxCache() throws Exception { + testChangingBaselineDown(REPLICATED_TX_CACHE_NAME, false); + } + + /** + * Tests that changing baseline down under load won't break cache. + */ + private void testChangingBaselineDown(String cacheName, boolean lateActivation) throws Exception { + IgniteEx ig0 = (IgniteEx)startGrids(DEFAULT_NODES_COUNT); + + IgniteEx client1 = null; + IgniteEx client2 = null; + + if (lateActivation) { + client1 = (IgniteEx)startGrid("client1"); + client2 = (IgniteEx)startGrid("client2"); + } + else + ig0.cluster().active(true); + + AtomicBoolean stopLoad = new AtomicBoolean(false); + + AtomicReference loadError = new AtomicReference<>(null); + + if (lateActivation) + ig0.cluster().active(true); + + IgniteCache cache = ig0.cache(cacheName); + + System.out.println("### Starting preloading"); + + for (int i = 0; i < ENTRIES; i++) { + ThreadLocalRandom r = ThreadLocalRandom.current(); + + byte[] randBytes = new byte[r.nextInt(10, 100)]; + + cache.put(r.nextInt(ENTRIES), new String(randBytes)); + } + + System.out.println("### Preloading is finished"); + + if (!lateActivation) { + client1 = (IgniteEx)startGrid("client1"); + client2 = (IgniteEx)startGrid("client2"); + } + + ConcurrentMap threadProgressTracker = new ConcurrentHashMap<>(); + + startSimpleLoadThread(client1, cacheName, stopLoad, loadError, threadProgressTracker); + startSimpleLoadThread(client1, cacheName, stopLoad, loadError, threadProgressTracker); + startSimpleLoadThread(client1, cacheName, stopLoad, loadError, threadProgressTracker); + startTxLoadThread(client2, cacheName, stopLoad, loadError, threadProgressTracker); + startTxLoadThread(client2, cacheName, stopLoad, loadError, threadProgressTracker); + startTxLoadThread(client2, cacheName, stopLoad, loadError, threadProgressTracker); + + awaitProgressInAllLoaders(10_000, loadError, threadProgressTracker); + + List fullBlt = new ArrayList<>(); + for (int i = 0; i < DEFAULT_NODES_COUNT; i++) + fullBlt.add(grid(i).localNode()); + + stopGrid(DEFAULT_NODES_COUNT - 1, true); + stopGrid(DEFAULT_NODES_COUNT - 2, true); + + awaitProgressInAllLoaders(10_000, loadError, threadProgressTracker); + + tryChangeBaselineDown(ig0, fullBlt, DEFAULT_NODES_COUNT - 1, loadError, threadProgressTracker); + tryChangeBaselineDown(ig0, fullBlt, DEFAULT_NODES_COUNT - 2, loadError, threadProgressTracker); + + stopLoad.set(true); + } + + /** + * Tests that rejoin of baseline node with clear LFS under load won't break cache. + */ + public void testRejoinWithCleanLfs() throws Exception { + IgniteEx ig0 = (IgniteEx)startGrids(DEFAULT_NODES_COUNT - 1); + startGrid("flaky"); + + ig0.cluster().active(true); + + AtomicBoolean stopLoad = new AtomicBoolean(false); + + AtomicReference loadError = new AtomicReference<>(null); + + IgniteCache cache1 = ig0.cache(PARTITIONED_ATOMIC_CACHE_NAME); + IgniteCache cache2 = ig0.cache(PARTITIONED_TX_CACHE_NAME); + IgniteCache cache3 = ig0.cache(REPLICATED_ATOMIC_CACHE_NAME); + IgniteCache cache4 = ig0.cache(REPLICATED_TX_CACHE_NAME); + + System.out.println("### Starting preloading"); + + for (int i = 0; i < ENTRIES; i++) { + ThreadLocalRandom r = ThreadLocalRandom.current(); + + cache1.put(r.nextInt(ENTRIES), new String(new byte[r.nextInt(10, 100)])); + cache2.put(r.nextInt(ENTRIES), new String(new byte[r.nextInt(10, 100)])); + cache3.put(r.nextInt(ENTRIES), new String(new byte[r.nextInt(10, 100)])); + cache4.put(r.nextInt(ENTRIES), new String(new byte[r.nextInt(10, 100)])); + } + + System.out.println("### Preloading is finished"); + + IgniteEx client1 = (IgniteEx)startGrid("client1"); + IgniteEx client2 = (IgniteEx)startGrid("client2"); + + ConcurrentMap threadProgressTracker = new ConcurrentHashMap<>(); + + startSimpleLoadThread(client1, PARTITIONED_ATOMIC_CACHE_NAME, stopLoad, loadError, threadProgressTracker); + startSimpleLoadThread(client1, PARTITIONED_TX_CACHE_NAME, stopLoad, loadError, threadProgressTracker); + startSimpleLoadThread(client1, REPLICATED_ATOMIC_CACHE_NAME, stopLoad, loadError, threadProgressTracker); + startTxLoadThread(client2, PARTITIONED_ATOMIC_CACHE_NAME, stopLoad, loadError, threadProgressTracker); + startTxLoadThread(client2, PARTITIONED_TX_CACHE_NAME, stopLoad, loadError, threadProgressTracker); + startTxLoadThread(client2, REPLICATED_TX_CACHE_NAME, stopLoad, loadError, threadProgressTracker); + + awaitProgressInAllLoaders(10_000, loadError, threadProgressTracker); + + stopGrid("flaky"); + + awaitProgressInAllLoaders(10_000, loadError, threadProgressTracker); + + File store = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false); + + U.delete(new File(store, FLAKY_WAL_PATH)); + U.delete(new File(store, FLAKY_WAL_ARCHIVE_PATH)); + U.delete(new File(store, FLAKY_STORAGE_PATH)); + + startGrid("flaky"); + + System.out.println("### Starting rebalancing after flaky node join"); + waitForRebalancing(); + System.out.println("### Rebalancing is finished after flaky node join"); + + awaitProgressInAllLoaders(10_000, loadError, threadProgressTracker); + + stopLoad.set(true); + } + + /** + * Test that changing baseline down under cross-cache txs load won't break cache. + */ + public void testCrossCacheTxs() throws Exception { + IgniteEx ig0 = (IgniteEx)startGrids(DEFAULT_NODES_COUNT); + + ig0.cluster().active(true); + + AtomicBoolean stopLoad = new AtomicBoolean(false); + + AtomicReference loadError = new AtomicReference<>(null); + + String cacheName1 = PARTITIONED_TX_CACHE_NAME; + String cacheName2 = PARTITIONED_TX_PRIM_SYNC_CACHE_NAME; + + IgniteCache cache1 = ig0.cache(PARTITIONED_TX_CACHE_NAME); + IgniteCache cache2 = ig0.cache(PARTITIONED_TX_PRIM_SYNC_CACHE_NAME); + + System.out.println("### Starting preloading"); + + for (int i = 0; i < ENTRIES; i++) { + ThreadLocalRandom r = ThreadLocalRandom.current(); + + byte[] randBytes1 = new byte[r.nextInt(10, 100)]; + byte[] randBytes2 = new byte[r.nextInt(10, 100)]; + + cache1.put(r.nextInt(ENTRIES), new String(randBytes1)); + cache2.put(r.nextInt(ENTRIES), new String(randBytes2)); + } + + System.out.println("### Preloading is finished"); + + IgniteEx client1 = (IgniteEx)startGrid("client1"); + IgniteEx client2 = (IgniteEx)startGrid("client2"); + + ConcurrentMap threadProgressTracker = new ConcurrentHashMap<>(); + + startCrossCacheTxLoadThread(client1, cacheName1, cacheName2, stopLoad, loadError, threadProgressTracker); + startCrossCacheTxLoadThread(client1, cacheName1, cacheName2, stopLoad, loadError, threadProgressTracker); + startCrossCacheTxLoadThread(client1, cacheName2, cacheName1, stopLoad, loadError, threadProgressTracker); + startCrossCacheTxLoadThread(client2, cacheName1, cacheName2, stopLoad, loadError, threadProgressTracker); + startCrossCacheTxLoadThread(client2, cacheName1, cacheName2, stopLoad, loadError, threadProgressTracker); + startCrossCacheTxLoadThread(client2, cacheName2, cacheName1, stopLoad, loadError, threadProgressTracker); + + awaitProgressInAllLoaders(10_000, loadError, threadProgressTracker); + + List fullBlt = new ArrayList<>(); + for (int i = 0; i < DEFAULT_NODES_COUNT; i++) + fullBlt.add(grid(i).localNode()); + + stopGrid(DEFAULT_NODES_COUNT - 1, true); + stopGrid(DEFAULT_NODES_COUNT - 2, true); + + awaitProgressInAllLoaders(10_000, loadError, threadProgressTracker); + + tryChangeBaselineDown(ig0, fullBlt, DEFAULT_NODES_COUNT - 1, loadError, threadProgressTracker); + tryChangeBaselineDown(ig0, fullBlt, DEFAULT_NODES_COUNT - 2, loadError, threadProgressTracker); + + stopLoad.set(true); + } + + /** + * Tests that join of non-baseline node while long transactions are running won't break dynamically started cache. + */ + public void testDynamicCacheLongTransactionNodeStart() throws Exception { + IgniteEx ig0 = (IgniteEx)startGrids(4); + + ig0.cluster().active(true); + + IgniteEx client = (IgniteEx)startGrid("client"); + + CacheConfiguration dynamicCacheCfg = cacheConfig(REPLICATED_TX_CACHE_NAME); + dynamicCacheCfg.setName("dyn"); + + IgniteCache dynamicCache = client.getOrCreateCache(dynamicCacheCfg); + + for (int i = 0; i < ENTRIES; i++) + dynamicCache.put(i, "abacaba" + i); + + AtomicBoolean releaseTx = new AtomicBoolean(false); + CountDownLatch allTxsDoneLatch = new CountDownLatch(10); + + for (int i = 0; i < 10; i++) { + final int i0 = i; + + GridTestUtils.runAsync(new Runnable() { + @Override public void run() { + try (Transaction tx = client.transactions().txStart(TransactionConcurrency.PESSIMISTIC, + TransactionIsolation.REPEATABLE_READ)) { + dynamicCache.put(i0, "txtxtxtx" + i0); + + while (!releaseTx.get()) + LockSupport.parkNanos(1_000_000); + + tx.commit(); + + System.out.println("Tx #" + i0 + " committed"); + } + catch (Throwable t) { + System.out.println("Tx #" + i0 + " failed"); + + t.printStackTrace(); + } + finally { + allTxsDoneLatch.countDown(); + } + } + }); + } + + GridTestUtils.runAsync(new Runnable() { + @Override public void run() { + try { + startGrid(4); + } + catch (Exception e) { + e.printStackTrace(); + } + } + }); + + U.sleep(1_000); + + releaseTx.set(true); + + allTxsDoneLatch.await(); + + for (int i = 0; i < 10_000; i++) + assertEquals("txtxtxtx" + (i % 10), dynamicCache.get(i % 10)); + } + + /** + * Tests that if dynamic cache has no affinity nodes at the moment of start, + * it will still work correctly when affinity nodes will appear. + */ + public void testDynamicCacheStartNoAffinityNodes() throws Exception { + fail("IGNITE-8652"); + + IgniteEx ig0 = startGrid(0); + + ig0.cluster().active(true); + + IgniteEx client = (IgniteEx)startGrid("client"); + + CacheConfiguration dynamicCacheCfg = new CacheConfiguration() + .setName("dyn") + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL) + .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC) + .setAffinity(new RendezvousAffinityFunction(false, 32)) + .setBackups(2) + .setNodeFilter(new ConsistentIdNodeFilter((Serializable)ig0.localNode().consistentId())); + + IgniteCache dynamicCache = client.getOrCreateCache(dynamicCacheCfg); + + for (int i = 1; i < 4; i++) + startGrid(i); + + resetBaselineTopology(); + + for (int i = 0; i < ENTRIES; i++) + dynamicCache.put(i, "abacaba" + i); + + AtomicBoolean releaseTx = new AtomicBoolean(false); + CountDownLatch allTxsDoneLatch = new CountDownLatch(10); + + for (int i = 0; i < 10; i++) { + final int i0 = i; + + GridTestUtils.runAsync(new Runnable() { + @Override public void run() { + try (Transaction tx = client.transactions().txStart(TransactionConcurrency.PESSIMISTIC, + TransactionIsolation.REPEATABLE_READ)) { + dynamicCache.put(i0, "txtxtxtx" + i0); + + while (!releaseTx.get()) + LockSupport.parkNanos(1_000_000); + + tx.commit(); + + System.out.println("Tx #" + i0 + " committed"); + } + catch (Throwable t) { + System.out.println("Tx #" + i0 + " failed"); + + t.printStackTrace(); + } + finally { + allTxsDoneLatch.countDown(); + } + } + }); + } + + GridTestUtils.runAsync(new Runnable() { + @Override public void run() { + try { + startGrid(4); + } + catch (Exception e) { + e.printStackTrace(); + } + } + }); + + U.sleep(1_000); + + releaseTx.set(true); + + allTxsDoneLatch.await(); + + for (int i = 0; i < 10_000; i++) + assertEquals("txtxtxtx" + (i % 10), dynamicCache.get(i % 10)); + } + + /** + * Tests that join of non-baseline node while long transactions are running won't break cache started on client join. + */ + public void testClientJoinCacheLongTransactionNodeStart() throws Exception { + IgniteEx ig0 = (IgniteEx)startGrids(4); + + ig0.cluster().active(true); + + IgniteEx client = (IgniteEx)startGrid("client"); + + IgniteCache clientJoinCache = client.cache(PARTITIONED_TX_CLIENT_CACHE_NAME); + + for (int i = 0; i < ENTRIES; i++) + clientJoinCache.put(i, "abacaba" + i); + + AtomicBoolean releaseTx = new AtomicBoolean(false); + CountDownLatch allTxsDoneLatch = new CountDownLatch(10); + + for (int i = 0; i < 10; i++) { + final int i0 = i; + + GridTestUtils.runAsync(new Runnable() { + @Override public void run() { + try (Transaction tx = client.transactions().txStart(TransactionConcurrency.PESSIMISTIC, + TransactionIsolation.REPEATABLE_READ)) { + clientJoinCache.put(i0, "txtxtxtx" + i0); + + while (!releaseTx.get()) + LockSupport.parkNanos(1_000_000); + + tx.commit(); + + System.out.println("Tx #" + i0 + " committed"); + } + catch (Throwable t) { + System.out.println("Tx #" + i0 + " failed"); + + t.printStackTrace(); + } + finally { + allTxsDoneLatch.countDown(); + } + } + }); + } + + GridTestUtils.runAsync(new Runnable() { + @Override public void run() { + try { + startGrid(4); + } + catch (Exception e) { + e.printStackTrace(); + } + } + }); + + U.sleep(1_000); + + releaseTx.set(true); + + allTxsDoneLatch.await(); + + for (int i = 0; i < 10_000; i++) + assertEquals("txtxtxtx" + (i % 10), clientJoinCache.get(i % 10)); + } + + /** + * @param ig0 Ignite. + * @param fullBlt Initial BLT list. + * @param newBaselineSize New baseline size. + * @param threadProgressTracker Thread progress tracker. + */ + private void tryChangeBaselineDown( + IgniteEx ig0, + List fullBlt, + int newBaselineSize, + AtomicReference loadError, + ConcurrentMap threadProgressTracker + ) throws Exception { + System.out.println("### Changing BLT: " + (newBaselineSize + 1) + " -> " + newBaselineSize); + ig0.cluster().setBaselineTopology(fullBlt.subList(0, newBaselineSize)); + + System.out.println("### Starting rebalancing after BLT change: " + (newBaselineSize + 1) + " -> " + newBaselineSize); + waitForRebalancing(); + System.out.println("### Rebalancing is finished after BLT change: " + (newBaselineSize + 1) + " -> " + newBaselineSize); + + awaitProgressInAllLoaders(10_000, loadError, threadProgressTracker); + + if (loadError.get() != null) { + loadError.get().printStackTrace(); + + fail("Unexpected error in load thread: " + loadError.get().toString()); + } + } + + /** + * @param ig Ignite instance. + * @param cacheName Cache name. + * @param stopFlag Stop flag. + * @param loadError Load error reference. + * @param threadProgressTracker Progress tracker. + */ + private void startSimpleLoadThread( + IgniteEx ig, + String cacheName, + AtomicBoolean stopFlag, + AtomicReference loadError, + ConcurrentMap threadProgressTracker + ) { + GridTestUtils.runAsync(new Runnable() { + @Override public void run() { + ThreadLocalRandom r = ThreadLocalRandom.current(); + + IgniteCache cache = ig.cache(cacheName); + + try { + while (!stopFlag.get()) { + try { + int op = r.nextInt(3); + + switch (op) { + case 0: + byte[] randBytes = new byte[r.nextInt(10, 100)]; + + cache.put(r.nextInt(ENTRIES), new String(randBytes)); + + break; + case 1: + cache.remove(r.nextInt(ENTRIES)); + + break; + case 2: + cache.get(r.nextInt(ENTRIES)); + + break; + } + + threadProgressTracker.compute(Thread.currentThread().getId(), + (tId, ops) -> ops == null ? 1 : ops + 1); + } + catch (CacheException e) { + if (e.getCause() instanceof ClusterTopologyException) + ((ClusterTopologyException)e.getCause()).retryReadyFuture().get(); + } + catch (ClusterTopologyException e) { + e.retryReadyFuture().get(); + } + } + } + catch (Throwable t) { + loadError.compareAndSet(null, t); + + stopFlag.set(true); + } + } + }); + } + + /** + * @param ig Ignite instance. + * @param cacheName Cache name. + * @param stopFlag Stop flag. + * @param loadError Load error reference. + * @param threadProgressTracker Progress tracker. + */ + private void startTxLoadThread( + IgniteEx ig, + String cacheName, + AtomicBoolean stopFlag, + AtomicReference loadError, + ConcurrentMap threadProgressTracker + ) { + GridTestUtils.runAsync(new Runnable() { + @Override public void run() { + ThreadLocalRandom r = ThreadLocalRandom.current(); + + IgniteCache cache = ig.cache(cacheName); + + boolean pessimistic = r.nextBoolean(); + + boolean rollback = r.nextBoolean(); + + try { + while (!stopFlag.get()) { + try (Transaction tx = ig.transactions().txStart( + pessimistic ? TransactionConcurrency.PESSIMISTIC : TransactionConcurrency.OPTIMISTIC, + TransactionIsolation.REPEATABLE_READ + )) { + int key1 = -1; + String val1 = null; + while (val1 == null) { + key1 = r.nextInt(ENTRIES); + val1 = cache.get(key1); + } + + int key2 = -1; + String val2 = null; + while (val2 == null) { + key2 = r.nextInt(ENTRIES); + val2 = cache.get(key2); + } + + cache.put(key1, val2); + cache.put(key2, val1); + + if (rollback) + tx.rollback(); + else + tx.commit(); + + threadProgressTracker.compute(Thread.currentThread().getId(), + (tId, ops) -> ops == null ? 1 : ops + 1); + } + catch (CacheException e) { + if (e.getCause() instanceof ClusterTopologyException) + ((ClusterTopologyException)e.getCause()).retryReadyFuture().get(); + } + catch (ClusterTopologyException e) { + e.retryReadyFuture().get(); + } + } + } + catch (Throwable t) { + loadError.compareAndSet(null, t); + + stopFlag.set(true); + } + } + }); + } + + /** + * @param ig Ignite instance. + * @param cacheName1 Cache name 1. + * @param cacheName2 Cache name 2. + * @param stopFlag Stop flag. + * @param loadError Load error reference. + * @param threadProgressTracker Progress tracker. + */ + private void startCrossCacheTxLoadThread( + IgniteEx ig, + String cacheName1, + String cacheName2, + AtomicBoolean stopFlag, + AtomicReference loadError, + ConcurrentMap threadProgressTracker + ) { + GridTestUtils.runAsync(new Runnable() { + @Override public void run() { + ThreadLocalRandom r = ThreadLocalRandom.current(); + + IgniteCache cache1 = ig.cache(cacheName1); + IgniteCache cache2 = ig.cache(cacheName2); + + boolean pessimistic = r.nextBoolean(); + + boolean rollback = r.nextBoolean(); + + try { + while (!stopFlag.get()) { + try (Transaction tx = ig.transactions().txStart( + pessimistic ? TransactionConcurrency.PESSIMISTIC : TransactionConcurrency.OPTIMISTIC, + TransactionIsolation.REPEATABLE_READ + )) { + int key1 = -1; + String val1 = null; + while (val1 == null) { + key1 = r.nextInt(ENTRIES); + val1 = cache1.get(key1); + } + + int key2 = -1; + String val2 = null; + while (val2 == null) { + key2 = r.nextInt(ENTRIES); + val2 = cache2.get(key2); + } + + cache1.put(key1, val2); + cache2.put(key2, val1); + + if (rollback) + tx.rollback(); + else + tx.commit(); + + threadProgressTracker.compute(Thread.currentThread().getId(), + (tId, ops) -> ops == null ? 1 : ops + 1); + } + catch (CacheException e) { + if (e.getCause() instanceof ClusterTopologyException) + ((ClusterTopologyException)e.getCause()).retryReadyFuture().get(); + } + catch (ClusterTopologyException e) { + e.retryReadyFuture().get(); + } + } + } + catch (Throwable t) { + loadError.compareAndSet(null, t); + + stopFlag.set(true); + } + } + }); + } + + /** + * @param waitMs Wait milliseconds. + * @param loadError Load error. + * @param threadProgressTracker Thread progress tracker. + */ + private void awaitProgressInAllLoaders( + long waitMs, + AtomicReference loadError, + ConcurrentMap threadProgressTracker + ) throws Exception { + Map view1 = new HashMap<>(threadProgressTracker); + + long startTs = U.currentTimeMillis(); + + while (U.currentTimeMillis() < startTs + waitMs) { + Map view2 = new HashMap<>(threadProgressTracker); + + if (loadError.get() != null) { + loadError.get().printStackTrace(); + + fail("Unexpected error in load thread: " + loadError.get().toString()); + } + + boolean frozenThreadExists = false; + + for (Map.Entry entry : view1.entrySet()) { + if (entry.getValue().equals(view2.get(entry.getKey()))) + frozenThreadExists = true; + } + + if (!frozenThreadExists) + return; + + U.sleep(100); + } + + fail("No progress in load thread"); + } + + /** + * Accepts all nodes except one with specified consistent ID. + */ + private static class ConsistentIdNodeFilter implements IgnitePredicate { + /** Consistent ID. */ + private final Serializable consId0; + + /** + * @param consId0 Consistent ID. + */ + public ConsistentIdNodeFilter(Serializable consId0) { + this.consId0 = consId0; + } + + /** {@inheritDoc} */ + @Override public boolean apply(ClusterNode node) { + return !node.consistentId().equals(consId0); + } + } +} 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 af797407c9e15..316ae1b8a7ad6 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 @@ -30,6 +30,7 @@ import org.apache.ignite.internal.processors.cache.persistence.IgnitePersistentStoreDataStructuresTest; 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.IgniteAllBaselineNodesOnlineFullApiSelfTest; import org.apache.ignite.internal.processors.cache.persistence.baseline.IgniteOfflineBaselineNodeFullApiSelfTest; import org.apache.ignite.internal.processors.cache.persistence.baseline.IgniteOnlineNodeOutOfBaselineFullApiSelfTest; @@ -71,6 +72,7 @@ public static TestSuite suite() { suite.addTestSuite(IgniteAllBaselineNodesOnlineFullApiSelfTest.class); suite.addTestSuite(IgniteOfflineBaselineNodeFullApiSelfTest.class); suite.addTestSuite(IgniteOnlineNodeOutOfBaselineFullApiSelfTest.class); + suite.addTestSuite(ClientAffinityAssignmentWithBaselineTest.class); suite.addTestSuite(IgniteAbsentEvictionNodeOutOfBaselineTest.class); return suite; From ddb4d45f2a6b16a4530b8c383239f3758cbd7d05 Mon Sep 17 00:00:00 2001 From: Anton Kalashnikov Date: Thu, 31 May 2018 16:06:52 +0300 Subject: [PATCH 0209/1463] IGNITE-8530 fixed onNodeLeft for InitNewCoordinatorFuture - Fixes #4086. Signed-off-by: Alexey Goncharuk (cherry picked from commit 49fe8cd) --- .../distributed/dht/preloader/InitNewCoordinatorFuture.java | 3 +++ .../java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java | 4 ++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/InitNewCoordinatorFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/InitNewCoordinatorFuture.java index 42a9ba6891955..5909a05cb6098 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/InitNewCoordinatorFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/InitNewCoordinatorFuture.java @@ -338,6 +338,9 @@ public void onNodeLeft(UUID nodeId) { synchronized (this) { done = awaited.remove(nodeId) && awaited.isEmpty(); + + if (done) + onAllReceived(); } if (done) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 6b2293a1d275c..35e945208eb65 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -4212,10 +4212,10 @@ else if (!locNodeId.equals(node.id()) && ring.node(node.id()) != null) { DiscoveryDataPacket dataPacket = msg.gridDiscoveryData(); - dataPacket.joiningNodeClient(msg.client()); - assert dataPacket != null : msg; + dataPacket.joiningNodeClient(msg.client()); + if (dataPacket.hasJoiningNodeData()) spi.onExchange(dataPacket, U.resolveClassLoader(spi.ignite().configuration())); From f317a6533f0f713a7a806155967f0337754c3c2f Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Thu, 31 May 2018 20:58:48 +0300 Subject: [PATCH 0210/1463] IGNITE-8659 Wrong FreeList usage in PendingTree may lead to page corruption. - Fixes #4100. Signed-off-by: Ivan Rakov (cherry picked from commit 7a72e01) --- .../persistence/GridCacheOffheapManager.java | 2 +- .../persistence/db/IgnitePdsWithTtlTest.java | 83 +++++++++++++++---- 2 files changed, 67 insertions(+), 18 deletions(-) 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 b62d778f975fe..9445a43ab2265 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 @@ -1245,7 +1245,7 @@ private CacheDataStore init0(boolean checkExists) throws IgniteCheckedException "PendingEntries-" + partId, grp.dataRegion().pageMemory(), pendingTreeRoot.pageId().pageId(), - reuseList, + freeList, pendingTreeRoot.isAllocated()) { /** {@inheritDoc} */ @Override protected long allocatePageNoReuse() throws IgniteCheckedException { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWithTtlTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWithTtlTest.java index be09e70e256c5..bb371dce29bca 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWithTtlTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWithTtlTest.java @@ -17,13 +17,16 @@ package org.apache.ignite.internal.processors.cache.persistence.db; -import com.google.common.base.Strings; import java.util.TreeMap; import java.util.concurrent.TimeUnit; import javax.cache.expiry.AccessedExpiryPolicy; +import javax.cache.expiry.CreatedExpiryPolicy; import javax.cache.expiry.Duration; +import javax.cache.expiry.ExpiryPolicy; import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cache.CacheRebalanceMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.DataRegionConfiguration; @@ -33,6 +36,7 @@ import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.PA; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; @@ -44,13 +48,13 @@ */ public class IgnitePdsWithTtlTest extends GridCommonAbstractTest { /** */ - public static final String CACHE = "expirableCache"; + public static final String CACHE_NAME = "expirableCache"; /** */ private static final int EXPIRATION_TIMEOUT = 10; /** */ - public static final int ENTRIES = 7000; + public static final int ENTRIES = 100_000; /** */ private static final TcpDiscoveryVmIpFinder FINDER = new TcpDiscoveryVmIpFinder(true); @@ -82,22 +86,24 @@ public class IgnitePdsWithTtlTest extends GridCommonAbstractTest { cfg.setDiscoverySpi(disco); final CacheConfiguration ccfg = new CacheConfiguration(); - ccfg.setName(CACHE); - ccfg.setAffinity(new RendezvousAffinityFunction(false, 128)); + ccfg.setName(CACHE_NAME); + ccfg.setAffinity(new RendezvousAffinityFunction(false, 32)); ccfg.setExpiryPolicyFactory(AccessedExpiryPolicy.factoryOf(new Duration(TimeUnit.SECONDS, EXPIRATION_TIMEOUT))); ccfg.setEagerTtl(true); ccfg.setGroupName("group1"); - + ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); ccfg.setRebalanceMode(CacheRebalanceMode.SYNC); + cfg.setDataStorageConfiguration( new DataStorageConfiguration() .setDefaultDataRegionConfiguration( new DataRegionConfiguration() - .setMaxSize(256L * 1024 * 1024) + .setMaxSize(192L * 1024 * 1024) .setPersistenceEnabled(true) - ).setWalMode(WALMode.DEFAULT)); + ).setWalMode(WALMode.LOG_ONLY)); cfg.setCacheConfiguration(ccfg); + return cfg; } @@ -122,7 +128,7 @@ private void loadAndWaitForCleanup(boolean restartGrid) throws Exception { IgniteEx srv = startGrid(0); srv.cluster().active(true); - fillCache(srv.cache(CACHE)); + fillCache(srv.cache(CACHE_NAME)); if (restartGrid) { stopGrid(0); @@ -130,7 +136,7 @@ private void loadAndWaitForCleanup(boolean restartGrid) throws Exception { srv.cluster().active(true); } - final IgniteCache cache = srv.cache(CACHE); + final IgniteCache cache = srv.cache(CACHE_NAME); pringStatistics((IgniteCacheProxy)cache, "After restart from LFS"); @@ -146,12 +152,14 @@ public void testRebalancingWithTtlExpirable() throws Exception { IgniteEx srv = startGrid(0); srv.cluster().active(true); - fillCache(srv.cache(CACHE)); + fillCache(srv.cache(CACHE_NAME)); - //causes rebalancing start srv = startGrid(1); - final IgniteCache cache = srv.cache(CACHE); + //causes rebalancing start + srv.cluster().setBaselineTopology(srv.cluster().topologyVersion()); + + final IgniteCache cache = srv.cache(CACHE_NAME); pringStatistics((IgniteCacheProxy)cache, "After rebalancing start"); @@ -160,11 +168,51 @@ public void testRebalancingWithTtlExpirable() throws Exception { stopAllGrids(); } + /** + * @throws Exception if failed. + */ + public void testStartStopAfterRebalanceWithTtlExpirable() throws Exception { + try { + IgniteEx srv = startGrid(0); + startGrid(1); + srv.cluster().active(true); + + ExpiryPolicy plc = CreatedExpiryPolicy.factoryOf(Duration.ONE_DAY).create(); + + IgniteCache cache0 = srv.cache(CACHE_NAME); + + fillCache(cache0.withExpiryPolicy(plc)); + + srv = startGrid(2); + + IgniteCache cache = srv.cache(CACHE_NAME); + + //causes rebalancing start + srv.cluster().setBaselineTopology(srv.cluster().topologyVersion()); + + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return Boolean.TRUE.equals(cache.rebalance().get()) && cache.localSizeLong(CachePeekMode.ALL) > 0; + } + }, 20_000); + + //check if pds is consistent + stopGrid(0); + startGrid(0); + + stopGrid(1); + startGrid(1); + } + finally { + stopAllGrids(); + } + } + /** */ - protected void fillCache(IgniteCache cache) { - cache.putAll(new TreeMap() {{ + protected void fillCache(IgniteCache cache) { + cache.putAll(new TreeMap() {{ for (int i = 0; i < ENTRIES; i++) - put(i, Strings.repeat("Some value " + i, 125)); + put(i, new byte[1024]); }}); //Touch entries. @@ -175,7 +223,8 @@ protected void fillCache(IgniteCache cache) { } /** */ - protected void waitAndCheckExpired(final IgniteCache cache) throws IgniteInterruptedCheckedException { + protected void waitAndCheckExpired( + final IgniteCache cache) throws IgniteInterruptedCheckedException { GridTestUtils.waitForCondition(new PA() { @Override public boolean apply() { return cache.size() == 0; From 7e3b22fac1b9165e0ffd933bca00aa3485316183 Mon Sep 17 00:00:00 2001 From: mcherkasov Date: Fri, 1 Jun 2018 17:09:14 +0300 Subject: [PATCH 0211/1463] IGNITE-8658 Add info message for complete partition exchange Signed-off-by: Andrey Gura --- .../dht/preloader/GridDhtPartitionsExchangeFuture.java | 3 +++ 1 file changed, 3 insertions(+) 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 944dce014115a..7ecebdd190179 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 @@ -1748,6 +1748,9 @@ public void finishMerged() { if (log.isDebugEnabled()) log.debug("Completed partition exchange [localNode=" + cctx.localNodeId() + ", exchange= " + this + ", durationFromInit=" + (U.currentTimeMillis() - initTs) + ']'); + else if(log.isInfoEnabled()) + log.info("Completed partition exchange [localNode=" + cctx.localNodeId() + ", exchange=" + shortInfo() + + ", topVer=" + topologyVersion() + ", durationFromInit=" + (U.currentTimeMillis() - initTs) + ']'); initFut.onDone(err == null); From 261c86753217f06e81486a23690047cd51f2550a Mon Sep 17 00:00:00 2001 From: dgladkikh Date: Fri, 1 Jun 2018 18:34:50 +0300 Subject: [PATCH 0212/1463] IGNITE-8603 Add JMX-metric to cluster: baseline nodes - Fixes #4060. Signed-off-by: Ivan Rakov (cherry picked from commit 1f6266c) --- .../ClusterLocalNodeMetricsMXBeanImpl.java | 21 +++ .../internal/ClusterMetricsMXBeanImpl.java | 32 ++++ .../ignite/mxbean/ClusterMetricsMXBean.java | 16 ++ .../ClusterBaselineNodesMetricsSelfTest.java | 178 ++++++++++++++++++ .../IgniteComputeGridTestSuite.java | 2 + 5 files changed, 249 insertions(+) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/ClusterBaselineNodesMetricsSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/ClusterLocalNodeMetricsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/ClusterLocalNodeMetricsMXBeanImpl.java index a242345df0fe5..eed501ae65444 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/ClusterLocalNodeMetricsMXBeanImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/ClusterLocalNodeMetricsMXBeanImpl.java @@ -18,9 +18,11 @@ package org.apache.ignite.internal; import java.util.Collections; +import java.util.List; import java.util.Set; import java.util.TreeSet; import java.util.UUID; +import org.apache.ignite.cluster.BaselineNode; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; import org.apache.ignite.internal.util.typedef.internal.S; @@ -332,6 +334,25 @@ public ClusterLocalNodeMetricsMXBeanImpl(GridDiscoveryManager discoMgr) { return node.metrics().getTotalNodes(); } + /** {@inheritDoc} */ + @Override public int getTotalBaselineNodes() { + if (!node.isClient() && !node.isDaemon()) { + List baselineNodes = discoMgr.baselineNodes(discoMgr.topologyVersionEx()); + + if (baselineNodes != null) + for (BaselineNode baselineNode : baselineNodes) + if (baselineNode.consistentId().equals(node.consistentId())) + return 1; + } + + return 0; + } + + /** {@inheritDoc} */ + @Override public int getActiveBaselineNodes() { + return getTotalBaselineNodes(); + } + /** {@inheritDoc} */ @Override public int getTotalServerNodes() { return !node.isClient() ? 1 : 0; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/ClusterMetricsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/ClusterMetricsMXBeanImpl.java index e09ad3c110dc9..1efb5906f3472 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/ClusterMetricsMXBeanImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/ClusterMetricsMXBeanImpl.java @@ -18,10 +18,13 @@ package org.apache.ignite.internal; import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.TreeSet; import java.util.UUID; +import org.apache.ignite.cluster.BaselineNode; import org.apache.ignite.cluster.ClusterGroup; import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.cluster.ClusterNode; @@ -358,6 +361,35 @@ private ClusterMetrics metrics() { return metrics().getTotalNodes(); } + /** {@inheritDoc} */ + @Override public int getTotalBaselineNodes() { + Collection baselineNodes = cluster.ignite().cluster().currentBaselineTopology(); + + return baselineNodes != null ? baselineNodes.size() : 0; + } + + /** {@inheritDoc} */ + @Override public int getActiveBaselineNodes() { + Collection baselineNodes = cluster.ignite().cluster().currentBaselineTopology(); + + if (baselineNodes != null && !baselineNodes.isEmpty()) { + Set bltIds = new HashSet<>(baselineNodes.size()); + + for (BaselineNode baselineNode : baselineNodes) + bltIds.add(baselineNode.consistentId()); + + int count = 0; + + for (ClusterNode node : cluster.forServers().nodes()) + if (bltIds.contains(node.consistentId())) + count++; + + return count; + } + + return 0; + } + /** {@inheritDoc} */ @Override public int getTotalServerNodes() { return cluster.forServers().nodes().size(); diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/ClusterMetricsMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/ClusterMetricsMXBean.java index 6385604e783f1..537cef7dc45da 100644 --- a/modules/core/src/main/java/org/apache/ignite/mxbean/ClusterMetricsMXBean.java +++ b/modules/core/src/main/java/org/apache/ignite/mxbean/ClusterMetricsMXBean.java @@ -257,6 +257,22 @@ public interface ClusterMetricsMXBean extends ClusterMetrics { @MXBeanDescription("Total number of nodes.") public int getTotalNodes(); + /** + * Get count of total baseline nodes. + * + * @return Count of total baseline nodes. + */ + @MXBeanDescription("Total baseline nodes count.") + public int getTotalBaselineNodes(); + + /** + * Get count of active baseline nodes. + * + * @return Count of active baseline nodes. + */ + @MXBeanDescription("Active baseline nodes count.") + public int getActiveBaselineNodes(); + /** * Get count of server nodes. * diff --git a/modules/core/src/test/java/org/apache/ignite/internal/ClusterBaselineNodesMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/ClusterBaselineNodesMetricsSelfTest.java new file mode 100644 index 0000000000000..565317720d2a9 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/ClusterBaselineNodesMetricsSelfTest.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; + +import java.lang.management.ManagementFactory; +import java.util.Collection; +import javax.management.MBeanServer; +import javax.management.MBeanServerInvocationHandler; +import javax.management.MalformedObjectNameException; +import javax.management.ObjectName; +import org.apache.ignite.Ignition; +import org.apache.ignite.cluster.BaselineNode; +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.util.typedef.internal.U; +import org.apache.ignite.mxbean.ClusterMetricsMXBean; +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.testframework.junits.common.GridCommonTest; + +/** + * Baseline nodes metrics self test. + */ +@GridCommonTest(group = "Kernal Self") +public class ClusterBaselineNodesMetricsSelfTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testBaselineNodes() throws Exception { + // Start 2 server nodes. + IgniteEx ignite0 = startGrid(0); + startGrid(1); + + // Cluster metrics. + ClusterMetricsMXBean mxBeanCluster = mxBean(0, ClusterMetricsMXBeanImpl.class); + + ignite0.cluster().active(true); + + // Added 2 server nodes to baseline. + resetBlt(); + + // Add server node outside of the baseline. + startGrid(2); + + // Start client node. + Ignition.setClientMode(true); + startGrid(3); + Ignition.setClientMode(false); + + Collection baselineNodes; + + // State #0: 3 server nodes (2 total baseline nodes, 2 active baseline nodes), 1 client node + log.info(String.format(">>> State #0: topology version = %d", ignite0.cluster().topologyVersion())); + + assertEquals(3, mxBeanCluster.getTotalServerNodes()); + assertEquals(1, mxBeanCluster.getTotalClientNodes()); + assertEquals(2, mxBeanCluster.getTotalBaselineNodes()); + assertEquals(2, mxBeanCluster.getActiveBaselineNodes()); + assertEquals(2, (baselineNodes = ignite0.cluster().currentBaselineTopology()) != null + ? baselineNodes.size() + : 0); + + stopGrid(1, true); + + // State #1: 2 server nodes (2 total baseline nodes, 1 active baseline node), 1 client node + log.info(String.format(">>> State #1: topology version = %d", ignite0.cluster().topologyVersion())); + + assertEquals(2, mxBeanCluster.getTotalServerNodes()); + assertEquals(1, mxBeanCluster.getTotalClientNodes()); + assertEquals(2, mxBeanCluster.getTotalBaselineNodes()); + assertEquals(1, mxBeanCluster.getActiveBaselineNodes()); + assertEquals(2, (baselineNodes = ignite0.cluster().currentBaselineTopology()) != null + ? baselineNodes.size() + : 0); + + startGrid(1); + + ClusterMetricsMXBean mxBeanLocalNode1 = mxBean(1, ClusterLocalNodeMetricsMXBeanImpl.class); + + // State #2: 3 server nodes (2 total baseline nodes, 2 active baseline nodes), 1 client node + log.info(String.format(">>> State #2: topology version = %d", ignite0.cluster().topologyVersion())); + + assertEquals(3, mxBeanCluster.getTotalServerNodes()); + assertEquals(1, mxBeanCluster.getTotalClientNodes()); + assertEquals(2, mxBeanCluster.getTotalBaselineNodes()); + assertEquals(2, mxBeanCluster.getActiveBaselineNodes()); + assertEquals(1, mxBeanLocalNode1.getTotalBaselineNodes()); + assertEquals(2, (baselineNodes = ignite0.cluster().currentBaselineTopology()) != null + ? baselineNodes.size() + : 0); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String name) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(name); + + cfg.setConsistentId(name); + + String storePath = getClass().getSimpleName().toLowerCase() + "/" + getName(); + + cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER)); + + cfg.setDataStorageConfiguration( + new DataStorageConfiguration() + .setWalMode(WALMode.LOG_ONLY) + .setStoragePath(storePath) + .setWalPath(storePath + "/wal") + .setWalArchivePath(storePath + "/archive") + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration() + .setPersistenceEnabled(true) + .setMaxSize(2L * 1024 * 1024 * 1024) + ) + ); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + private void resetBlt() throws Exception { + resetBaselineTopology(); + + waitForRebalancing(); + awaitPartitionMapExchange(); + } + + /** + * Gets ClusterMetricsMXBean for given node. + * + * @param nodeIdx Node index. + * @param clazz Class of ClusterMetricsMXBean implementation. + * @return MBean instance. + */ + private ClusterMetricsMXBean mxBean(int nodeIdx, Class clazz) + throws MalformedObjectNameException { + + ObjectName mbeanName = U.makeMBeanName( + getTestIgniteInstanceName(nodeIdx), + "Kernal", + clazz.getSimpleName()); + + MBeanServer mbeanSrv = ManagementFactory.getPlatformMBeanServer(); + + if (!mbeanSrv.isRegistered(mbeanName)) + fail("MBean is not registered: " + mbeanName.getCanonicalName()); + + return MBeanServerInvocationHandler.newProxyInstance(mbeanSrv, mbeanName, ClusterMetricsMXBean.class, true); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java index 14eb296424a47..bab7099e3ebe5 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java @@ -18,6 +18,7 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; +import org.apache.ignite.internal.ClusterBaselineNodesMetricsSelfTest; import org.apache.ignite.internal.ClusterNodeMetricsSelfTest; import org.apache.ignite.internal.ClusterNodeMetricsUpdateTest; import org.apache.ignite.internal.GridAffinityNoCacheSelfTest; @@ -124,6 +125,7 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(GridTaskInstanceExecutionSelfTest.class); suite.addTestSuite(ClusterNodeMetricsSelfTest.class); suite.addTestSuite(ClusterNodeMetricsUpdateTest.class); + suite.addTestSuite(ClusterBaselineNodesMetricsSelfTest.class); suite.addTestSuite(GridNonHistoryMetricsSelfTest.class); suite.addTestSuite(GridCancelledJobsMetricsSelfTest.class); suite.addTestSuite(GridCollisionJobsContextSelfTest.class); From f1c1b1f6887b94b7767cde64b1e682a25a4ee007 Mon Sep 17 00:00:00 2001 From: Sergey Skudnov Date: Mon, 14 May 2018 13:35:14 +0300 Subject: [PATCH 0213/1463] GG-13888 Uptime output with days - Fixes #3775. Cherry-picked from dfb0b9ee35afeb6adc546160c37b08a85d869f59 Signed-off-by: dpavlov --- .../java/org/apache/ignite/internal/IgniteKernal.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 0c4e386ab544f..90edfe337b6e5 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 @@ -492,7 +492,7 @@ public IgniteKernal(@Nullable GridSpringResourceContext rsrcCtx) { /** {@inheritDoc} */ @Override public String getUpTimeFormatted() { - return X.timeSpan2HMSM(U.currentTimeMillis() - startTime); + return X.timeSpan2DHMSM(U.currentTimeMillis() - startTime); } /** {@inheritDoc} */ @@ -2327,10 +2327,10 @@ else if (state == STARTING) if (!errOnStop) U.quiet(false, "Ignite node stopped OK [" + nodeName + "uptime=" + - X.timeSpan2HMSM(U.currentTimeMillis() - startTime) + ']'); + X.timeSpan2DHMSM(U.currentTimeMillis() - startTime) + ']'); else U.quiet(true, "Ignite node stopped wih ERRORS [" + nodeName + "uptime=" + - X.timeSpan2HMSM(U.currentTimeMillis() - startTime) + ']'); + X.timeSpan2DHMSM(U.currentTimeMillis() - startTime) + ']'); } if (log.isInfoEnabled()) @@ -2345,7 +2345,7 @@ else if (state == STARTING) ">>> " + ack + NL + ">>> " + dash + NL + (igniteInstanceName == null ? "" : ">>> Ignite instance name: " + igniteInstanceName + NL) + - ">>> Grid uptime: " + X.timeSpan2HMSM(U.currentTimeMillis() - startTime) + + ">>> Grid uptime: " + X.timeSpan2DHMSM(U.currentTimeMillis() - startTime) + NL + NL); } @@ -2359,7 +2359,7 @@ else if (state == STARTING) ">>> " + ack + NL + ">>> " + dash + NL + (igniteInstanceName == null ? "" : ">>> Ignite instance name: " + igniteInstanceName + NL) + - ">>> Grid uptime: " + X.timeSpan2HMSM(U.currentTimeMillis() - startTime) + + ">>> Grid uptime: " + X.timeSpan2DHMSM(U.currentTimeMillis() - startTime) + NL + ">>> See log above for detailed error message." + NL + ">>> Note that some errors during stop can prevent grid from" + NL + From 588498f4ad5a9fc98d758f59b849668cc8b32f29 Mon Sep 17 00:00:00 2001 From: Sergey Chugunov Date: Wed, 16 May 2018 17:29:58 +0300 Subject: [PATCH 0214/1463] IGNITE-8514 internal packages were added to excluded-list, package description for zk base package was added (cherry picked from commit 2fcf203) --- .../ignite/spi/discovery/zk/package-info.java | 21 +++++++++++++++++++ pom.xml | 2 +- 2 files changed, 22 insertions(+), 1 deletion(-) create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/package-info.java diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/package-info.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/package-info.java new file mode 100644 index 0000000000000..f01ecdad8129a --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 Zookeeper Discovery Spi classes and interfaces. + */ +package org.apache.ignite.spi.discovery.zk; \ No newline at end of file diff --git a/pom.xml b/pom.xml index 086d241f829af..b88e85f9804ac 100644 --- a/pom.xml +++ b/pom.xml @@ -193,7 +193,7 @@ ${basedir}/target/javadoc core ${basedir}/assembly/docfiles/javadoc.css - org.apache.ignite -exclude org.apache.ignite.client:org.apache.ignite.codegen:org.apache.ignite.examples:org.apache.ignite.internal:org.apache.ignite.schema:org.apache.ignite.tests:org.apache.ignite.tools:org.apache.ignite.util:org.apache.ignite.spi.discovery.tcp.messages:org.apache.ignite.spi.discovery.tcp.internal:org.apache.ignite.spi.deployment.uri.scanners:org.apache.ignite.spi.deployment.uri.tasks:org.apache.ignite.yardstick:org.apache.ignite.webtest + org.apache.ignite -exclude org.apache.ignite.client:org.apache.ignite.codegen:org.apache.ignite.examples:org.apache.ignite.internal:org.apache.ignite.schema:org.apache.ignite.tests:org.apache.ignite.tools:org.apache.ignite.util:org.apache.ignite.spi.discovery.tcp.messages:org.apache.ignite.spi.discovery.tcp.internal:org.apache.ignite.spi.communication.tcp.internal:org.apache.ignite.spi.discovery.zk.internal:org.apache.ignite.spi.deployment.uri.scanners:org.apache.ignite.spi.deployment.uri.tasks:org.apache.ignite.yardstick:org.apache.ignite.webtest From a3278ce3e40d3a9462bbc6ed56e3e164a325e859 Mon Sep 17 00:00:00 2001 From: YuriBabak Date: Wed, 16 May 2018 17:35:12 +0300 Subject: [PATCH 0215/1463] IGNITE-8514: Missed descriptions for a few packages in Javadoc. fixed for ml packages. (cherry picked from commit 915040e) --- .../ignite/ml/structures/package-info.java | 2 +- .../ml/structures/partition/package-info.java | 22 +++++++++++++++++++ .../preprocessing/package-info.java | 2 +- 3 files changed, 24 insertions(+), 2 deletions(-) create mode 100644 modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/package-info.java diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/structures/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/structures/package-info.java index ec9d79ec95859..359ac6c583039 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/structures/package-info.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/structures/package-info.java @@ -17,6 +17,6 @@ /** * - * Contains some utility structures. + * Contains some internal utility structures. */ package org.apache.ignite.ml.structures; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/package-info.java new file mode 100644 index 0000000000000..41345230b15dc --- /dev/null +++ b/modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/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 internal APIs for dataset partitioned labeled datasets. + */ +package org.apache.ignite.ml.structures.partition; \ No newline at end of file diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/package-info.java index c243074ae8b64..77419d0e122c5 100644 --- a/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/package-info.java +++ b/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/package-info.java @@ -17,6 +17,6 @@ /** * - * Contains main APIs for dataset pre-processing. + * Contains internal APIs for dataset pre-processing. */ package org.apache.ignite.ml.structures.preprocessing; From 17767ead6a12b8f0d46c5d2d31df8b61c18acc05 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Mon, 4 Jun 2018 16:25:56 +0300 Subject: [PATCH 0216/1463] IGNITE-8690 Missed package-info for some packages - Fixes #4116. Signed-off-by: Ivan Rakov (cherry picked from commit d473b50) --- .../tcp/internal/package-info.java | 22 +++++++++++++++++++ .../discovery/zk/internal/package-info.java | 21 ++++++++++++++++++ 2 files changed, 43 insertions(+) create mode 100644 modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/package-info.java create mode 100644 modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/package-info.java diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/package-info.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/package-info.java new file mode 100644 index 0000000000000..341c088b0c818 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/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 connection health-checkers for TCP/IP-based implementation of communication SPI. + */ +package org.apache.ignite.spi.communication.tcp.internal; \ No newline at end of file diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/package-info.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/package-info.java new file mode 100644 index 0000000000000..b3ef799989a65 --- /dev/null +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 Zookeeper Discovery Spi implementation classes. + */ +package org.apache.ignite.spi.discovery.zk.internal; \ No newline at end of file From db5f01380fdad377aa29091f93031d2c79795d60 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Mon, 4 Jun 2018 16:24:00 +0300 Subject: [PATCH 0217/1463] IGNITE-8691 Removed jar-plugin from ignite-zookeeper module - Fixes #4118. Signed-off-by: Ivan Rakov (cherry picked from commit b3dc334) --- modules/zookeeper/pom.xml | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/modules/zookeeper/pom.xml b/modules/zookeeper/pom.xml index 0777f1e36edbd..2de6b02df387f 100644 --- a/modules/zookeeper/pom.xml +++ b/modules/zookeeper/pom.xml @@ -160,18 +160,6 @@ - - org.apache.maven.plugins - maven-jar-plugin - - - - test-jar - - - - - org.codehaus.mojo exec-maven-plugin From 27b2ccbcef5376b891fa117835b9028d5b8dd66d Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Mon, 4 Jun 2018 15:05:47 +0300 Subject: [PATCH 0218/1463] IGNITE-8675: Fixed flacky test PdsWithTtlCompatibilityTest.testNodeStartByOldVersionPersistenceData_2_1. - Fixes #4105. Signed-off-by: Dmitriy Pavlov (cherry picked from commit 3beb3c3) --- .../ignite/compatibility/PdsWithTtlCompatibilityTest.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/PdsWithTtlCompatibilityTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/PdsWithTtlCompatibilityTest.java index f3649f6de83cc..946caddb5f203 100644 --- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/PdsWithTtlCompatibilityTest.java +++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/PdsWithTtlCompatibilityTest.java @@ -31,6 +31,7 @@ import org.apache.ignite.configuration.DataRegionConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.MemoryConfiguration; import org.apache.ignite.configuration.PersistentStoreConfiguration; import org.apache.ignite.configuration.WALMode; import org.apache.ignite.internal.IgniteEx; @@ -72,6 +73,7 @@ public class PdsWithTtlCompatibilityTest extends IgnitePersistenceCompatibilityA new DataRegionConfiguration() .setMaxSize(32L * 1024 * 1024) .setPersistenceEnabled(true) + .setCheckpointPageBufferSize(16L * 1024 * 1024) ).setWalMode(WALMode.LOG_ONLY)); return cfg; @@ -162,7 +164,9 @@ public static class ConfigurationClosure implements IgniteInClosure Date: Tue, 5 Jun 2018 11:28:43 +0300 Subject: [PATCH 0219/1463] IGNITE-8468: SQL: optimized inlined UUID comparison. This closes #3977. (cherry picked from commit c50b3c2) --- .../query/h2/database/InlineIndexHelper.java | 37 ++++++++++++++++++- .../h2/database/InlineIndexHelperTest.java | 22 +++++++---- 2 files changed, 49 insertions(+), 10 deletions(-) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java index 34191273694f6..05d5c46e61e13 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java @@ -413,6 +413,9 @@ private int tryCompareOptimized(long pageAddr, int off, int maxSize, Value v) { break; + case Value.UUID: + return compareAsUUID(pageAddr, off, v, type); + case Value.BYTES: return compareAsBytes(pageAddr, off, v); } @@ -425,7 +428,37 @@ private int tryCompareOptimized(long pageAddr, int off, int maxSize, Value v) { * @param off Offset. * @param v Value to compare. * @param type Highest value type. - * @return Compare result ({@code -2} means we can't compare). + * @return Compare result ({@code Integer.MIN_VALUE} means unsupported operation. + */ + private int compareAsUUID(long pageAddr, int off, Value v, int type) { + // only compatible types are supported now. + if(PageUtils.getByte(pageAddr, off) == type) { + assert type == Value.UUID; + + ValueUuid uuid = (ValueUuid)v.convertTo(Value.UUID); + long long1 = PageUtils.getLong(pageAddr, off + 1); + + int c = Long.compare(long1, uuid.getHigh()); + + if(c != 0) + return fixSort(c, sortType()); + + long1 = PageUtils.getLong(pageAddr, off + 9); + + c = Long.compare(long1, uuid.getLow()); + + return fixSort(c, sortType()); + } + + return Integer.MIN_VALUE; + } + + /** + * @param pageAddr Page address. + * @param off Offset. + * @param v Value to compare. + * @param type Highest value type. + * @return Compare result ({@code Integer.MIN_VALUE} means unsupported operation. */ private int compareAsDateTime(long pageAddr, int off, Value v, int type) { // only compatible types are supported now. @@ -470,7 +503,7 @@ private int compareAsDateTime(long pageAddr, int off, Value v, int type) { * @param off Offset. * @param v Value to compare. * @param type Highest value type. - * @return Compare result ({@code -2} means we can't compare). + * @return Compare result ({@code Integer.MIN_VALUE} means unsupported operation. */ private int compareAsPrimitive(long pageAddr, int off, Value v, int type) { // only compatible types are supported now. diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java index 41dd4f18a8bd4..ea23483ef0741 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java @@ -64,7 +64,7 @@ public class InlineIndexHelperTest extends GridCommonAbstractTest { private static final long MB = 1024; /** */ - private static final int CPUS = Runtime.getRuntime().availableProcessors(); + private static final Comparator ALWAYS_FAILS_COMPARATOR = new AlwaysFailsComparator(); /** Test utf-8 string cutting. */ public void testConvert() { @@ -209,13 +209,7 @@ private int putAndCompare(String v1, String v2, int maxSize) throws Exception { ih.put(pageAddr, off, v1 == null ? ValueNull.INSTANCE : ValueString.get(v1), maxSize); - Comparator comp = new Comparator() { - @Override public int compare(Value o1, Value o2) { - throw new AssertionError("Optimized algorithm should be used."); - } - }; - - return ih.compare(pageAddr, off, maxSize, v2 == null ? ValueNull.INSTANCE : ValueString.get(v2), comp); + return ih.compare(pageAddr, off, maxSize, v2 == null ? ValueNull.INSTANCE : ValueString.get(v2), ALWAYS_FAILS_COMPARATOR); } finally { if (page != 0L) @@ -485,6 +479,8 @@ private void testPutGet(Value v1, Value v2, Value v3) throws Exception { assertEquals(v1.getObject(), v11.getObject()); assertEquals(v2.getObject(), v22.getObject()); + + assertEquals(0, ih.compare(pageAddr, 0, max, v1, ALWAYS_FAILS_COMPARATOR)); } finally { if (page != 0L) @@ -560,4 +556,14 @@ else if(ch >= 56192 && ch <= 56319) return new String(buffer); } + + /** + * + */ + private static class AlwaysFailsComparator implements Comparator { + /** {@inheritDoc} */ + @Override public int compare(Value o1, Value o2) { + throw new AssertionError("Optimized algorithm should be used."); + } + } } From c81ea7442f034cc2cab87d0829c126f78a5a3c24 Mon Sep 17 00:00:00 2001 From: Sergey Chugunov Date: Fri, 4 May 2018 15:26:02 +0300 Subject: [PATCH 0220/1463] IGNITE-8421 new implementation of getChildren method is added tolerating KeeperException.NoNodeException - Fixes #3939. Signed-off-by: dpavlov (cherry-picked from commit #02e9ca993178d4aa648d06cb93ce1a9277eb22b1) --- .../ZkDistributedCollectDataFuture.java | 2 +- .../zk/internal/ZookeeperClient.java | 25 +++++++++++++++++++ 2 files changed, 26 insertions(+), 1 deletion(-) 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 174d698fe27e4..e9b28e192216c 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 @@ -150,7 +150,7 @@ static void deleteFutureData(ZookeeperClient client, try { client.deleteAll(evtDir, - client.getChildren(evtDir), + client.getChildrenIfPathExists(evtDir), -1); } catch (KeeperException.NoNodeException e) { 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 cc525d31eb7ee..6cc77a5875eaa 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 @@ -489,6 +489,31 @@ List getChildren(String path) } } + /** + * @param path Path. + * @return Children nodes. + * @throws KeeperException.NoNodeException If provided path does not exist. + * @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; + + try { + return zk.getChildren(path, false); + } + catch (KeeperException.NoNodeException e) { + throw e; + } + catch (Exception e) { + onZookeeperError(connStartTime, e); + } + } + } + + /** * @param path Path. * @throws InterruptedException If interrupted. From 3bec70692385f482fbcb7c0ebbcfb66b59d077d5 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Tue, 5 Jun 2018 14:42:43 +0300 Subject: [PATCH 0221/1463] IGNITE-8682 Attempt to configure IGFS in persistent mode without specific data region ends with AssertionError (cherry picked from commit 2fe0a10) --- .../processors/cache/GridCacheProcessor.java | 9 ++++++--- .../internal/processors/cache/GridCacheUtils.java | 12 +++++++++++- .../IgniteCacheDatabaseSharedManager.java | 2 +- 3 files changed, 18 insertions(+), 5 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 68b684027f002..6cecf5c8774de 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 @@ -2023,22 +2023,25 @@ private CacheGroupContext startCacheGroup( String memPlcName = cfg.getDataRegionName(); - DataRegion memPlc = sharedCtx.database().dataRegion(memPlcName); + DataRegion dataRegion = sharedCtx.database().dataRegion(memPlcName); FreeList freeList = sharedCtx.database().freeList(memPlcName); ReuseList reuseList = sharedCtx.database().reuseList(memPlcName); + boolean persistenceEnabled = sharedCtx.localNode().isClient() ? desc.persistenceEnabled() : + dataRegion != null && dataRegion.config().isPersistenceEnabled(); + CacheGroupContext grp = new CacheGroupContext(sharedCtx, desc.groupId(), desc.receivedFrom(), cacheType, cfg, affNode, - memPlc, + dataRegion, cacheObjCtx, freeList, reuseList, exchTopVer, - desc.persistenceEnabled(), + persistenceEnabled, desc.walEnabled() ); 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 bb64cc6f2e73e..4f349a4b5c385 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 @@ -119,6 +119,7 @@ import static org.apache.ignite.configuration.CacheConfiguration.DFLT_CACHE_MODE; import static org.apache.ignite.internal.GridTopic.TOPIC_REPLICATION; import static org.apache.ignite.internal.processors.cache.GridCacheOperation.READ; +import static org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager.SYSTEM_DATA_REGION_NAME; /** * Cache utility methods. @@ -1800,7 +1801,7 @@ public static boolean isPersistentCache(CacheConfiguration ccfg, DataStorageConf return false; // Special handling for system cache is needed. - if (isSystemCache(ccfg.getName())) { + if (isSystemCache(ccfg.getName()) || isIgfsCacheInSystemRegion(ccfg)) { if (dsCfg.getDefaultDataRegionConfiguration().isPersistenceEnabled()) return true; @@ -1829,6 +1830,15 @@ public static boolean isPersistentCache(CacheConfiguration ccfg, DataStorageConf return false; } + /** + * Checks whether cache configuration represents IGFS cache that will be placed in system memory region. + * + * @param ccfg Cache config. + */ + private static boolean isIgfsCacheInSystemRegion(CacheConfiguration ccfg) { + return IgfsUtils.matchIgfsCacheName(ccfg.getName()) && + (SYSTEM_DATA_REGION_NAME.equals(ccfg.getDataRegionName()) || ccfg.getDataRegionName() == null); + } /** * @return {@code true} if persistence is enabled for at least one data region, {@code false} if not. 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 5e8f40706e984..e3ce04d8809ac 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 @@ -76,7 +76,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdapter implements IgniteChangeGlobalStateSupport, CheckpointLockStateChecker { /** DataRegionConfiguration name reserved for internal caches. */ - static final String SYSTEM_DATA_REGION_NAME = "sysMemPlc"; + public static final String SYSTEM_DATA_REGION_NAME = "sysMemPlc"; /** Minimum size of memory chunk */ private static final long MIN_PAGE_MEMORY_SIZE = 10 * 1024 * 1024; From d872cbcb634d3472ceedfb890f946751a80269d9 Mon Sep 17 00:00:00 2001 From: devozerov Date: Tue, 5 Jun 2018 16:59:07 +0300 Subject: [PATCH 0222/1463] IGNITE-8701: JDBC thin: exception no longer thrown if client supplies credentials, but server-side auth is disabled. This closes #4137. --- .../ignite/jdbc/thin/JdbcThinConnectionSelfTest.java | 12 ------------ .../processors/odbc/jdbc/JdbcConnectionContext.java | 12 +++++++----- 2 files changed, 7 insertions(+), 17 deletions(-) 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 b462e4679fe03..8d1b8b53063f0 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 @@ -1918,18 +1918,6 @@ public void testSslClientAndPlainServer() { }, SQLException.class, "Failed to SSL connect to server"); } - /** - */ - public void testAuthenticateDisableOnServerClientTryAuthenticate() { - GridTestUtils.assertThrows(log, new Callable() { - @Override public Object call() throws Exception { - DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1/?user=test&password=test"); - - return null; - } - }, SQLException.class, "Can not perform the operation because the authentication is not enabled for the cluster"); - } - /** * @throws Exception If failed. */ 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 272c2f0ea41e2..d3c327b5a50f6 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,13 +142,15 @@ public JdbcConnectionContext(GridKernalContext ctx, GridNioSession ses, GridSpin String user = reader.readString(); String passwd = reader.readString(); - if (F.isEmpty(user) && ctx.authentication().enabled()) - throw new IgniteCheckedException("Unauthenticated sessions are prohibited"); + if (ctx.authentication().enabled()) { + if (F.isEmpty(user)) + throw new IgniteCheckedException("Unauthenticated sessions are prohibited"); - actx = ctx.authentication().authenticate(user, passwd); + actx = ctx.authentication().authenticate(user, passwd); - if (actx == null) - throw new IgniteCheckedException("Unknown authentication error"); + if (actx == null) + throw new IgniteCheckedException("Unknown authentication error"); + } } else { if (ctx.authentication().enabled()) From 1e4e4f5518dce04456433a297d9884f7c08da15d Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Tue, 5 Jun 2018 17:30:46 +0300 Subject: [PATCH 0223/1463] IGNITE-8691 Removed jar-plugin from ignite-zookeeper module - rollback. Signed-off-by: Ivan Rakov (cherry picked from commit 0f4d6fe) --- modules/zookeeper/pom.xml | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/modules/zookeeper/pom.xml b/modules/zookeeper/pom.xml index 2de6b02df387f..0777f1e36edbd 100644 --- a/modules/zookeeper/pom.xml +++ b/modules/zookeeper/pom.xml @@ -160,6 +160,18 @@ + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + org.codehaus.mojo exec-maven-plugin From f02f5648b90f1efcd8a7d1c63b34979e9fbbef10 Mon Sep 17 00:00:00 2001 From: a-polyakov Date: Tue, 5 Jun 2018 18:49:19 +0300 Subject: [PATCH 0224/1463] IGNITE-8602 Add support filter "label null" for control.sh tx utility - Fixes #4073. Signed-off-by: Alexey Goncharuk (cherry picked from commit 3372590) --- .../apache/ignite/internal/visor/tx/VisorTxTask.java | 2 +- .../apache/ignite/util/GridCommandHandlerTest.java | 11 +++++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java index b411e29006757..5a7ffdd2abbca 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java @@ -173,7 +173,7 @@ private VisorTxJob(VisorTxTaskArg arg, boolean debug) { if (arg.getMinSize() != null && locTx.size() < arg.getMinSize()) continue; - if (lbMatch != null && (locTx.label() == null || !lbMatch.matcher(locTx.label()).matches())) + if (lbMatch != null && !lbMatch.matcher(locTx.label() == null ? "null" : locTx.label()).matches()) continue; Collection mappings = new ArrayList<>(); diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java index 385e79b3926fa..f78f46626f2c7 100644 --- a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java @@ -445,6 +445,17 @@ else if (entry.getKey().equals(node2)) { } }, "--tx", "label", "^label[0-9]"); + // Test filter by empty label. + validate(h, map -> { + VisorTxTaskResult res = map.get(grid(0).localNode()); + + for (VisorTxInfo info:res.getInfos()){ + assertNull(info.getLabel()); + + } + + }, "--tx", "label", "null"); + // test order by size. validate(h, map -> { VisorTxTaskResult res = map.get(grid(0).localNode()); From 46fd0b080ea43ad5da29617c9883b5ccf428fbc5 Mon Sep 17 00:00:00 2001 From: a-polyakov Date: Tue, 5 Jun 2018 19:01:03 +0300 Subject: [PATCH 0225/1463] IGNITE-8467 Fixed filter minSize for transactions utility control.sh. Fixes #4069 (cherry picked from commit d61c068) --- .../ignite/internal/visor/tx/VisorTxTask.java | 6 +++--- .../ignite/util/GridCommandHandlerTest.java | 15 +++++++++++++++ 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java index 5a7ffdd2abbca..579abbe9261a9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java @@ -170,9 +170,6 @@ private VisorTxJob(VisorTxTaskArg arg, boolean debug) { duration < arg.getMinDuration()) continue; - if (arg.getMinSize() != null && locTx.size() < arg.getMinSize()) - continue; - if (lbMatch != null && !lbMatch.matcher(locTx.label() == null ? "null" : locTx.label()).matches()) continue; @@ -194,6 +191,9 @@ private VisorTxJob(VisorTxTaskArg arg, boolean debug) { } } + if (arg.getMinSize() != null && size < arg.getMinSize()) + continue; + infos.add(new VisorTxInfo(locTx.xid(), duration, locTx.isolation(), locTx.concurrency(), locTx.timeout(), locTx.label(), mappings, locTx.state(), size)); diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java index f78f46626f2c7..3ec0617e9164a 100644 --- a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java @@ -456,6 +456,21 @@ else if (entry.getKey().equals(node2)) { }, "--tx", "label", "null"); + + // test check minSize + int minSize=10; + + validate(h, map -> { + VisorTxTaskResult res = map.get(grid(0).localNode()); + + assertNotNull(res); + + for (VisorTxInfo txInfo : res.getInfos()) { + assertTrue(txInfo.getSize() >= minSize); + + } + }, "--tx", "minSize", Integer.toString(minSize)); + // test order by size. validate(h, map -> { VisorTxTaskResult res = map.get(grid(0).localNode()); From a010e286ec9fb60b67b9df9bc362e5cab40eba7a Mon Sep 17 00:00:00 2001 From: Sergey Kosarev Date: Wed, 6 Jun 2018 11:19:26 +0300 Subject: [PATCH 0226/1463] IGNITE-8587 GridToStringBuilder use ConcurrentHashMap to avoid global locks on classCache IGNITE-8587 GridToStringBuilder use ConcurrentHashMap to avoid global locks on classCache - Fixes #4059. Signed-off-by: Alexey Goncharuk (cherry picked from commit ecd8261f1add62099fde39aa7dca49855a866eda) --- .../util/tostring/GridToStringBuilder.java | 38 +++---------------- 1 file changed, 5 insertions(+), 33 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java index d36111263dd01..77c333daa774c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java @@ -28,15 +28,14 @@ import java.util.Collection; import java.util.Collections; import java.util.EventListener; -import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Queue; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.internal.util.typedef.F; @@ -85,10 +84,7 @@ */ public class GridToStringBuilder { /** */ - private static final Map classCache = new HashMap<>(); - - /** */ - private static final ReadWriteLock rwLock = new ReentrantReadWriteLock(); + private static final Map classCache = new ConcurrentHashMap<>(); /** {@link IgniteSystemProperties#IGNITE_TO_STRING_INCLUDE_SENSITIVE} */ public static final boolean INCLUDE_SENSITIVE = @@ -1017,16 +1013,10 @@ private static String toStringImpl( } // Specifically catching all exceptions. catch (Exception e) { - rwLock.writeLock().lock(); // Remove entry from cache to avoid potential memory leak // in case new class loader got loaded under the same identity hash. - try { - classCache.remove(cls.getName() + System.identityHashCode(cls.getClassLoader())); - } - finally { - rwLock.writeLock().unlock(); - } + classCache.remove(cls.getName() + System.identityHashCode(cls.getClassLoader())); // No other option here. throw new IgniteException(e); @@ -1737,14 +1727,7 @@ private static GridToStringClassDescriptor getClassDescriptor(Class cls) GridToStringClassDescriptor cd; - rwLock.readLock().lock(); - - try { - cd = classCache.get(key); - } - finally { - rwLock.readLock().unlock(); - } + cd = classCache.get(key); if (cd == null) { cd = new GridToStringClassDescriptor(cls); @@ -1806,18 +1789,7 @@ else if (!f.isAnnotationPresent(GridToStringExclude.class) && cd.sortFields(); - /* - * Allow multiple puts for the same class - they will simply override. - */ - - rwLock.writeLock().lock(); - - try { - classCache.put(key, cd); - } - finally { - rwLock.writeLock().unlock(); - } + classCache.putIfAbsent(key, cd); } return cd; From 05c8e05a181b5151197c28ee2a604101ee6cb481 Mon Sep 17 00:00:00 2001 From: Ilya Lantukh Date: Tue, 5 Jun 2018 01:24:37 +0300 Subject: [PATCH 0227/1463] IGNITE-8693 SQL JOIN between PARTITIONED and REPLICATED cache fails - Fixes #4120. Signed-off-by: Ivan Rakov (cherry picked from commit bc35ce0) --- .../h2/twostep/GridReduceQueryExecutor.java | 15 ++ ...QueryJoinWithDifferentNodeFiltersTest.java | 163 ++++++++++++++++++ .../IgniteCacheQuerySelfTestSuite2.java | 3 + 3 files changed, 181 insertions(+) create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/QueryJoinWithDifferentNodeFiltersTest.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 20bf32c5bc0d5..cd76bc1977bc3 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 @@ -452,6 +452,21 @@ private Map stableDataNodes(boolean isReplicatedOnly, Aff List cacheIds, int[] parts) { GridCacheContext cctx = cacheContext(cacheIds.get(0)); + // If the first cache is not partitioned, find it (if it's present) and move it to index 0. + if (!cctx.isPartitioned()) { + for (int cacheId = 1; cacheId < cacheIds.size(); cacheId++) { + GridCacheContext currCctx = cacheContext(cacheIds.get(cacheId)); + + if (currCctx.isPartitioned()) { + Collections.swap(cacheIds, 0, cacheId); + + cctx = currCctx; + + break; + } + } + } + Map map = stableDataNodesMap(topVer, cctx, parts); Set nodes = map.keySet(); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/QueryJoinWithDifferentNodeFiltersTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/QueryJoinWithDifferentNodeFiltersTest.java new file mode 100644 index 0000000000000..47666b93cc04b --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/QueryJoinWithDifferentNodeFiltersTest.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.*; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * + */ +public class QueryJoinWithDifferentNodeFiltersTest extends GridCommonAbstractTest { + /** */ + private static final String CACHE_NAME = "cache"; + + /** */ + private static final String CACHE_NAME_2 = "cache2"; + + /** */ + private static final int NODE_COUNT = 4; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setCacheConfiguration( + new CacheConfiguration<>(CACHE_NAME) + .setBackups(1) + .setCacheMode(CacheMode.REPLICATED) + .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC) + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL) + .setIndexedTypes(Integer.class, Organization.class), + new CacheConfiguration<>(CACHE_NAME_2) + .setNodeFilter(new TestFilter()) + .setBackups(1) + .setCacheMode(CacheMode.PARTITIONED) + .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC) + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL) + .setIndexedTypes(Integer.class, Person.class) + ); + + if (getTestIgniteInstanceName(0).equals(igniteInstanceName) || getTestIgniteInstanceName(1).equals(igniteInstanceName)) + cfg.setUserAttributes(F.asMap("DATA", "true")); + + if ("client".equals(igniteInstanceName)) + cfg.setClientMode(true); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), "snapshot", false)); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), "snapshot", false)); + } + + /** + * @throws Exception if failed. + */ + public void testSize() throws Exception { + startGrids(NODE_COUNT); + + Ignite client = startGrid("client"); + + client.cluster().active(true); + + IgniteCache cache = client.cache(CACHE_NAME); + IgniteCache cache2 = client.cache(CACHE_NAME_2); + + int size = 100; + + for (int i = 0; i < size; i++) { + cache.put(i, new Organization(i, "Org-" + i)); + cache2.put(i, new Person(i, i, "Person-" + i)); + } + + info(cache2.query(new SqlFieldsQuery("select * from \"cache\".Organization r, \"cache2\".Person p where p.orgId=r.orgId")).getAll().toString()); + } + + /** + * + */ + private static class Organization { + /** */ + @SuppressWarnings("unused") @QuerySqlField(index = true) private int orgId; + + /** */ + @SuppressWarnings("unused") private String orgName; + + /** + * + */ + public Organization(int orgId, String orgName) { + this.orgId = orgId; + this.orgName = orgName; + } + } + + /** + * + */ + private static class Person { + /** */ + @SuppressWarnings("unused") @QuerySqlField(index = true) private int personId; + + /** */ + @SuppressWarnings("unused") @QuerySqlField(index = true) private int orgId; + + /** */ + @SuppressWarnings("unused") private String name; + + /** + * + */ + public Person(int personId, int orgId, String name) { + this.personId = personId; + this.orgId = orgId; + this.name = name; + } + } + + /** + * + */ + private static class TestFilter implements IgnitePredicate { + /** {@inheritDoc} */ + @Override public boolean apply(ClusterNode clusterNode) { + return clusterNode.attribute("DATA") != null; + } + } +} \ No newline at end of file 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 5b888ce7c65f8..1b76283f8d685 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 @@ -18,6 +18,7 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; +import org.apache.ignite.internal.processors.cache.QueryJoinWithDifferentNodeFiltersTest; import org.apache.ignite.internal.processors.cache.CacheScanPartitionQueryFallbackSelfTest; import org.apache.ignite.internal.processors.cache.IgniteCacheCrossCacheJoinRandomTest; import org.apache.ignite.internal.processors.cache.IgniteCacheObjectKeyIndexingSelfTest; @@ -104,6 +105,8 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteCacheGroupsSqlSegmentedIndexMultiNodeSelfTest.class); suite.addTestSuite(IgniteCacheGroupsSqlDistributedJoinSelfTest.class); + suite.addTestSuite(QueryJoinWithDifferentNodeFiltersTest.class); + return suite; } } From 581f4924121f608280b874de5806cc29a17f9d25 Mon Sep 17 00:00:00 2001 From: Sergey Kosarev Date: Wed, 6 Jun 2018 12:59:28 +0300 Subject: [PATCH 0228/1463] IGNITE-8696 control.sh utility does not show atomicity mode - Fixes #4127. Signed-off-by: Ivan Rakov (cherry picked from commit 78e5d970be74c04b01857123b1a623038aa18440) --- .../processors/cache/verify/CacheInfo.java | 30 +++++++++++++++++-- .../cache/verify/ViewCacheClosure.java | 2 ++ 2 files changed, 30 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/CacheInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/CacheInfo.java index 9a090a042e961..31c0b3fcbe2d0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/CacheInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/CacheInfo.java @@ -20,6 +20,8 @@ import java.io.IOException; import java.io.ObjectInput; import java.io.ObjectOutput; + +import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.util.typedef.internal.S; @@ -67,6 +69,9 @@ public class CacheInfo extends VisorDataTransferObject { /** Mode. */ private CacheMode mode; + /** Atomicity mode. */ + private CacheAtomicityMode atomicityMode; + /** Backups count. */ private int backupsCnt; @@ -225,6 +230,20 @@ public void setMode(CacheMode mode) { this.mode = mode; } + /** + * + */ + public CacheAtomicityMode getAtomicityMode() { + return atomicityMode; + } + + /** + * @param atomicityMode + */ + public void setAtomicityMode(CacheAtomicityMode atomicityMode) { + this.atomicityMode = atomicityMode; + } + /** * */ @@ -269,18 +288,23 @@ public void print(VisorViewCacheCmd cmd) { case GROUPS: System.out.println("[grpName=" + getGrpName() + ", grpId=" + getGrpId() + ", cachesCnt=" + getCachesCnt() + ", prim=" + getPartitions() + ", mapped=" + getMapped() + ", mode=" + getMode() + - ", backups=" + getBackupsCnt() + ", affCls=" + getAffinityClsName() + ']'); + ", atomicity=" + getAtomicityMode() + ", backups=" + getBackupsCnt() + ", affCls=" + getAffinityClsName() + ']'); break; default: System.out.println("[cacheName=" + getCacheName() + ", cacheId=" + getCacheId() + ", grpName=" + getGrpName() + ", grpId=" + getGrpId() + ", prim=" + getPartitions() + - ", mapped=" + getMapped() + ", mode=" + getMode() + + ", mapped=" + getMapped() + ", mode=" + getMode() + ", atomicity=" + getAtomicityMode() + ", backups=" + getBackupsCnt() + ", affCls=" + getAffinityClsName() + ']'); } } + /** {@inheritDoc} */ + @Override public byte getProtocolVersion() { + return V2; + } + /** {@inheritDoc} */ @Override protected void writeExternalData(ObjectOutput out) throws IOException { U.writeString(out, seqName); @@ -296,6 +320,7 @@ public void print(VisorViewCacheCmd cmd) { out.writeInt(backupsCnt); U.writeString(out, affinityClsName); out.writeInt(cachesCnt); + U.writeEnum(out, atomicityMode); } /** {@inheritDoc} */ @@ -313,6 +338,7 @@ public void print(VisorViewCacheCmd cmd) { backupsCnt = in.readInt(); affinityClsName = U.readString(in); cachesCnt = in.readInt(); + atomicityMode = protoVer >= V2 ? CacheAtomicityMode.fromOrdinal(in.readByte()) : null; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/ViewCacheClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/ViewCacheClosure.java index 1f363f3643db7..9003ac00257f1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/ViewCacheClosure.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/ViewCacheClosure.java @@ -99,6 +99,7 @@ public ViewCacheClosure(String regex, VisorViewCacheCmd cmd) { ci.setBackupsCnt(context.config().getBackups()); ci.setAffinityClsName(context.config().getAffinity().getClass().getSimpleName()); ci.setMode(context.config().getCacheMode()); + ci.setAtomicityMode(context.config().getAtomicityMode()); ci.setMapped(mapped(context.caches().iterator().next().name())); cacheInfo.add(ci); @@ -126,6 +127,7 @@ public ViewCacheClosure(String regex, VisorViewCacheCmd cmd) { ci.setBackupsCnt(desc.cacheConfiguration().getBackups()); ci.setAffinityClsName(desc.cacheConfiguration().getAffinity().getClass().getSimpleName()); ci.setMode(desc.cacheConfiguration().getCacheMode()); + ci.setAtomicityMode(desc.cacheConfiguration().getAtomicityMode()); ci.setMapped(mapped(desc.cacheName())); cacheInfo.add(ci); From 476aa99627a378586bbdd4d856cd08f6150ad433 Mon Sep 17 00:00:00 2001 From: Aleksei Scherbakov Date: Thu, 3 May 2018 19:25:23 +0300 Subject: [PATCH 0229/1463] IGNITE-8412 Issues with cache names in GridCommandHandlerTest broke corresponding tests in security module. - Fixes #3937. Signed-off-by: dpavlov (cherry picked from commit c2369ff) --- .../ignite/util/GridCommandHandlerTest.java | 27 +++++++------------ 1 file changed, 9 insertions(+), 18 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java index 3ec0617e9164a..f6d1e1930d6af 100644 --- a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java @@ -582,7 +582,7 @@ public void testCacheIdleVerify() throws Exception { IgniteCache cache = ignite.createCache(new CacheConfiguration<>() .setAffinity(new RendezvousAffinityFunction(false, 32)) .setBackups(1) - .setName("cacheIV")); + .setName(DEFAULT_CACHE_NAME)); for (int i = 0; i < 100; i++) cache.put(i, i); @@ -595,7 +595,7 @@ public void testCacheIdleVerify() throws Exception { HashSet clearKeys = new HashSet<>(Arrays.asList(1, 2, 3, 4, 5, 6)); - ((IgniteEx)ignite).context().cache().cache("cacheIV").clearLocallyAll(clearKeys, true, true, true); + ((IgniteEx)ignite).context().cache().cache(DEFAULT_CACHE_NAME).clearLocallyAll(clearKeys, true, true, true); assertEquals(EXIT_CODE_OK, execute("--cache", "idle_verify")); @@ -619,7 +619,7 @@ public void testCacheContention() throws Exception { .setAffinity(new RendezvousAffinityFunction(false, 32)) .setAtomicityMode(TRANSACTIONAL) .setBackups(1) - .setName("cacheCont")); + .setName(DEFAULT_CACHE_NAME)); final CountDownLatch l = new CountDownLatch(1); @@ -707,23 +707,14 @@ public void testCacheGroups() throws Exception { ignite.cluster().active(true); - IgniteCache cache1 = ignite.createCache(new CacheConfiguration<>() - .setAffinity(new RendezvousAffinityFunction(false, 32)) - .setBackups(1) - .setGroupName("G100") - .setName("cacheG1")); - - IgniteCache cache2 = ignite.createCache(new CacheConfiguration<>() + IgniteCache cache = ignite.createCache(new CacheConfiguration<>() .setAffinity(new RendezvousAffinityFunction(false, 32)) .setBackups(1) .setGroupName("G100") - .setName("cacheG2")); + .setName(DEFAULT_CACHE_NAME)); - for (int i = 0; i < 100; i++) { - cache1.put(i, i); - - cache2.put(i, i); - } + for (int i = 0; i < 100; i++) + cache.put(i, i); injectTestSystemOut(); @@ -743,7 +734,7 @@ public void testCacheAffinity() throws Exception { IgniteCache cache1 = ignite.createCache(new CacheConfiguration<>() .setAffinity(new RendezvousAffinityFunction(false, 32)) .setBackups(1) - .setName("cacheAf")); + .setName(DEFAULT_CACHE_NAME)); for (int i = 0; i < 100; i++) cache1.put(i, i); @@ -752,7 +743,7 @@ public void testCacheAffinity() throws Exception { assertEquals(EXIT_CODE_OK, execute("--cache", "list", ".*")); - assertTrue(testOut.toString().contains("cacheName=cacheAf")); + assertTrue(testOut.toString().contains("cacheName=" + DEFAULT_CACHE_NAME)); assertTrue(testOut.toString().contains("prim=32")); assertTrue(testOut.toString().contains("mapped=32")); assertTrue(testOut.toString().contains("affCls=RendezvousAffinityFunction")); From 53d8ceb136f4cb665920b49703426e0e3a86d953 Mon Sep 17 00:00:00 2001 From: Andrey Kuznetsov Date: Wed, 6 Jun 2018 15:02:35 +0300 Subject: [PATCH 0230/1463] IGNITE-8642 Added thread dumping to FailureProcessor Signed-off-by: Andrey Gura --- .../java/org/apache/ignite/IgniteSystemProperties.java | 7 +++++++ .../internal/processors/failure/FailureProcessor.java | 9 +++++++++ 2 files changed, 16 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index 727e8092b7cf7..83d15f0bba30a 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -898,6 +898,13 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_SKIP_PARTITION_SIZE_VALIDATION = "IGNITE_SKIP_PARTITION_SIZE_VALIDATION"; + /** + * Enables threads dumping on critical node failure. + * + * Default is {@code true}. + */ + public static final String IGNITE_DUMP_THREADS_ON_FAILURE = "IGNITE_DUMP_THREADS_ON_FAILURE"; + /** * Enforces singleton. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java index b11cb95126674..722de1870c6ef 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java @@ -29,10 +29,16 @@ import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_DUMP_THREADS_ON_FAILURE; + /** * General failure processing API */ public class FailureProcessor extends GridProcessorAdapter { + /** Value of the system property that enables threads dumping on failure. */ + private static final boolean IGNITE_DUMP_THREADS_ON_FAILURE = + IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_DUMP_THREADS_ON_FAILURE, true); + /** Ignite. */ private final Ignite ignite; @@ -115,6 +121,9 @@ public synchronized void process(FailureContext failureCtx, FailureHandler hnd) if (reserveBuf != null && X.hasCause(failureCtx.error(), OutOfMemoryError.class)) reserveBuf = null; + if (IGNITE_DUMP_THREADS_ON_FAILURE) + U.dumpThreads(log); + boolean invalidated = hnd.onFailure(ignite, failureCtx); if (invalidated) { From 809b3bf3f1afab52621a8b00e9ebc8c352633af6 Mon Sep 17 00:00:00 2001 From: Dmitriy Sorokin Date: Wed, 6 Jun 2018 14:34:04 +0300 Subject: [PATCH 0231/1463] IGNITE-8311 IgniteClientRejoinTest.testClientsReconnectDisabled causes exchange-worker to terminate via NPE Signed-off-by: Andrey Gura --- .../GridCachePartitionExchangeManager.java | 4 +- .../failure/FailureHandlerTriggeredTest.java | 35 ----------- .../ignite/failure/TestFailureHandler.java | 63 +++++++++++++++++++ .../internal/IgniteClientRejoinTest.java | 11 +++- .../IgniteChangeGlobalStateTest.java | 2 + .../CacheHibernateStoreFactorySelfTest.java | 2 + .../CacheHibernateStoreFactorySelfTest.java | 2 + .../query/IgniteSqlSchemaIndexingTest.java | 5 +- .../CacheJdbcBlobStoreFactorySelfTest.java | 2 + .../CacheJdbcPojoStoreFactorySelfTest.java | 2 + 10 files changed, 90 insertions(+), 38 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/failure/TestFailureHandler.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index 6533bbc5f9f89..3a990892bfe8d 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 @@ -2358,7 +2358,7 @@ else if (err != null) /** * */ - private void body0() throws InterruptedException, IgniteInterruptedCheckedException { + private void body0() throws InterruptedException, IgniteCheckedException { long timeout = cctx.gridConfig().getNetworkTimeout(); long cnt = 0; @@ -2662,6 +2662,8 @@ else if (r != null) { catch (IgniteCheckedException e) { U.error(log, "Failed to wait for completion of partition map exchange " + "(preloading will not start): " + task, e); + + throw e; } } } diff --git a/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlerTriggeredTest.java b/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlerTriggeredTest.java index 5eca7d698af7c..8d56ced75f2cc 100644 --- a/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlerTriggeredTest.java +++ b/modules/core/src/test/java/org/apache/ignite/failure/FailureHandlerTriggeredTest.java @@ -19,7 +19,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import org.apache.ignite.Ignite; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.processors.cache.CachePartitionExchangeWorkerTask; @@ -66,40 +65,6 @@ public void testFailureHandlerTriggeredOnExchangeWorkerTermination() throws Exce } } - /** - * Test failure handler implementation - */ - private class TestFailureHandler implements FailureHandler { - /** Invalidate. */ - private final boolean invalidate; - - /** Latch. */ - private final CountDownLatch latch; - - /** Failure context. */ - volatile FailureContext failureCtx; - - /** - * @param invalidate Invalidate. - * @param latch Latch. - */ - TestFailureHandler(boolean invalidate, CountDownLatch latch) { - this.invalidate = invalidate; - this.latch = latch; - } - - /** {@inheritDoc} */ - @Override public boolean onFailure(Ignite ignite, FailureContext failureCtx) { - this.failureCtx = failureCtx; - - this.latch.countDown(); - - ignite.log().warning("Handled ignite failure: " + failureCtx); - - return invalidate; - } - } - /** * Custom exchange worker task implementation for delaying exchange worker processing. */ diff --git a/modules/core/src/test/java/org/apache/ignite/failure/TestFailureHandler.java b/modules/core/src/test/java/org/apache/ignite/failure/TestFailureHandler.java new file mode 100644 index 0000000000000..1159683e6b54f --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/failure/TestFailureHandler.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.failure; + +import java.util.concurrent.CountDownLatch; +import org.apache.ignite.Ignite; + +/** + * Test failure handler implementation + */ +public class TestFailureHandler implements FailureHandler { + /** Invalidate. */ + private final boolean invalidate; + + /** Latch. */ + private final CountDownLatch latch; + + /** Failure context. */ + volatile FailureContext failureCtx; + + /** + * @param invalidate Invalidate. + * @param latch Latch. + */ + public TestFailureHandler(boolean invalidate, CountDownLatch latch) { + this.invalidate = invalidate; + this.latch = latch; + } + + /** {@inheritDoc} */ + @Override public boolean onFailure(Ignite ignite, FailureContext failureCtx) { + this.failureCtx = failureCtx; + + if (latch != null) + latch.countDown(); + + ignite.log().warning("Handled ignite failure: " + failureCtx); + + return invalidate; + } + + /** + * @return Failure context. + */ + public FailureContext failureContext() { + return failureCtx; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java index 462c8ab6197d8..9a98a888aa2cb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientRejoinTest.java @@ -27,6 +27,7 @@ import java.util.Random; import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; @@ -35,6 +36,7 @@ import org.apache.ignite.Ignition; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.failure.TestFailureHandler; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.plugin.extensions.communication.Message; @@ -282,6 +284,8 @@ public void testClientsReconnectDisabled() throws Exception { final int CLIENTS_NUM = 5; + final CountDownLatch failureHndLatch = new CountDownLatch(CLIENTS_NUM); + for (int i = 0; i < CLIENTS_NUM; i++) { final int idx = i; @@ -289,7 +293,10 @@ public void testClientsReconnectDisabled() throws Exception { @Override public Ignite call() throws Exception { latch.await(); - return startGrid("client" + idx); + String igniteInstanceName = "client" + idx; + + return startGrid(igniteInstanceName, getConfiguration(igniteInstanceName) + .setFailureHandler(new TestFailureHandler(true, failureHndLatch))); } }); @@ -309,6 +316,8 @@ public void testClientsReconnectDisabled() throws Exception { }, IgniteCheckedException.class, null); } + assertTrue(failureHndLatch.await(1000, TimeUnit.MILLISECONDS)); + assertEquals(0, srv1.cluster().forClients().nodes().size()); assertEquals(0, srv2.cluster().forClients().nodes().size()); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateTest.java index cbf0ec9256b2e..9152ab90319df 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateTest.java @@ -543,6 +543,8 @@ public void testFailGetLock() throws Exception { * @throws Exception If failed. */ public void testActivateAfterFailGetLock() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-1094"); + Ignite ig1P = primary(0); Ignite ig2P = primary(1); Ignite ig3P = primary(2); diff --git a/modules/hibernate-4.2/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreFactorySelfTest.java b/modules/hibernate-4.2/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreFactorySelfTest.java index 0dab22cbb0748..db0e1bebe8026 100644 --- a/modules/hibernate-4.2/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreFactorySelfTest.java +++ b/modules/hibernate-4.2/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreFactorySelfTest.java @@ -83,6 +83,8 @@ public void testXmlConfiguration() throws Exception { * @throws Exception If failed. */ public void testIncorrectBeanConfiguration() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-1094"); + GridTestUtils.assertThrows(log, new Callable() { @Override public Object call() throws Exception { try(Ignite ignite = diff --git a/modules/hibernate-5.1/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreFactorySelfTest.java b/modules/hibernate-5.1/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreFactorySelfTest.java index a329f2b7f5b6f..0ffe52ef1176f 100644 --- a/modules/hibernate-5.1/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreFactorySelfTest.java +++ b/modules/hibernate-5.1/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreFactorySelfTest.java @@ -83,6 +83,8 @@ public void testXmlConfiguration() throws Exception { * @throws Exception If failed. */ public void testIncorrectBeanConfiguration() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-1094"); + GridTestUtils.assertThrows(log, new Callable() { @Override public Object call() throws Exception { try(Ignite ignite = diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSchemaIndexingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSchemaIndexingTest.java index e375df2cf6e7e..2dee617502424 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSchemaIndexingTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSchemaIndexingTest.java @@ -87,7 +87,8 @@ private static CacheConfiguration cacheConfig(String name, boolean partitioned, */ public void testCaseSensitive() throws Exception { //TODO rewrite with dynamic cache creation, and GRID start in #beforeTest after resolve of - //https://issues.apache.org/jira/browse/IGNITE-1094 + //TODO https://issues.apache.org/jira/browse/IGNITE-1094 + fail("https://issues.apache.org/jira/browse/IGNITE-1094"); GridTestUtils.assertThrows(log, new Callable() { @Override public Object call() throws Exception { @@ -117,6 +118,8 @@ public void testCaseSensitive() throws Exception { public void testCustomSchemaMultipleCachesTablesCollision() throws Exception { //TODO: Rewrite with dynamic cache creation, and GRID start in #beforeTest after resolve of //TODO: https://issues.apache.org/jira/browse/IGNITE-1094 + fail("https://issues.apache.org/jira/browse/IGNITE-1094"); + GridTestUtils.assertThrows(log, new Callable() { @Override public Object call() throws Exception { final CacheConfiguration cfg = cacheConfig("cache1", true, Integer.class, Fact.class) diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactorySelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactorySelfTest.java index bc4f2684dfffd..be2238bd8bf93 100644 --- a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactorySelfTest.java +++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactorySelfTest.java @@ -77,6 +77,8 @@ public void testCacheConfiguration() throws Exception { * @throws Exception If failed. */ public void testIncorrectBeanConfiguration() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-1094"); + GridTestUtils.assertThrows(log, new Callable() { @Override public Object call() throws Exception { try(Ignite ignite = Ignition.start("modules/spring/src/test/config/incorrect-store-cache.xml")) { diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java index 0d6d6a28279fe..ba4984d22cab9 100644 --- a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java +++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java @@ -69,6 +69,8 @@ public void testSerializable() throws Exception { * @throws Exception If failed. */ public void testIncorrectBeanConfiguration() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-1094"); + GridTestUtils.assertThrowsAnyCause(log, new Callable() { @Override public Object call() throws Exception { try (Ignite ignored = Ignition.start("modules/spring/src/test/config/pojo-incorrect-store-cache.xml")) { From 6bf0dbd2e3095bfbaef91f228587af539eb747ea Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 6 Jun 2018 15:24:39 +0300 Subject: [PATCH 0232/1463] IGNITE-8482 Skip 2-phase partition release wait in case of activation or dynamic caches start - Fixes #4078. Signed-off-by: Alexey Goncharuk (cherry picked from commit 7c565d2) --- .../GridDhtPartitionsExchangeFuture.java | 32 ++++++++++++++----- .../preloader/latch/ExchangeLatchManager.java | 18 ++++++----- 2 files changed, 34 insertions(+), 16 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java index 7ecebdd190179..0c51835fbfd30 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 @@ -520,6 +520,13 @@ public boolean changedBaseline() { return exchActions != null && exchActions.changedBaseline(); } + /** + * @return {@code True} if there are caches to start. + */ + public boolean hasCachesToStart() { + return exchActions != null && !exchActions.cacheStartRequests().isEmpty(); + } + /** * @return First event discovery event. * @@ -1132,11 +1139,17 @@ private void distributedExchange() throws IgniteCheckedException { // To correctly rebalance when persistence is enabled, it is necessary to reserve history within exchange. partHistReserved = cctx.database().reserveHistoryForExchange(); - // On first phase we wait for finishing all local tx updates, atomic updates and lock releases. - waitPartitionRelease(1); + boolean distributed = true; + + // Do not perform distributed partition release in case of cluster activation or caches start. + if (activateCluster() || hasCachesToStart()) + distributed = false; + + // On first phase we wait for finishing all local tx updates, atomic updates and lock releases on all nodes. + waitPartitionRelease(distributed); // Second phase is needed to wait for finishing all tx updates from primary to backup nodes remaining after first phase. - waitPartitionRelease(2); + waitPartitionRelease(false); boolean topChanged = firstDiscoEvt.type() != EVT_DISCOVERY_CUSTOM_EVT || affChangeMsg != null; @@ -1241,15 +1254,14 @@ private void changeWalModeIfNeeded() { * For the exact list of the objects being awaited for see * {@link GridCacheSharedContext#partitionReleaseFuture(AffinityTopologyVersion)} javadoc. * - * @param phase Phase of partition release. + * @param distributed If {@code true} then node should wait for partition release completion on all other nodes. * * @throws IgniteCheckedException If failed. */ - private void waitPartitionRelease(int phase) throws IgniteCheckedException { + private void waitPartitionRelease(boolean distributed) throws IgniteCheckedException { Latch releaseLatch = null; - // Wait for other nodes only on first phase. - if (phase == 1) + if (distributed) releaseLatch = cctx.exchange().latch().getOrCreate("exchange", initialVersion()); IgniteInternalFuture partReleaseFut = cctx.partitionReleaseFuture(initialVersion()); @@ -2597,7 +2609,11 @@ private void finishExchangeOnCoordinator(@Nullable Collection sndRe } } - validatePartitionsState(); + // Don't validate partitions state in case of caches start. + boolean skipValidation = hasCachesToStart(); + + if (!skipValidation) + 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/latch/ExchangeLatchManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java index b9c7dee7ba6b2..25424b8822ba7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java @@ -452,11 +452,12 @@ class ServerLatch extends CompletableLatch { io.sendToGridTopic(node, GridTopic.TOPIC_EXCHANGE, new LatchAckMessage(id, topVer, true), GridIoPolicy.SYSTEM_POOL); if (log.isDebugEnabled()) - log.debug("Final ack is ackSent [latch=" + latchId() + ", to=" + node.id() + "]"); + log.debug("Final ack has sent [latch=" + latchId() + ", to=" + node.id() + "]"); } - } catch (IgniteCheckedException e) { + } + catch (IgniteCheckedException e) { if (log.isDebugEnabled()) - log.debug("Unable to send final ack [latch=" + latchId() + ", to=" + node.id() + "]"); + log.debug("Failed to send final ack [latch=" + latchId() + ", to=" + node.id() + "]: " + e.getMessage()); } } }); @@ -500,7 +501,7 @@ private void countDown0(UUID node) { int remaining = permits.decrementAndGet(); if (log.isDebugEnabled()) - log.debug("Count down + [latch=" + latchId() + ", remaining=" + remaining + "]"); + log.debug("Count down [latch=" + latchId() + ", remaining=" + remaining + "]"); if (remaining == 0) complete(); @@ -563,7 +564,7 @@ private boolean hasCoordinator(UUID node) { */ private void newCoordinator(ClusterNode coordinator) { if (log.isDebugEnabled()) - log.debug("Coordinator is changed [latch=" + latchId() + ", crd=" + coordinator.id() + "]"); + log.debug("Coordinator is changed [latch=" + latchId() + ", newCrd=" + coordinator.id() + "]"); synchronized (this) { this.coordinator = coordinator; @@ -585,11 +586,12 @@ private void sendAck() { io.sendToGridTopic(coordinator, GridTopic.TOPIC_EXCHANGE, new LatchAckMessage(id, topVer, false), GridIoPolicy.SYSTEM_POOL); if (log.isDebugEnabled()) - log.debug("Ack is ackSent + [latch=" + latchId() + ", to=" + coordinator.id() + "]"); - } catch (IgniteCheckedException e) { + log.debug("Ack has sent [latch=" + latchId() + ", to=" + coordinator.id() + "]"); + } + catch (IgniteCheckedException e) { // Coordinator is unreachable. On coodinator node left discovery event ack will be resent. if (log.isDebugEnabled()) - log.debug("Unable to send ack [latch=" + latchId() + ", to=" + coordinator.id() + "]: " + e.getMessage()); + log.debug("Failed to send ack [latch=" + latchId() + ", to=" + coordinator.id() + "]: " + e.getMessage()); } } From f330951aa4fd6f13c2f69c890e02c6620a374293 Mon Sep 17 00:00:00 2001 From: Dmitriy Govorukhin Date: Wed, 6 Jun 2018 17:20:18 +0300 Subject: [PATCH 0233/1463] IGNITE-8685 Fixed switch segment record size - Fixes #4130. Signed-off-by: Alexey Goncharuk (cherry picked from commit 2a048bd) --- .../wal/record/BaselineTopologyRecord.java | 76 ---- .../pagemem/wal/record/WALRecord.java | 12 +- .../wal/AbstractWalRecordsIterator.java | 15 +- .../reader/StandaloneGridKernalContext.java | 8 +- .../BaselineTopologyRecordSerializer.java | 168 -------- .../serializer/RecordDataV2Serializer.java | 16 - .../wal/serializer/RecordV1Serializer.java | 28 +- .../wal/serializer/RecordV2Serializer.java | 27 +- .../IgniteWalIteratorSwitchSegmentTest.java | 386 ++++++++++++++++++ .../testsuites/IgnitePdsTestSuite2.java | 1 + 10 files changed, 457 insertions(+), 280 deletions(-) delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/BaselineTopologyRecord.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/BaselineTopologyRecordSerializer.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/BaselineTopologyRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/BaselineTopologyRecord.java deleted file mode 100644 index 48b60b3924362..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/BaselineTopologyRecord.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.pagemem.wal.record; - -import java.util.Map; -import org.apache.ignite.internal.util.typedef.internal.S; - -/** - * Record for storing baseline topology compact node ID to consistent node ID mapping. - */ -public class BaselineTopologyRecord extends WALRecord { - /** Id. */ - private int id; - - /** Compact ID to consistent ID mapping. */ - private Map mapping; - - /** - * Default constructor. - */ - private BaselineTopologyRecord() { - // No-op, used from factory methods. - } - - /** - * @param id Baseline topology ID. - * @param mapping Compact ID to consistent ID mapping. - */ - public BaselineTopologyRecord(int id, Map mapping) { - this.id = id; - this.mapping = mapping; - } - - /** {@inheritDoc} */ - @Override public RecordType type() { - return RecordType.BASELINE_TOP_RECORD; - } - - /** - * Returns baseline topology ID. - * - * @return Baseline topology ID. - */ - public int id() { - return id; - } - - /** - * Returns mapping. - * - * @return Compact ID to consistent ID mapping. - */ - public Map mapping() { - return mapping; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(BaselineTopologyRecord.class, this, "super", super.toString()); - } -} 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 4fae179b63d28..87ba07db7f341 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 @@ -153,7 +153,13 @@ public enum RecordType { /** Page list meta reset count record. */ PAGE_LIST_META_RESET_COUNT_RECORD, - /** Switch segment record. */ + /** Switch segment record. + * Marker record for indicate end of segment. + * If the next one record is written down exactly at the end of segment, + * SWITCH_SEGMENT_RECORD will not be written, if not then it means that we have more + * that one byte in the end,then we write SWITCH_SEGMENT_RECORD as marker end of segment. + * No need write CRC or WAL pointer for this record. It is byte marker record. + * */ SWITCH_SEGMENT_RECORD, /** */ @@ -174,8 +180,8 @@ public enum RecordType { /** Exchange record. */ EXCHANGE, - /** Baseline topology record. */ - BASELINE_TOP_RECORD; + /** Reserved for future record. */ + RESERVED; /** */ private static final RecordType[] VALS = RecordType.values(); 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 d9312f6888b2f..e442386c03e87 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 @@ -190,18 +190,20 @@ protected void advance() throws IgniteCheckedException { /** * 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 - * @throws IgniteCheckedException if reading failed + * 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 + * @throws IgniteCheckedException if reading failed */ protected abstract AbstractReadFileHandle advanceSegment( @Nullable final AbstractReadFileHandle curWalSegment) throws IgniteCheckedException; /** - * Switches to new record - * @param hnd currently opened read handle - * @return next advanced record + * Switches to new record. + * + * @param hnd currently opened read handle. + * @return next advanced record. */ private IgniteBiTuple advanceRecord( @Nullable final AbstractReadFileHandle hnd @@ -242,7 +244,8 @@ private IgniteBiTuple advanceRecord( } /** - * Handler for record deserialization exception + * Handler for record deserialization exception. + * * @param e problem from records reading * @param ptr file pointer was accessed */ 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 cb04575c8810a..795d46004f727 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 @@ -128,9 +128,11 @@ public class StandaloneGridKernalContext implements GridKernalContext { * {@code null} means no specific folder is configured. * Providing {@code null} will disable unmarshall for non primitive objects, BinaryObjects will be provided
    */ - StandaloneGridKernalContext(IgniteLogger log, + public StandaloneGridKernalContext( + IgniteLogger log, @Nullable File binaryMetadataFileStoreDir, - @Nullable File marshallerMappingFileStoreDir) throws IgniteCheckedException { + @Nullable File marshallerMappingFileStoreDir + ) throws IgniteCheckedException { this.log = log; try { @@ -179,7 +181,7 @@ private IgniteCacheObjectProcessor binaryProcessor( /** * @return Ignite configuration which allows to start requied processors for WAL reader */ - private IgniteConfiguration prepareIgniteConfiguration() { + protected IgniteConfiguration prepareIgniteConfiguration() { IgniteConfiguration cfg = new IgniteConfiguration(); cfg.setDiscoverySpi(new StandaloneNoopDiscoverySpi()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/BaselineTopologyRecordSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/BaselineTopologyRecordSerializer.java deleted file mode 100644 index 94b51c5cdd8cb..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/BaselineTopologyRecordSerializer.java +++ /dev/null @@ -1,168 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.processors.cache.persistence.wal.serializer; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Map; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.pagemem.wal.record.BaselineTopologyRecord; -import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; -import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput; -import org.apache.ignite.internal.util.typedef.internal.U; - -/** - * {@link BaselineTopologyRecord} WAL serializer. - */ -public class BaselineTopologyRecordSerializer { - /** Cache shared context. */ - private GridCacheSharedContext cctx; - - /** Class loader to unmarshal consistent IDs. */ - private ClassLoader clsLdr; - - /** - * Create an instance of serializer. - * - * @param cctx Cache shared context. - */ - public BaselineTopologyRecordSerializer(GridCacheSharedContext cctx) { - this.cctx = cctx; - - clsLdr = U.resolveClassLoader(cctx.gridConfig()); - } - - /** - * Writes {@link BaselineTopologyRecord} to given buffer. - * - * @param rec {@link BaselineTopologyRecord} instance. - * @param buf Byte buffer. - * @throws IgniteCheckedException In case of fail. - */ - public void write(BaselineTopologyRecord rec, ByteBuffer buf) throws IgniteCheckedException { - buf.putInt(rec.id()); - - Map mapping = rec.mapping(); - - if (mapping != null && !mapping.isEmpty()) { - buf.putInt(mapping.size()); - - for (Map.Entry e : mapping.entrySet()) { - buf.putShort(e.getKey()); - - writeConsistentId(e.getValue(), buf); - } - } - else - buf.putInt(0); - } - - /** - * Reads {@link BaselineTopologyRecord} from given input. - * - * @param in Input - * @return BaselineTopologyRecord instance. - * @throws IOException In case of fail. - * @throws IgniteCheckedException In case of fail. - */ - public BaselineTopologyRecord read(ByteBufferBackedDataInput in) throws IOException, IgniteCheckedException { - int id = in.readInt(); - - int size = in.readInt(); - - Map mapping = size > 0 ? U.newHashMap(size) : null; - - for (int i = 0; i < size; i++) { - short compactId = in.readShort(); - - Object consistentId = readConsistentId(in); - - mapping.put(compactId, consistentId); - } - - return new BaselineTopologyRecord(id, mapping); - } - - /** - * Returns size of marshalled {@link BaselineTopologyRecord} in bytes. - * - * @param rec BaselineTopologyRecord instance. - * @return Size of BaselineTopologyRecord instance in bytes. - * @throws IgniteCheckedException In case of fail. - */ - public int size(BaselineTopologyRecord rec) throws IgniteCheckedException { - int size = 0; - - size += /* Baseline topology ID. */ 4; - - size += /* Consistent ID mapping size. */ 4; - - if (rec.mapping() != null) { - for (Object consistentId : rec.mapping().values()) { - size += /* Compact ID size */ 2; - - size += marshalConsistentId(consistentId).length; - } - } - - return size; - } - - /** - * Write consistent id to given buffer. - * - * @param consistentId Consistent id. - * @param buf Byte buffer. - * @throws IgniteCheckedException In case of fail. - */ - private void writeConsistentId(Object consistentId, ByteBuffer buf) throws IgniteCheckedException { - byte[] content = marshalConsistentId(consistentId); - - buf.putInt(content.length); - buf.put(content); - } - - /** - * Read consistent id from given input. - * - * @param in Input. - * @return Consistent id. - * @throws IOException In case of fail. - * @throws IgniteCheckedException In case of fail. - */ - private Object readConsistentId(ByteBufferBackedDataInput in) throws IOException, IgniteCheckedException { - int len = in.readInt(); - in.ensure(len); - - byte[] content = new byte[len]; - in.readFully(content); - - return cctx.marshaller().unmarshal(content, clsLdr); - } - - /** - * Marshal consistent id to byte array. - * - * @param consistentId Consistent id. - * @return Marshalled byte array. - * @throws IgniteCheckedException In case of fail. - */ - private byte[] marshalConsistentId(Object consistentId) throws IgniteCheckedException { - return cctx.marshaller().marshal(consistentId); - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java index b3a00beeb6aa8..b760547e6e933 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java @@ -27,7 +27,6 @@ import java.util.Map; import java.util.UUID; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.pagemem.wal.record.BaselineTopologyRecord; import org.apache.ignite.internal.pagemem.wal.record.CacheState; import org.apache.ignite.internal.pagemem.wal.record.CheckpointRecord; import org.apache.ignite.internal.pagemem.wal.record.DataEntry; @@ -53,9 +52,6 @@ public class RecordDataV2Serializer implements RecordDataSerializer { /** Serializer of {@link TxRecord} records. */ private final TxRecordSerializer txRecordSerializer; - /** Serializer of {@link BaselineTopologyRecord} records. */ - private final BaselineTopologyRecordSerializer bltRecSerializer; - /** * Create an instance of V2 data serializer. * @@ -64,7 +60,6 @@ public class RecordDataV2Serializer implements RecordDataSerializer { public RecordDataV2Serializer(RecordDataV1Serializer delegateSerializer) { this.delegateSerializer = delegateSerializer; this.txRecordSerializer = new TxRecordSerializer(); - this.bltRecSerializer = new BaselineTopologyRecordSerializer(delegateSerializer.cctx()); } /** {@inheritDoc} */ @@ -97,9 +92,6 @@ public RecordDataV2Serializer(RecordDataV1Serializer delegateSerializer) { case TX_RECORD: return txRecordSerializer.size((TxRecord)rec); - case BASELINE_TOP_RECORD: - return bltRecSerializer.size((BaselineTopologyRecord)rec); - default: return delegateSerializer.size(rec); } @@ -157,9 +149,6 @@ public RecordDataV2Serializer(RecordDataV1Serializer delegateSerializer) { case TX_RECORD: return txRecordSerializer.read(in); - case BASELINE_TOP_RECORD: - return bltRecSerializer.read(in); - default: return delegateSerializer.readRecord(type, in); } @@ -231,11 +220,6 @@ public RecordDataV2Serializer(RecordDataV1Serializer delegateSerializer) { break; - case BASELINE_TOP_RECORD: - bltRecSerializer.write((BaselineTopologyRecord)rec, buf); - - break; - default: delegateSerializer.writeRecord(rec, buf); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java index dd0819c859ce8..caa096294e88b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java @@ -47,6 +47,7 @@ import org.apache.ignite.lang.IgniteBiPredicate; import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_SKIP_CRC; +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.SWITCH_SEGMENT_RECORD; /** * Record V1 serializer. @@ -111,7 +112,13 @@ public class RecordV1Serializer implements RecordSerializer { /** {@inheritDoc} */ @Override public int sizeWithHeaders(WALRecord record) throws IgniteCheckedException { - return dataSerializer.size(record) + REC_TYPE_SIZE + FILE_WAL_POINTER_SIZE + CRC_SIZE; + int recordSize = dataSerializer.size(record); + + int recordSizeWithType = recordSize + REC_TYPE_SIZE; + + // Why this condition here, see SWITCH_SEGMENT_RECORD doc. + return record.type() != SWITCH_SEGMENT_RECORD ? + recordSizeWithType + FILE_WAL_POINTER_SIZE + CRC_SIZE : recordSizeWithType; } /** {@inheritDoc} */ @@ -160,6 +167,10 @@ else if (marshalledMode) { // Write record type. putRecordType(buf, rec); + // SWITCH_SEGMENT_RECORD should have only type, no need to write pointer. + if (rec.type() == SWITCH_SEGMENT_RECORD) + return; + // Write record file position. putPositionOfRecord(buf, rec); @@ -176,8 +187,13 @@ else if (marshalledMode) { * @param skipPositionCheck Skip position check mode. * @param recordFilter Record type filter. {@link FilteredRecord} is deserialized instead of original record */ - public RecordV1Serializer(RecordDataV1Serializer dataSerializer, boolean writePointer, - boolean marshalledMode, boolean skipPositionCheck, IgniteBiPredicate recordFilter) { + public RecordV1Serializer( + RecordDataV1Serializer dataSerializer, + boolean writePointer, + boolean marshalledMode, + boolean skipPositionCheck, + IgniteBiPredicate recordFilter + ) { this.dataSerializer = dataSerializer; this.writePointer = writePointer; this.recordFilter = recordFilter; @@ -376,10 +392,16 @@ static WALRecord readWithCrc(FileInput in0, WALPointer expPtr, RecordIO reader) static void writeWithCrc(WALRecord rec, ByteBuffer buf, RecordIO writer) throws IgniteCheckedException { assert rec.size() >= 0 && buf.remaining() >= rec.size() : rec.size(); + boolean switchSegmentRec = rec.type() == RecordType.SWITCH_SEGMENT_RECORD; + int startPos = buf.position(); writer.writeWithHeaders(rec, buf); + // No need calculate and write CRC for SWITCH_SEGMENT_RECORD. + if (switchSegmentRec) + return; + if (!skipCrc) { int curPos = buf.position(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java index feeb810a94320..2b81210e6fd0a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java @@ -38,6 +38,7 @@ import org.apache.ignite.lang.IgniteBiPredicate; import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType; +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.SWITCH_SEGMENT_RECORD; import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.CRC_SIZE; import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.REC_TYPE_SIZE; @@ -93,7 +94,13 @@ public class RecordV2Serializer implements RecordSerializer { /** {@inheritDoc} */ @Override public int sizeWithHeaders(WALRecord record) throws IgniteCheckedException { - return dataSerializer.size(record) + REC_TYPE_SIZE + FILE_WAL_POINTER_SIZE + CRC_SIZE; + int recordSize = dataSerializer.size(record); + + int recordSizeWithType = recordSize + REC_TYPE_SIZE; + + // Why this condition here, see SWITCH_SEGMENT_RECORD doc. + return record.type() != SWITCH_SEGMENT_RECORD ? + recordSizeWithType + FILE_WAL_POINTER_SIZE + CRC_SIZE : recordSizeWithType; } /** {@inheritDoc} */ @@ -103,7 +110,7 @@ public class RecordV2Serializer implements RecordSerializer { ) throws IOException, IgniteCheckedException { WALRecord.RecordType recType = RecordV1Serializer.readRecordType(in); - if (recType == WALRecord.RecordType.SWITCH_SEGMENT_RECORD) + if (recType == SWITCH_SEGMENT_RECORD) throw new SegmentEofException("Reached end of segment", null); FileWALPointer ptr = readPositionAndCheckPoint(in, expPtr, skipPositionCheck); @@ -162,6 +169,10 @@ else if (marshalledMode) { // Write record type. RecordV1Serializer.putRecordType(buf, record); + // SWITCH_SEGMENT_RECORD should have only type, no need to write pointer. + if (record.type() == SWITCH_SEGMENT_RECORD) + return; + // Write record file position. putPositionOfRecord(buf, record); @@ -172,13 +183,19 @@ else if (marshalledMode) { /** * Create an instance of Record V2 serializer. + * * @param dataSerializer V2 data serializer. * @param marshalledMode Marshalled mode. * @param skipPositionCheck Skip position check mode. - * @param recordFilter Record type filter. {@link FilteredRecord} is deserialized instead of original record + * @param recordFilter Record type filter. {@link FilteredRecord} is deserialized instead of original record. */ - public RecordV2Serializer(RecordDataV2Serializer dataSerializer, boolean writePointer, - boolean marshalledMode, boolean skipPositionCheck, IgniteBiPredicate recordFilter) { + public RecordV2Serializer( + RecordDataV2Serializer dataSerializer, + boolean writePointer, + boolean marshalledMode, + boolean skipPositionCheck, + IgniteBiPredicate recordFilter + ) { this.dataSerializer = dataSerializer; this.writePointer = writePointer; this.marshalledMode = marshalledMode; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java new file mode 100644 index 0000000000000..b30466e5acf49 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java @@ -0,0 +1,386 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.nio.file.Paths; +import java.util.Random; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.WALMode; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager; +import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; +import org.apache.ignite.internal.pagemem.wal.WALIterator; +import org.apache.ignite.internal.pagemem.wal.WALPointer; +import org.apache.ignite.internal.pagemem.wal.record.MetastoreDataRecord; +import org.apache.ignite.internal.pagemem.wal.record.SwitchSegmentRecord; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.processors.cache.GridCacheIoManager; +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.IgniteCacheDatabaseSharedManager; +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.FileWriteAheadLogManager; +import org.apache.ignite.internal.processors.cache.persistence.wal.FsyncModeFileWriteAheadLogManager; +import org.apache.ignite.internal.processors.cache.persistence.wal.reader.StandaloneGridKernalContext; +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.cacheobject.IgniteCacheObjectProcessor; +import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessorImpl; +import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.spi.eventstorage.NoopEventStorageSpi; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.Assert; + +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.METASTORE_DATA_RECORD; +import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.HEADER_RECORD_SIZE; + +/*** + * Test check correct switch segment if in the tail of segment have garbage. + */ +public class IgniteWalIteratorSwitchSegmentTest extends GridCommonAbstractTest { + /** Segment file size. */ + private static final int SEGMENT_SIZE = 1024 * 1024; + + /** WAL segment file sub directory. */ + private static final String WORK_SUB_DIR = "/NODE/wal"; + + /** WAL archive segment file sub directory. */ + private static final String ARCHIVE_SUB_DIR = "/NODE/walArchive"; + + /** Serializer versions for check. */ + private int[] checkSerializerVers = new int[] { + 1, + 2 + }; + + /** FileWriteAheadLogManagers for check. */ + private Class[] checkWalManagers = new Class[] { + FileWriteAheadLogManager.class, + FsyncModeFileWriteAheadLogManager.class + }; + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + U.delete(Paths.get(U.defaultWorkDirectory())); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + U.delete(Paths.get(U.defaultWorkDirectory())); + } + + /** + * Test for check invariant, size of SWITCH_SEGMENT_RECORD should be 1 byte. + * + * @throws Exception If some thing failed. + */ + public void testCheckSerializer() throws Exception { + for (int serVer : checkSerializerVers) { + checkInvariantSwitchSegmentSize(serVer); + } + } + + /** + * @param serVer WAL serializer version. + * @throws Exception If some thing failed. + */ + private void checkInvariantSwitchSegmentSize(int serVer) throws Exception { + GridKernalContext kctx = new StandaloneGridKernalContext( + log, null, null) { + @Override public IgniteCacheObjectProcessor cacheObjects() { + return new IgniteCacheObjectProcessorImpl(this); + } + }; + + RecordSerializer serializer = new RecordSerializerFactoryImpl( + new GridCacheSharedContext<>( + kctx, + null, + null, + null, + null, + null, + null, + new IgniteCacheDatabaseSharedManager() { + @Override public int pageSize() { + return DataStorageConfiguration.DFLT_PAGE_SIZE; + } + }, + null, + null, + null, + null, + null, + null, + null, + + null) + ).createSerializer(serVer); + + SwitchSegmentRecord switchSegmentRecord = new SwitchSegmentRecord(); + + int recordSize = serializer.size(switchSegmentRecord); + + Assert.assertEquals(1, recordSize); + } + + /** + * Test for check invariant, size of SWITCH_SEGMENT_RECORD should be 1 byte. + * + * @throws Exception If some thing failed. + */ + public void test() throws Exception { + for (int serVer : checkSerializerVers) { + for (Class walMgrClass : checkWalManagers) { + try { + log.info("checking wal manager " + walMgrClass + " with serializer version " + serVer); + + checkInvariantSwitchSegment(walMgrClass, serVer); + } + finally { + U.delete(Paths.get(U.defaultWorkDirectory())); + } + } + } + } + + /** + * @param walMgrClass WAL manager class. + * @param serVer WAL serializer version. + * @throws Exception If some thing failed. + */ + private void checkInvariantSwitchSegment(Class walMgrClass, int serVer) throws Exception { + String workDir = U.defaultWorkDirectory(); + + T2 initTup = initiate(walMgrClass, serVer, workDir); + + IgniteWriteAheadLogManager walMgr = initTup.get1(); + + RecordSerializer recordSerializer = initTup.get2(); + + int switchSegmentRecordSize = recordSerializer.size(new SwitchSegmentRecord()); + + log.info("switchSegmentRecordSize:" + switchSegmentRecordSize); + + int tailSize = 0; + + /* Initial record payload size. */ + int payloadSize = 1024; + + int recSize = 0; + + MetastoreDataRecord rec = null; + + /* Record size. */ + int recordTypeSize = 1; + + /* Record pointer. */ + int recordPointerSize = 8 + 4 + 4; + + int lowBound = recordTypeSize + recordPointerSize; + int highBound = lowBound + /*CRC*/4; + + int attempt = 1000; + + // Try find how many record need for specific tail size. + while (true) { + if (attempt < 0) + throw new IgniteCheckedException("Can not find any payload size for test, " + + "lowBound=" + lowBound + ", highBound=" + highBound); + + if (tailSize >= lowBound && tailSize < highBound) + break; + + payloadSize++; + + byte[] payload = new byte[payloadSize]; + + // Fake record for payload. + rec = new MetastoreDataRecord("0", payload); + + recSize = recordSerializer.size(rec); + + tailSize = (SEGMENT_SIZE - HEADER_RECORD_SIZE) % recSize; + + attempt--; + } + + Assert.assertNotNull(rec); + + int recordsToWrite = SEGMENT_SIZE / recSize; + + log.info("records to write " + recordsToWrite + " tail size " + + (SEGMENT_SIZE - HEADER_RECORD_SIZE) % recSize); + + // Add more record for rollover to the next segment. + recordsToWrite += 100; + + for (int i = 0; i < recordsToWrite; i++) { + walMgr.log(new MetastoreDataRecord(rec.key(), rec.value())); + } + + walMgr.flush(null, true); + + // Await archiver move segment to WAL archive. + Thread.sleep(5000); + + // If switchSegmentRecordSize more that 1, it mean that invariant is broke. + // Filling tail some garbage. Simulate tail garbage on rotate segment in WAL work directory. + if (switchSegmentRecordSize > 1) { + File seg = new File(workDir + ARCHIVE_SUB_DIR + "/0000000000000000.wal"); + + FileIOFactory ioFactory = new RandomAccessFileIOFactory(); + + FileIO seg0 = ioFactory.create(seg); + + byte[] bytes = new byte[tailSize]; + + Random rnd = new Random(); + + rnd.nextBytes(bytes); + + // Some record type. + bytes[0] = (byte)(METASTORE_DATA_RECORD.ordinal() + 1); + + seg0.position((int)(seg0.size() - tailSize)); + + seg0.write(bytes, 0, tailSize); + + seg0.force(true); + + seg0.close(); + } + + int expectedRecords = recordsToWrite; + int actualRecords = 0; + + // Check that switch segment works as expected and all record is reachable. + try (WALIterator it = walMgr.replay(null)) { + while (it.hasNext()) { + IgniteBiTuple tup = it.next(); + + WALRecord rec0 = tup.get2(); + + if (rec0.type() == METASTORE_DATA_RECORD) + actualRecords++; + } + } + + Assert.assertEquals("Not all records read during iteration.", expectedRecords, actualRecords); + } + + /*** + * Initiate WAL manager. + * + * @param walMgrClass WAL manager class. + * @param serVer WAL serializer version. + * @param workDir Work directory path. + * @return Tuple of WAL manager and WAL record serializer. + * @throws IgniteCheckedException If some think failed. + */ + private T2 initiate( + Class walMgrClass, + int serVer, + String workDir + ) throws IgniteCheckedException { + + GridKernalContext kctx = new StandaloneGridKernalContext( + log, null, null + ) { + @Override protected IgniteConfiguration prepareIgniteConfiguration() { + IgniteConfiguration cfg = super.prepareIgniteConfiguration(); + + cfg.setDataStorageConfiguration( + new DataStorageConfiguration() + .setWalSegmentSize(SEGMENT_SIZE) + .setWalMode(WALMode.FSYNC) + .setWalPath(workDir + WORK_SUB_DIR) + .setWalArchivePath(workDir + ARCHIVE_SUB_DIR) + ); + + cfg.setEventStorageSpi(new NoopEventStorageSpi()); + + return cfg; + } + + @Override public GridInternalSubscriptionProcessor internalSubscriptionProcessor() { + return new GridInternalSubscriptionProcessor(this); + } + + @Override public GridEventStorageManager event() { + return new GridEventStorageManager(this); + } + }; + + IgniteWriteAheadLogManager walMgr = null; + + if (walMgrClass.equals(FileWriteAheadLogManager.class)) { + walMgr = new FileWriteAheadLogManager(kctx); + + GridTestUtils.setFieldValue(walMgr, "serializerVer", serVer); + } + else if (walMgrClass.equals(FsyncModeFileWriteAheadLogManager.class)) { + walMgr = new FsyncModeFileWriteAheadLogManager(kctx); + + GridTestUtils.setFieldValue(walMgr, "serializerVersion", serVer); + } + + GridCacheSharedContext ctx = new GridCacheSharedContext<>( + kctx, + null, + null, + null, + null, + walMgr, + null, + new GridCacheDatabaseSharedManager(kctx), + null, + null, + null, + null, + new GridCacheIoManager(), + null, + null, + null + ); + + walMgr.start(ctx); + + walMgr.onActivate(kctx); + + walMgr.resumeLogging(null); + + RecordSerializer recordSerializer = new RecordSerializerFactoryImpl(ctx) + .createSerializer(walMgr.serializerVersion()); + + return new T2<>(walMgr, recordSerializer); + } +} 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 316ae1b8a7ad6..35c3d3ef70770 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 @@ -46,6 +46,7 @@ import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushFailoverTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushLogOnlySelfTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalHistoryReservationsTest; +import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalIteratorSwitchSegmentTest; 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.crc.IgniteDataIntegrityTests; From 6c47e23a626247ada5a692cb3acfaf746cddff5e Mon Sep 17 00:00:00 2001 From: Dmitriy Govorukhin Date: Wed, 6 Jun 2018 17:36:40 +0300 Subject: [PATCH 0234/1463] IGNITE-8685 Fixed switch segment record size - Fixes #4130. add test to suit Signed-off-by: Alexey Goncharuk (cherry picked from commit 2a048bd) --- .../java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java | 2 ++ 1 file changed, 2 insertions(+) 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 35c3d3ef70770..4287a292ddcf5 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 @@ -151,5 +151,7 @@ public static void addRealPageStoreTests(TestSuite suite) { suite.addTestSuite(IgnitePdsCorruptedStoreTest.class); suite.addTestSuite(LocalWalModeChangeDuringRebalancingSelfTest.class); + + suite.addTestSuite(IgniteWalIteratorSwitchSegmentTest.class); } } From 131d3629f9fba2df3eb05d98f0e801f728f13342 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Wed, 18 Apr 2018 17:16:17 +0300 Subject: [PATCH 0235/1463] IGNITE-6681 .NET: QueryMetrics This closes #3839 --- .../platform/cache/PlatformCache.java | 37 ++++ .../ExpiryCacheHolderTest.cs | 10 + .../Apache.Ignite.Core.Tests.csproj | 1 + .../ApiParity/CacheParityTest.cs | 2 - .../Cache/CacheQueryMetricsTest.cs | 190 ++++++++++++++++++ .../Cache/CacheTestAsyncWrapper.cs | 10 + .../Apache.Ignite.Core.csproj | 2 + .../dotnet/Apache.Ignite.Core/Cache/ICache.cs | 11 + .../Apache.Ignite.Core/Cache/IQueryMetrics.cs | 65 ++++++ .../Impl/Cache/CacheImpl.cs | 17 ++ .../Apache.Ignite.Core/Impl/Cache/CacheOp.cs | 4 +- .../Impl/Cache/QueryMetricsImpl.cs | 71 +++++++ 12 files changed, 417 insertions(+), 3 deletions(-) create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheQueryMetricsTest.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Cache/IQueryMetrics.cs create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/QueryMetricsImpl.cs diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java index 19dac83758d23..c78f2d62d488a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java @@ -39,6 +39,7 @@ import org.apache.ignite.cache.CachePartialUpdateException; import org.apache.ignite.cache.CachePeekMode; import org.apache.ignite.cache.query.Query; +import org.apache.ignite.cache.query.QueryMetrics; import org.apache.ignite.cache.query.ScanQuery; import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.SqlQuery; @@ -326,6 +327,12 @@ public class PlatformCache extends PlatformAbstractTarget { /** */ public static final int OP_GET_LOST_PARTITIONS = 84; + /** */ + public static final int OP_QUERY_METRICS = 85; + + /** */ + public static final int OP_RESET_QUERY_METRICS = 86; + /** Underlying JCache in binary mode. */ private final IgniteCacheProxy cache; @@ -988,6 +995,14 @@ private IgniteFuture loadCacheAsync0(BinaryRawReaderEx reader, boolean loc break; + case OP_QUERY_METRICS: { + QueryMetrics metrics = cache.queryMetrics(); + + writeQueryMetrics(writer, metrics); + + break; + } + default: super.processOutStream(type, writer); } @@ -1093,6 +1108,11 @@ private IgniteFuture loadCacheAsync0(BinaryRawReaderEx reader, boolean loc case OP_REMOVE_ALL2: cache.removeAll(); + return TRUE; + + case OP_RESET_QUERY_METRICS: + cache.resetQueryMetrics(); + return TRUE; } return super.processInLongOutLong(type, val); @@ -1485,6 +1505,23 @@ public static void writeCacheMetrics(BinaryRawWriter writer, CacheMetrics metric writer.writeLong(metrics.getRebalanceClearingPartitionsLeft()); } + /** + * Writes query metrics. + * + * @param writer Writer. + * @param metrics Metrics. + */ + public static void writeQueryMetrics(BinaryRawWriter writer, QueryMetrics metrics) { + assert writer != null; + assert metrics != null; + + writer.writeLong(metrics.minimumTime()); + writer.writeLong(metrics.maximumTime()); + writer.writeDouble(metrics.averageTime()); + writer.writeInt(metrics.executions()); + writer.writeInt(metrics.fails()); + } + /** * Writes error with EntryProcessorException cause. */ diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs index 93e3a53846b05..34d4b4ddd070e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/ExpiryCacheHolderTest.cs @@ -513,6 +513,16 @@ public ICollection GetLostPartitions() { throw new NotImplementedException(); } + + public IQueryMetrics GetQueryMetrics() + { + throw new NotImplementedException(); + } + + public void ResetQueryMetrics() + { + throw new NotImplementedException(); + } } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj index 8cd64cc9d6c61..a04509a13fd14 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj @@ -106,6 +106,7 @@ + diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/CacheParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/CacheParityTest.cs index 68b822c38f73a..6471ec5865a03 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/CacheParityTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/CacheParityTest.cs @@ -51,8 +51,6 @@ public class CacheParityTest /** Members that are missing on .NET side and should be added in future. */ private static readonly string[] MissingMembers = { - "queryMetrics", // IGNITE-6681 - "resetQueryMetrics", // IGNITE-6681 "queryDetailMetrics", // IGNITE-6680 "resetQueryDetailMetrics", // IGNITE-6680 "sizeLong", // IGNITE-6563 diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheQueryMetricsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheQueryMetricsTest.cs new file mode 100644 index 0000000000000..5b85328b181a0 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheQueryMetricsTest.cs @@ -0,0 +1,190 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// ReSharper disable UnusedAutoPropertyAccessor.Local +namespace Apache.Ignite.Core.Tests.Cache +{ + extern alias ExamplesDll; + using System.Collections.Generic; + using System.Diagnostics.CodeAnalysis; + using System.Linq; + using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Cache.Configuration; + using Apache.Ignite.Core.Cache.Query; + using Apache.Ignite.Core.Common; + using Apache.Ignite.Linq; + using NUnit.Framework; + + /// + /// Tests query metrics propagation. + /// + public class CacheQueryMetricsTest + { + /// + /// Fixture set up. + /// + [TestFixtureSetUp] + public void FixtureSetUp() + { + Ignition.Start(TestUtils.GetTestConfiguration()); + } + + /// + /// Fixture tear down. + /// + [TestFixtureTearDown] + public void FixtureTearDown() + { + Ignition.StopAll(true); + } + + /// + /// Tests query metrics. + /// + [Test] + public void TestQueryMetrics() + { + var cache = GetCache(); + + cache.ResetQueryMetrics(); + + ExecuteTwoQueries(cache); + + CheckMetrics(cache.GetQueryMetrics()); + } + + /// + /// Tests failed queries count. + /// + [Test] + public void TestQueryFails() + { + var cache = GetCache(); + + Assert.Throws(() => cache.Query(new SqlFieldsQuery("select * from NOT_A_TABLE"))); + + Assert.AreEqual(1, cache.GetQueryMetrics().Fails, "Check Fails count."); + } + + /// + /// Tests query metrics reset. + /// + [Test] + public void TestQueryMetricsReset() + { + var cache = GetCache(); + + ExecuteTwoQueries(cache); + + cache.ResetQueryMetrics(); + + IQueryMetrics metrics = cache.GetQueryMetrics(); + + Assert.AreEqual(0, metrics.Executions, "Check Executions count."); + Assert.AreEqual(0, metrics.Fails, "Check Fails count."); + Assert.AreEqual(0, metrics.MinimumTime, "Check MinimumTime."); + Assert.AreEqual(0, metrics.MaximumTime, "Check MaximumTime."); + Assert.AreEqual(0, metrics.AverageTime, 0.1, "Check AverageTime."); + + ExecuteTwoQueries(cache); + + CheckMetrics(cache.GetQueryMetrics()); + } + + /// + /// Get Cache instance. + /// + /// Cache instance. + private ICache GetCache() + { + var ignite = Ignition.GetIgnite(); + + var cache = ignite.GetOrCreateCache(GetCacheConfiguration()); + + if (cache.GetSize() == 0) + { + Person person = new Person() + { + Name = "Adam", + Age = 35000 + }; + + cache.Put(1, person); + } + + return cache; + } + + /// + /// Check cache configuration. + /// + /// Cache configuration. + private CacheConfiguration GetCacheConfiguration() + { + return new CacheConfiguration("cacheName") + { + EnableStatistics = true, + + QueryEntities = new List() + { + new QueryEntity(typeof(int), typeof(Person)) + } + }; + } + + /// + /// Person. + /// + private class Person + { + [QuerySqlField] + public string Name { get; set; } + + [QuerySqlField] + public int Age { get; set; } + } + + /// + /// Execute two queries. + /// + /// Cache instance. + [SuppressMessage("ReSharper", "ReturnValueOfPureMethodIsNotUsed")] + private static void ExecuteTwoQueries(ICache cache) + { + IQueryable> queryable = cache.AsCacheQueryable(); + + queryable.Count(p => p.Value.Age > 0); + + queryable.Count(p => p.Value.Age > 1000); + } + + /// + /// Check metrics after two queries are executed. + /// + /// Query metrics. + private static void CheckMetrics(IQueryMetrics metrics) + { + Assert.AreEqual(2, metrics.Executions, "Check Executions count."); + Assert.AreEqual(0, metrics.Fails, "Check Fails count."); + + Assert.GreaterOrEqual(metrics.MinimumTime, 0, "Check MinimumTime."); + Assert.GreaterOrEqual(metrics.MaximumTime, metrics.MinimumTime, "Check MaximumTime."); + + Assert.AreEqual((double)(metrics.MinimumTime + metrics.MaximumTime) / 2, metrics.AverageTime, 1, "Check AverageTime."); + } + } +} diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheTestAsyncWrapper.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheTestAsyncWrapper.cs index c4e50d148080a..0fe16a598d1f6 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheTestAsyncWrapper.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheTestAsyncWrapper.cs @@ -573,6 +573,16 @@ IEnumerator IEnumerable.GetEnumerator() { return GetEnumerator(); } + + public IQueryMetrics GetQueryMetrics() + { + return _cache.GetQueryMetrics(); + } + + public void ResetQueryMetrics() + { + _cache.ResetQueryMetrics(); + } } /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj index 93c45c34d4caa..f217a0d34abd7 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj @@ -54,6 +54,7 @@ + @@ -72,6 +73,7 @@ + diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICache.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICache.cs index 8a9fe59718296..8f52e03babed7 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICache.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICache.cs @@ -907,5 +907,16 @@ Task>> InvokeAllAsync. /// ICollection GetLostPartitions(); + + /// + /// Gets query metrics. + /// + /// Query metrics. + IQueryMetrics GetQueryMetrics(); + + /// + /// Reset query metrics. + /// + void ResetQueryMetrics(); } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/IQueryMetrics.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/IQueryMetrics.cs new file mode 100644 index 0000000000000..232bf99c1d3c8 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/IQueryMetrics.cs @@ -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. + */ + +namespace Apache.Ignite.Core.Cache +{ + /// + /// Cache query metrics used to obtain statistics on query. + /// + public interface IQueryMetrics + { + /// + /// Gets minimum execution time of query. + /// + /// + /// Minimum execution time of query. + /// + long MinimumTime { get; } + + /// + /// Gets maximum execution time of query. + /// + /// + /// Maximum execution time of query. + /// + long MaximumTime { get; } + + /// + /// Gets average execution time of query. + /// + /// + /// Average execution time of query. + /// + double AverageTime { get; } + + /// + /// Gets total number execution of query. + /// + /// + /// Number of executions. + /// + int Executions { get; } + + /// + /// Gets total number of times a query execution failed. + /// + /// + /// Total number of times a query execution failed. + /// + int Fails { get; } + } +} \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs index 666116189cfa7..6e46f4db7e19b 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs @@ -1408,5 +1408,22 @@ private void StartTx() if (_txManager != null) _txManager.StartTx(); } + + /** */ + public IQueryMetrics GetQueryMetrics() + { + return DoInOp((int)CacheOp.QueryMetrics, stream => + { + IBinaryRawReader reader = Marshaller.StartUnmarshal(stream, false); + + return new QueryMetricsImpl(reader); + }); + } + + /** */ + public void ResetQueryMetrics() + { + DoOutInOp((int)CacheOp.ResetQueryMetrics); + } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs index b3cbd9516761a..00a71350b0986 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs @@ -104,6 +104,8 @@ internal enum CacheOp PutIfAbsentAsync = 81, Extension = 82, GlobalMetrics = 83, - GetLostPartitions = 84 + GetLostPartitions = 84, + QueryMetrics = 85, + ResetQueryMetrics = 86 } } \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/QueryMetricsImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/QueryMetricsImpl.cs new file mode 100644 index 0000000000000..d71d40e00c122 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/QueryMetricsImpl.cs @@ -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. + */ + +namespace Apache.Ignite.Core.Impl.Cache +{ + using Apache.Ignite.Core.Binary; + using Apache.Ignite.Core.Cache; + + /// + /// Cache query metrics used to obtain statistics on query. + /// + internal class QueryMetricsImpl : IQueryMetrics + { + /** */ + private readonly long _minimumTime; + + /** */ + private readonly long _maximumTime; + + /** */ + private readonly double _averageTime; + + /** */ + private readonly int _executions; + + /** */ + private readonly int _fails; + + /// + /// Initializes a new instance of the class. + /// + /// The reader. + public QueryMetricsImpl(IBinaryRawReader reader) + { + _minimumTime = reader.ReadLong(); + _maximumTime = reader.ReadLong(); + _averageTime = reader.ReadDouble(); + _executions = reader.ReadInt(); + _fails = reader.ReadInt(); + } + + /** */ + public long MinimumTime { get { return _minimumTime; } } + + /** */ + public long MaximumTime { get { return _maximumTime; } } + + /** */ + public double AverageTime { get { return _averageTime; } } + + /** */ + public int Executions { get { return _executions; } } + + /** */ + public int Fails { get { return _fails; } } + } +} \ No newline at end of file From 90df4f216046229943d4aaf4ae01a2af560393f3 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Thu, 7 Jun 2018 14:55:13 +0300 Subject: [PATCH 0236/1463] IGNITE-8706 Fixed testMemoryUsageMultipleNodes --- .../processors/cache/persistence/file/FilePageStore.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java index ae4880d22715c..852eb0da870b1 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java @@ -279,9 +279,9 @@ public void truncate(int tag) throws PersistentStorageIOException { throw new PersistentStorageIOException("Failed to delete partition file: " + cfgFile.getPath(), e); } finally { - allocated.set(0); + allocatedTracker.updateTotalAllocatedPages(-1L * pages()); - allocatedTracker.updateTotalAllocatedPages(-1L * this.pages()); + allocated.set(0); inited = false; From 82c4bf490951c81e27c48c98e2ddf6dbee92caad Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Mon, 14 May 2018 13:14:52 +0300 Subject: [PATCH 0237/1463] IGNITE-8422 Tests fix for Zookeeper discovery split brain detection shouldn't consider client nodes - Fixes #3975. Signed-off-by: dpavlov (cherry-picked from commit #19cbf8058b48a1f8fec60f7ceac70229e7c5391a) --- .../internal/ZookeeperDiscoverySpiTest.java | 33 +++++++++++++++++-- 1 file changed, 30 insertions(+), 3 deletions(-) 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 03b874dce1bf9..09db1049211cc 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 @@ -127,6 +127,7 @@ import org.apache.zookeeper.ZkTestClientCnxnSocketNIO; import org.apache.zookeeper.ZooKeeper; import org.jetbrains.annotations.Nullable; +import org.junit.Assert; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.SECONDS; @@ -3589,6 +3590,10 @@ public void testSimpleSplitBrain() throws Exception { startGridsMultiThreaded(5, 3); + client = false; + + awaitPartitionMapExchange(); + List all = G.allGrids().stream() .map(g -> g.cluster().localNode()) .collect(Collectors.toList());; @@ -3600,6 +3605,8 @@ public void testSimpleSplitBrain() throws Exception { ClusterNode lastClient = startGrid(8).cluster().localNode(); + awaitPartitionMapExchange(); + // Make last client connected to other nodes. for (ClusterNode node : all) { if (node.id().equals(lastClient.id())) @@ -3633,15 +3640,21 @@ public void testNotActualSplitBrain() throws Exception { .map(g -> g.cluster().localNode()) .collect(Collectors.toList()); + Assert.assertEquals(5, srvNodes.size()); + client = true; startGridsMultiThreaded(5, 3); client = false; + awaitPartitionMapExchange(); + ConnectionsFailureMatrix matrix = new ConnectionsFailureMatrix(); - matrix.addAll(G.allGrids().stream().map(g -> g.cluster().localNode()).collect(Collectors.toList())); + List allNodes = G.allGrids().stream().map(g -> g.cluster().localNode()).collect(Collectors.toList()); + + matrix.addAll(allNodes); // Remove 2 connections between server nodes. matrix.removeConnection(srvNodes.get(0), srvNodes.get(1)); @@ -3674,6 +3687,8 @@ public void testAlmostSplitBrain() throws Exception { .map(g -> g.cluster().localNode()) .collect(Collectors.toList()); + Assert.assertEquals(6, srvNodes.size()); + List srvPart1 = srvNodes.subList(0, 3); List srvPart2 = srvNodes.subList(3, srvNodes.size()); @@ -3683,11 +3698,15 @@ public void testAlmostSplitBrain() throws Exception { client = false; + awaitPartitionMapExchange(); + List clientNodes = G.allGrids().stream() .map(g -> g.cluster().localNode()) .filter(ClusterNode::isClient) .collect(Collectors.toList()); + Assert.assertEquals(5, clientNodes.size()); + List clientPart1 = clientNodes.subList(0, 2); List clientPart2 = clientNodes.subList(2, 4); @@ -3793,17 +3812,24 @@ static class PeerToPeerCommunicationFailureSpi extends TcpCommunicationSpi { private static volatile boolean failure; /** Connections failure matrix. */ - private static ConnectionsFailureMatrix matrix; + private static volatile ConnectionsFailureMatrix matrix; /** * Start failing connections according to given matrix {@code with}. * @param with Failure matrix. */ - public static void fail(ConnectionsFailureMatrix with) { + static void fail(ConnectionsFailureMatrix with) { matrix = with; failure = true; } + /** + * Resets failure matrix. + */ + static void unfail() { + failure = false; + } + /** {@inheritDoc} */ @Override public IgniteFuture checkConnection(List nodes) { // Creates connections statuses according to failure matrix. @@ -4264,6 +4290,7 @@ private void reset() { err = false; failCommSpi = false; + PeerToPeerCommunicationFailureSpi.unfail(); evts.clear(); From 90e8c943eda578dc4a850a4392d614bfd5204091 Mon Sep 17 00:00:00 2001 From: Sergey Chugunov Date: Mon, 14 May 2018 20:09:33 +0300 Subject: [PATCH 0238/1463] IGNITE-8487 tmpfs is employed by ZooKeeper tests - Fixes #3994. Signed-off-by: dpavlov --- .../zk/ZookeeperDiscoverySpiAbstractTestSuite.java | 7 ++++++- .../spi/discovery/zk/internal/ZookeeperClientTest.java | 3 ++- .../discovery/zk/internal/ZookeeperDiscoverySpiTest.java | 5 +++-- 3 files changed, 11 insertions(+), 4 deletions(-) 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 766635cb5580c..c5d34884a8c08 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 @@ -76,7 +76,12 @@ public synchronized static void preprocessConfiguration(IgniteConfiguration cfg) * @return Test cluster. */ public static TestingCluster createTestingCluster(int instances) { - String tmpDir = System.getProperty("java.io.tmpdir"); + String tmpDir; + + if (System.getenv("TMPFS_ROOT") != null) + tmpDir = System.getenv("TMPFS_ROOT"); + else + tmpDir = System.getProperty("java.io.tmpdir"); List specs = new ArrayList<>(); 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 e7cb97a39434f..d228e03070ca7 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 @@ -31,6 +31,7 @@ import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteRunnable; +import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpiAbstractTestSuite; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.zookeeper.AsyncCallback; @@ -462,7 +463,7 @@ public void testReconnect4() throws Exception { private void startZK(int instances) throws Exception { assert zkCluster == null; - zkCluster = new TestingCluster(instances); + zkCluster = ZookeeperDiscoverySpiAbstractTestSuite.createTestingCluster(instances); zkCluster.start(); } 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 09db1049211cc..1585870053b87 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 @@ -118,6 +118,7 @@ import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator; import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpi; +import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpiAbstractTestSuite; import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpiMBean; import org.apache.ignite.spi.discovery.zk.ZookeeperDiscoverySpiTestSuite2; import org.apache.ignite.testframework.GridTestUtils; @@ -421,7 +422,7 @@ private void clearAckEveryEventSystemProperty() { super.beforeTest(); if (USE_TEST_CLUSTER && zkCluster == null) { - zkCluster = ZookeeperDiscoverySpiTestSuite2.createTestingCluster(ZK_SRVS); + zkCluster = ZookeeperDiscoverySpiAbstractTestSuite.createTestingCluster(ZK_SRVS); zkCluster.start(); } @@ -4315,7 +4316,7 @@ private IgniteInternalFuture startRestartZkServers(final long stopTime, final ThreadLocalRandom rnd = ThreadLocalRandom.current(); while (!stop.get() && System.currentTimeMillis() < stopTime) { - U.sleep(rnd.nextLong(500) + 500); + U.sleep(rnd.nextLong(2500) + 2500); int idx = rnd.nextInt(ZK_SRVS); From 568e3c3d02580cdb0b56a84345eb56f67300e103 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Thu, 7 Jun 2018 18:29:00 +0300 Subject: [PATCH 0239/1463] IGNITE-8702: Fixed ODBC crash in Linux when there is more than 1023 fds open. (cherry picked from commit fa34308) --- .../core-test/project/vs/core-test.vcxproj | 31 +++++-------- .../project/vs/core-test.vcxproj.filters | 1 - .../cpp/odbc-test/src/api_robustness_test.cpp | 20 +++++++++ .../cpp/odbc-test/src/queries_test.cpp | 21 +++++++++ .../os/linux/src/system/tcp_socket_client.cpp | 45 ++++++++----------- .../os/win/src/system/tcp_socket_client.cpp | 4 +- modules/platforms/cpp/odbc/src/connection.cpp | 6 ++- 7 files changed, 76 insertions(+), 52 deletions(-) diff --git a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj index b614e59a81422..0b0166f30511f 100644 --- a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj +++ b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj @@ -36,36 +36,25 @@ - - + - - - Designer - - - Designer - - - Designer - + + + + - + - + - - - Designer - + + - - Designer - + diff --git a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters index a445af840e954..b16877c875d49 100644 --- a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters +++ b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters @@ -187,7 +187,6 @@ Configs - Configs diff --git a/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp b/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp index eca0e00f92784..5f807c9cd909b 100644 --- a/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp @@ -22,6 +22,8 @@ #include #include +#include + #include #include @@ -1101,4 +1103,22 @@ BOOST_AUTO_TEST_CASE(TestSQLDiagnosticRecords) CheckSQLStatementDiagnosticError("HY092"); } +BOOST_AUTO_TEST_CASE(TestManyFds) +{ + enum { FDS_NUM = 2000 }; + + std::FILE* fds[FDS_NUM]; + + for (int i = 0; i < FDS_NUM; ++i) + fds[i] = tmpfile(); + + Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;schema=cache"); + + for (int i = 0; i < FDS_NUM; ++i) + { + if (fds[i]) + fclose(fds[i]); + } +} + BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index 0013d5cc1b223..142e277c60d14 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -1889,6 +1889,27 @@ BOOST_AUTO_TEST_CASE(TestLoginTimeout) BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_DBC, dbc)); } +BOOST_AUTO_TEST_CASE(TestLoginTimeoutFail) +{ + Prepare(); + + SQLRETURN ret = SQLSetConnectAttr(dbc, SQL_ATTR_LOGIN_TIMEOUT, reinterpret_cast(5), 0); + + ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); + + SQLCHAR connectStr[] = "DRIVER={Apache Ignite};ADDRESS=192.168.0.1:11120;SCHEMA=cache"; + + SQLCHAR outstr[ODBC_BUFFER_SIZE]; + SQLSMALLINT outstrlen; + + // Connecting to ODBC server. + ret = SQLDriverConnect(dbc, NULL, &connectStr[0], static_cast(sizeof(connectStr)), + outstr, sizeof(outstr), &outstrlen, SQL_DRIVER_COMPLETE); + + if (SQL_SUCCEEDED(ret)) + BOOST_FAIL("Should timeout"); +} + BOOST_AUTO_TEST_CASE(TestConnectionTimeoutQuery) { Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;SCHEMA=cache"); diff --git a/modules/platforms/cpp/odbc/os/linux/src/system/tcp_socket_client.cpp b/modules/platforms/cpp/odbc/os/linux/src/system/tcp_socket_client.cpp index c0e36b66a1918..0b849f7e4cfc6 100644 --- a/modules/platforms/cpp/odbc/os/linux/src/system/tcp_socket_client.cpp +++ b/modules/platforms/cpp/odbc/os/linux/src/system/tcp_socket_client.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include @@ -102,9 +103,11 @@ namespace ignite std::stringstream converter; converter << port; + std::string strPort = converter.str(); + // Resolve the server address and port addrinfo *result = NULL; - int res = getaddrinfo(hostname, converter.str().c_str(), &hints, &result); + int res = getaddrinfo(hostname, strPort.c_str(), &hints, &result); if (res != 0) { @@ -154,7 +157,7 @@ namespace ignite continue; } - res = WaitOnSocket(timeout, false); + res = WaitOnSocket(timeout == 0 ? -1 : timeout, false); if (res < 0 || res == WaitResult::TIMEOUT) { @@ -187,7 +190,7 @@ namespace ignite { if (!blocking) { - int res = WaitOnSocket(timeout, false); + int res = WaitOnSocket(timeout == 0 ? -1 : timeout, false); if (res < 0 || res == WaitResult::TIMEOUT) return res; @@ -200,7 +203,7 @@ namespace ignite { if (!blocking) { - int res = WaitOnSocket(timeout, true); + int res = WaitOnSocket(timeout == 0 ? -1 : timeout, true); if (res < 0 || res == WaitResult::TIMEOUT) return res; @@ -318,35 +321,23 @@ namespace ignite int TcpSocketClient::WaitOnSocket(int32_t timeout, bool rd) { - int ready = 0; int lastError = 0; + int ret; + do + { + struct pollfd fds[1]; - fd_set fds; - - do { - struct timeval tv = { 0 }; - tv.tv_sec = timeout; - - FD_ZERO(&fds); - FD_SET(socketHandle, &fds); - - fd_set* readFds = 0; - fd_set* writeFds = 0; - - if (rd) - readFds = &fds; - else - writeFds = &fds; + fds[0].fd = socketHandle; + fds[0].events = rd ? POLLIN : POLLOUT; - ready = select(static_cast((socketHandle) + 1), - readFds, writeFds, NULL, (timeout == 0 ? NULL : &tv)); + ret = poll(fds, 1, timeout * 1000); - if (ready == SOCKET_ERROR) + if (ret == SOCKET_ERROR) lastError = GetLastSocketError(); - } while (ready == SOCKET_ERROR && IsSocketOperationInterrupted(lastError)); + } while (ret == SOCKET_ERROR && IsSocketOperationInterrupted(lastError)); - if (ready == SOCKET_ERROR) + if (ret == SOCKET_ERROR) return -lastError; socklen_t size = sizeof(lastError); @@ -355,7 +346,7 @@ namespace ignite if (res != SOCKET_ERROR && lastError != 0) return -lastError; - if (ready == 0) + if (ret == 0) return WaitResult::TIMEOUT; return WaitResult::SUCCESS; diff --git a/modules/platforms/cpp/odbc/os/win/src/system/tcp_socket_client.cpp b/modules/platforms/cpp/odbc/os/win/src/system/tcp_socket_client.cpp index 38e8b977392d8..278340346b4da 100644 --- a/modules/platforms/cpp/odbc/os/win/src/system/tcp_socket_client.cpp +++ b/modules/platforms/cpp/odbc/os/win/src/system/tcp_socket_client.cpp @@ -148,9 +148,11 @@ namespace ignite std::stringstream converter; converter << port; + std::string strPort = converter.str(); + // Resolve the server address and port addrinfo *result = NULL; - int res = getaddrinfo(hostname, converter.str().c_str(), &hints, &result); + int res = getaddrinfo(hostname, strPort.c_str(), &hints, &result); if (res != 0) { diff --git a/modules/platforms/cpp/odbc/src/connection.cpp b/modules/platforms/cpp/odbc/src/connection.cpp index 05165ebb92cc6..b96753c2669b0 100644 --- a/modules/platforms/cpp/odbc/src/connection.cpp +++ b/modules/platforms/cpp/odbc/src/connection.cpp @@ -112,10 +112,12 @@ namespace ignite parser.ParseConnectionString(connectStr, &GetDiagnosticRecords()); - std::string dsn = config.GetDsn(); + if (config.IsDsnSet()) + { + std::string dsn = config.GetDsn(); - if (!dsn.empty()) ReadDsnConfiguration(dsn.c_str(), config); + } return InternalEstablish(config); } From 39a1e5edead2c067a7344ce3a113ab6197574be9 Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Fri, 4 May 2018 19:29:52 +0300 Subject: [PATCH 0240/1463] IGNITE-8395 Fixed test framework to not count non-zero arguments methods as tests (cherry picked from commit 6830201) --- .../apache/ignite/testframework/junits/GridAbstractTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 00929262e69c0..cc54d2345ce24 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 @@ -2514,7 +2514,7 @@ public int getNumberOfTests() { cnt = 0; for (Method m : this0.getClass().getMethods()) - if (m.getName().startsWith("test") && Modifier.isPublic(m.getModifiers())) + if (m.getName().startsWith("test") && Modifier.isPublic(m.getModifiers()) && m.getParameterCount() == 0) cnt++; } From 7605066211d3dc078d63e4a5d0967686048f5fc2 Mon Sep 17 00:00:00 2001 From: Anton Kalashnikov Date: Fri, 8 Jun 2018 18:06:42 +0300 Subject: [PATCH 0241/1463] IGNITE-8739 Implemented WA for tcp communication SPI hanging on descriptor reservation - Fixes #4148. Signed-off-by: Alexey Goncharuk (cherry picked from commit 7021651) --- .../apache/ignite/IgniteSystemProperties.java | 6 + .../util/nio/GridNioRecoveryDescriptor.java | 52 ++++++- .../internal/util/nio/GridNioSessionImpl.java | 5 + .../util/nio/GridSelectorNioSessionImpl.java | 23 +++ .../tcp/TcpCommunicationSpi.java | 20 ++- ...nectionConcurrentReserveAndRemoveTest.java | 146 ++++++++++++++++++ .../testsuites/IgniteKernalSelfTestSuite.java | 2 + 7 files changed, 248 insertions(+), 6 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/IgniteConnectionConcurrentReserveAndRemoveTest.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 83d15f0bba30a..85e13ad309ce4 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -887,6 +887,12 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_DISABLE_WAL_DURING_REBALANCING = "IGNITE_DISABLE_WAL_DURING_REBALANCING"; + /** + * Sets timeout for TCP client recovery descriptor reservation. + */ + public static final String IGNITE_NIO_RECOVERY_DESCRIPTOR_RESERVATION_TIMEOUT = + "IGNITE_NIO_RECOVERY_DESCRIPTOR_RESERVATION_TIMEOUT"; + /** * When set to {@code true}, Ignite will skip partitions sizes check on partition validation after rebalance has finished. * Partitions sizes may differs on nodes when Expiry Policy is in use and it is ok due to lazy entry eviction mechanics. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java index af7b757762982..bd1291aced450 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java @@ -22,16 +22,24 @@ import java.util.Deque; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteInClosure; import org.jetbrains.annotations.Nullable; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_NIO_RECOVERY_DESCRIPTOR_RESERVATION_TIMEOUT; + /** * Recovery information for single node. */ public class GridNioRecoveryDescriptor { + /** Timeout for outgoing recovery descriptor reservation. */ + private static final long DESC_RESERVATION_TIMEOUT = + Math.max(1_000, IgniteSystemProperties.getLong(IGNITE_NIO_RECOVERY_DESCRIPTOR_RESERVATION_TIMEOUT, 5_000)); + /** Number of acknowledged messages. */ private long acked; @@ -80,6 +88,10 @@ public class GridNioRecoveryDescriptor { /** */ private final boolean pairedConnections; + /** Session for the descriptor. */ + @GridToStringExclude + private GridNioSession ses; + /** * @param pairedConnections {@code True} if in/out connections pair is used for communication with node. * @param queueLimit Maximum size of unacknowledged messages queue. @@ -271,8 +283,19 @@ public boolean nodeAlive(@Nullable ClusterNode node) { */ public boolean reserve() throws InterruptedException { synchronized (this) { - while (!connected && reserved) - wait(); + long t0 = System.nanoTime(); + + while (!connected && reserved) { + wait(DESC_RESERVATION_TIMEOUT); + + if ((System.nanoTime() - t0) / 1_000_000 >= DESC_RESERVATION_TIMEOUT - 100) { + // Dumping a descriptor. + log.error("Failed to wait for recovery descriptor reservation " + + "[desc=" + this + ", ses=" + ses + ']'); + + return false; + } + } if (!connected) { reserved = true; @@ -354,6 +377,8 @@ public void release() { SessionWriteRequest[] futs = null; synchronized (this) { + ses = null; + connected = false; if (handshakeReq != null) { @@ -439,17 +464,36 @@ public int reserveCount() { } } + /** + * @return Current session. + */ + public synchronized GridNioSession session() { + return ses; + } + + /** + * @param ses Session. + */ + public synchronized void session(GridNioSession ses) { + this.ses = ses; + } + /** * @param reqs Requests to notify about error. */ private void notifyOnNodeLeft(SessionWriteRequest[] reqs) { IOException e = new IOException("Failed to send message, node has left: " + node.id()); + IgniteException cloErr = null; for (SessionWriteRequest req : reqs) { req.onError(e); - if (req.ackClosure() != null) - req.ackClosure().apply(new IgniteException(e)); + if (req.ackClosure() != null) { + if (cloErr == null) + cloErr = new IgniteException(e); + + req.ackClosure().apply(cloErr); + } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java index 98a22d6c350dd..c6410c40a360a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java @@ -77,6 +77,9 @@ public class GridNioSessionImpl implements GridNioSession { /** Accepted flag. */ private final boolean accepted; + /** For debug purposes. */ + private volatile boolean markedForClose; + /** * @param filterChain Chain. * @param locAddr Local address. @@ -156,6 +159,8 @@ public GridNioSessionImpl( /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public GridNioFuture close() { + markedForClose = true; + try { return filterChain.onSessionClose(this); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java index d30b122cb2970..d9c3cae4fa92d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java @@ -26,6 +26,7 @@ import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.internal.LT; @@ -377,6 +378,8 @@ Collection writeQueue() { assert recoveryDesc != null; outRecovery = recoveryDesc; + + outRecovery.session(this); } /** {@inheritDoc} */ @@ -436,6 +439,26 @@ Object systemMessage() { return ret; } + /** {@inheritDoc} */ + @Override public GridNioFuture close() { + GridNioFuture fut = super.close(); + + if (!fut.isDone()) { + fut.listen(fut0 -> { + try { + fut0.get(); + } + catch (IgniteCheckedException e) { + log.error("Failed to close session [ses=" + GridSelectorNioSessionImpl.this + ']', e); + } + }); + } + else if (fut.error() != null) + log.error("Failed to close session [ses=" + GridSelectorNioSessionImpl.this + ']', fut.error()); + + return fut; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(GridSelectorNioSessionImpl.class, this, super.toString()); 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 f9fd6fd504ad1..7583d96dd4af2 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 @@ -2714,8 +2714,16 @@ private void sendMessage0(ClusterNode node, Message msg, IgniteInClosure { + /** Serial version uid. */ + private static final long serialVersionUid = 0L; + + /** {@inheritDoc} */ + @Override public Integer call() throws Exception { + return 1; + } + } + + + public void test() throws Exception { + IgniteEx svr = startGrid(0); + + Ignite c1 = startGrid("client1"); + + assertTrue(c1.configuration().isClientMode()); + + Ignite c2 = startGrid("client2"); + + assertTrue(c2.configuration().isClientMode()); + + TestRecordingCommunicationSpi spi2 = (TestRecordingCommunicationSpi)c1.configuration().getCommunicationSpi(); + + spi2.blockMessages(HandshakeMessage2.class, c1.name()); + + AtomicInteger cnt = new AtomicInteger(); + + cnt.getAndAdd(c1.compute(c1.cluster().forNodeId(c2.cluster().localNode().id())).call(new TestClosure())); + + TcpCommunicationSpi spi1 = (TcpCommunicationSpi)c1.configuration().getCommunicationSpi(); + + ConcurrentMap clientsMap = U.field(spi1, "clients"); + + GridCommunicationClient[] arr = clientsMap.get(c2.cluster().localNode().id()); + + GridTcpNioCommunicationClient client = null; + + for (GridCommunicationClient c : arr) { + client = (GridTcpNioCommunicationClient)c; + + if(client != null) { + assertTrue(client.session().outRecoveryDescriptor().reserved()); + + assertFalse(client.session().outRecoveryDescriptor().connected()); + } + } + + assertNotNull(client); + + //spi1.failSend = true; + + IgniteInternalFuture fut = multithreadedAsync(new Runnable() { + @Override public void run() { + doSleep(1000); + + //spi1.failSend = false; + + cnt.getAndAdd(c1.compute(c1.cluster().forNodeId(c2.cluster().localNode().id())).call(new TestClosure())); + } + }, 1, "hang-thread"); + + try { + cnt.getAndAdd(c1.compute(c1.cluster().forNodeId(c2.cluster().localNode().id())).call(new TestClosure())); + + //fail(); + } + catch (IgniteException e) { + // Expected. + } + + fut.get(); + + assertEquals(3, cnt.get()); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java index 8e3411a8cc045..2a6683352ad45 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java @@ -36,6 +36,7 @@ import org.apache.ignite.internal.GridSpiExceptionSelfTest; import org.apache.ignite.internal.GridVersionSelfTest; import org.apache.ignite.internal.IgniteConcurrentEntryProcessorAccessStopTest; +import org.apache.ignite.internal.IgniteConnectionConcurrentReserveAndRemoveTest; import org.apache.ignite.internal.IgniteUpdateNotifierPerClusterSettingSelfTest; import org.apache.ignite.internal.managers.GridManagerStopSelfTest; import org.apache.ignite.internal.managers.communication.GridCommunicationSendMessageSelfTest; @@ -127,6 +128,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(GridLocalEventListenerSelfTest.class); suite.addTestSuite(IgniteTopologyPrintFormatSelfTest.class); suite.addTestSuite(ComputeJobCancelWithServiceSelfTest.class); + suite.addTestSuite(IgniteConnectionConcurrentReserveAndRemoveTest.class); // Managed Services. suite.addTestSuite(GridServiceProcessorSingleNodeSelfTest.class); From cf1d486c9a6444ed6ed83a473072b856fa7473f2 Mon Sep 17 00:00:00 2001 From: tledkov-gridgain Date: Thu, 3 May 2018 16:23:40 +0300 Subject: [PATCH 0242/1463] IGNITE-8347 Test of Memory leaks on restart Ignite node with enabled persistence at ThreadLocal. - Fixes #3889. Signed-off-by: dpavlov (cherry picked from commit 83b5c0e) --- .../ignite/internal/util/GridDebug.java | 69 +++++++++- .../MemoryLeaksOnRestartNodeTest.java | 118 ++++++++++++++++++ 2 files changed, 186 insertions(+), 1 deletion(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/MemoryLeaksOnRestartNodeTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridDebug.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridDebug.java index a8af0fbfcb435..2fa148e19d589 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridDebug.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridDebug.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.util; +import com.sun.management.HotSpotDiagnosticMXBean; import java.io.File; import java.io.FileNotFoundException; import java.io.FileOutputStream; @@ -32,6 +33,7 @@ import java.util.Date; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicReference; +import javax.management.MBeanServer; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgnitePredicate; @@ -67,7 +69,13 @@ public class GridDebug { /** */ private static boolean allowLog; - /** */ + /** This is the name of the HotSpot Diagnostic MBean */ + private static final String HOTSPOT_BEAN_NAME = "com.sun.management:type=HotSpotDiagnostic"; + + /** field to store the hotspot diagnostic MBean */ + private static volatile HotSpotDiagnosticMXBean hotspotMBean; + + /* */ static { if (LOGS_PATH != null) { File log = new File(new File(LOGS_PATH), new SimpleDateFormat("yyyy-MM-dd_HH-mm-ss-").format(new Date()) + @@ -302,6 +310,65 @@ private static String formatEntry(long ts, String threadName, long threadId, Obj Arrays.deepToString(data); } + /** + * Call this method from your application whenever you + * want to dump the heap snapshot into a file. + * + * @param fileName name of the heap dump file + * @param live flag that tells whether to dump + * only the live objects + */ + public static void dumpHeap(String fileName, boolean live) { + // initialize hotspot diagnostic MBean + initHotspotMBean(); + + File f = new File(fileName); + + if (f.exists()) + f.delete(); + + try { + hotspotMBean.dumpHeap(fileName, live); + } + catch (RuntimeException re) { + throw re; + } + catch (Exception exp) { + throw new RuntimeException(exp); + } + } + + /** + * Initialize the hotspot diagnostic MBean field + */ + private static void initHotspotMBean() { + if (hotspotMBean == null) { + synchronized (GridDebug.class) { + if (hotspotMBean == null) + hotspotMBean = getHotspotMBean(); + } + } + } + + /** + * Gets the hotspot diagnostic MBean from the platform MBean server + * @return Diagnostic bean. + */ + private static HotSpotDiagnosticMXBean getHotspotMBean() { + try { + MBeanServer server = ManagementFactory.getPlatformMBeanServer(); + + HotSpotDiagnosticMXBean bean = ManagementFactory.newPlatformMXBeanProxy(server, + HOTSPOT_BEAN_NAME, HotSpotDiagnosticMXBean.class); + + return bean; + } catch (RuntimeException re) { + throw re; + } catch (Exception exp) { + throw new RuntimeException(exp); + } + } + /** * Debug info queue item. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/MemoryLeaksOnRestartNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/MemoryLeaksOnRestartNodeTest.java new file mode 100644 index 0000000000000..56ab091fe2b7e --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/MemoryLeaksOnRestartNodeTest.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal; + +import java.io.File; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.configuration.DataRegionConfiguration; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.GridDebug; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests leaks on node restart with enabled persistence. + */ +public class MemoryLeaksOnRestartNodeTest extends GridCommonAbstractTest { + /** Heap dump file name. */ + private static final String HEAP_DUMP_FILE_NAME = "test.hprof"; + + /** Restarts count. */ + private static final int RESTARTS = 10; + + /** Nodes count. */ + private static final int NODES = 3; + + /** Allow 5Mb leaks on node restart. */ + private static final int ALLOW_LEAK_ON_RESTART_IN_MB = 1; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setDataStorageConfiguration(new DataStorageConfiguration() + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration().setName("mem0").setPersistenceEnabled(false)) + .setDataRegionConfigurations( + new DataRegionConfiguration().setName("disk").setPersistenceEnabled(true), + new DataRegionConfiguration().setName("mem2").setPersistenceEnabled(false))); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + + super.afterTestsStopped(); + } + + /** + * @throws Exception On failed. + */ + public void test() throws Exception { + System.setProperty(IgniteSystemProperties.IGNITE_DELAYED_REPLACED_PAGE_WRITE, "false"); + + // Warmup + for (int i = 0; i < RESTARTS / 2; ++i) { + startGrids(NODES); + + U.sleep(500); + + stopAllGrids(); + } + + GridDebug.dumpHeap(HEAP_DUMP_FILE_NAME, true); + + File dumpFile = new File(HEAP_DUMP_FILE_NAME); + + final long size0 = dumpFile.length(); + + // Restarts + for (int i = 0; i < RESTARTS; ++i) { + startGrids(NODES); + + U.sleep(500); + + stopAllGrids(); + + GridDebug.dumpHeap(HEAP_DUMP_FILE_NAME, true); + } + + GridDebug.dumpHeap(HEAP_DUMP_FILE_NAME, true); + + final float leakSize = (float)(dumpFile.length() - size0) / 1024 / 1024 / NODES / RESTARTS; + + assertTrue("Possible leaks detected. The " + leakSize + "M leaks per node restart after " + RESTARTS + + " restarts. See the '" + dumpFile.getAbsolutePath() + "'", + leakSize < ALLOW_LEAK_ON_RESTART_IN_MB); + + // Remove dump if successful. + dumpFile.delete(); + } +} \ No newline at end of file From 9ad3fea88a05c31de24ce00620fa32bc6be5ae4b Mon Sep 17 00:00:00 2001 From: dpavlov Date: Thu, 3 May 2018 18:26:49 +0300 Subject: [PATCH 0243/1463] IGNITE-8347 Memory leaks on restart Ignite node with enabled persistence at ThreadLocal - Fixes #3891. Signed-off-by: dpavlov (cherry picked from commit b1db693) --- .../pagemem/DelayedDirtyPageWrite.java | 6 +++--- .../pagemem/DelayedPageReplacementTracker.java | 16 +++++++--------- .../internal/MemoryLeaksOnRestartNodeTest.java | 3 --- 3 files changed, 10 insertions(+), 15 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/DelayedDirtyPageWrite.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/DelayedDirtyPageWrite.java index 6eec609999d3b..b08ddc2f89146 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/DelayedDirtyPageWrite.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/DelayedDirtyPageWrite.java @@ -24,9 +24,9 @@ import org.jetbrains.annotations.Nullable; /** - * Not thread safe and stateful class for replacement of one page with write() delay. This allows to write page content - * without holding segment lock. Page data is copied into temp buffer during {@link #writePage(FullPageId, ByteBuffer, - * int)} and then sent to real implementation by {@link #finishReplacement()}. + * Not thread safe and stateful class for page replacement of one page with write() delay. This allows to write page + * content without holding segment lock. Page data is copied into temp buffer during {@link #writePage(FullPageId, + * ByteBuffer, int)} and then sent to real implementation by {@link #finishReplacement()}. */ public class DelayedDirtyPageWrite implements ReplacedPageWriter { /** Real flush dirty page implementation. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/DelayedPageReplacementTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/DelayedPageReplacementTracker.java index 9cf5b777240b1..aa1b06161c042 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/DelayedPageReplacementTracker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/DelayedPageReplacementTracker.java @@ -21,6 +21,8 @@ import java.nio.ByteOrder; import java.util.Collection; import java.util.HashSet; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import org.apache.ignite.IgniteInterruptedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.pagemem.FullPageId; @@ -56,15 +58,10 @@ public class DelayedPageReplacementTracker { /** * Dirty page write for replacement operations thread local. Because page write {@link DelayedDirtyPageWrite} is - * stateful and not thread safe, this thread local protects from GC pressure on pages replacement. + * stateful and not thread safe, this thread local protects from GC pressure on pages replacement.
    Map is used + * instead of build-in thread local to allow GC to remove delayed writers for alive threads after node stop. */ - private final ThreadLocal delayedPageWriteThreadLoc - = new ThreadLocal() { - @Override protected DelayedDirtyPageWrite initialValue() { - return new DelayedDirtyPageWrite(flushDirtyPage, byteBufThreadLoc, pageSize, - DelayedPageReplacementTracker.this); - } - }; + private final Map delayedPageWriteThreadLocMap = new ConcurrentHashMap<>(); /** * @param pageSize Page size. @@ -87,7 +84,8 @@ public DelayedPageReplacementTracker(int pageSize, ReplacedPageWriter flushDirty * @return delayed page write implementation, finish method to be called to actually write page. */ public DelayedDirtyPageWrite delayedPageWrite() { - return delayedPageWriteThreadLoc.get(); + return delayedPageWriteThreadLocMap.computeIfAbsent(Thread.currentThread().getId(), + id -> new DelayedDirtyPageWrite(flushDirtyPage, byteBufThreadLoc, pageSize, this)); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/MemoryLeaksOnRestartNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/MemoryLeaksOnRestartNodeTest.java index 56ab091fe2b7e..df055fee53ede 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/MemoryLeaksOnRestartNodeTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/MemoryLeaksOnRestartNodeTest.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal; import java.io.File; -import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.configuration.DataRegionConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; @@ -76,8 +75,6 @@ public class MemoryLeaksOnRestartNodeTest extends GridCommonAbstractTest { * @throws Exception On failed. */ public void test() throws Exception { - System.setProperty(IgniteSystemProperties.IGNITE_DELAYED_REPLACED_PAGE_WRITE, "false"); - // Warmup for (int i = 0; i < RESTARTS / 2; ++i) { startGrids(NODES); From 81d2dbc02c1f03b9288a4d17b05784f0c324b852 Mon Sep 17 00:00:00 2001 From: Andrey Gura Date: Fri, 11 May 2018 20:41:30 +0300 Subject: [PATCH 0244/1463] IGNITE-8471 Dependencies upgraded (cherry picked from commit 7c6d5a4) --- modules/cassandra/store/pom.xml | 6 +++++- modules/flink/pom.xml | 12 ++++++++++++ modules/flume/pom.xml | 6 ++++++ modules/gce/pom.xml | 6 ++++++ modules/hadoop/pom.xml | 32 ++++++++++++++++++++++++++++++++ modules/spark-2.10/pom.xml | 10 ++++++++++ modules/spark/pom.xml | 20 ++++++++++++++++++++ modules/yarn/pom.xml | 20 ++++++++++++++++++++ parent/pom.xml | 6 +++--- 9 files changed, 114 insertions(+), 4 deletions(-) diff --git a/modules/cassandra/store/pom.xml b/modules/cassandra/store/pom.xml index b62050b1c1218..e236b529154c8 100644 --- a/modules/cassandra/store/pom.xml +++ b/modules/cassandra/store/pom.xml @@ -35,7 +35,7 @@ http://ignite.apache.org - 1.8.3 + 1.9.2 3.0.0 3.3 4.0.33.Final @@ -130,6 +130,10 @@ log4j-over-slf4j org.slf4j + + commons-codec + commons-codec + diff --git a/modules/flink/pom.xml b/modules/flink/pom.xml index 1013e4c8f4487..48a4e4ce4561b 100644 --- a/modules/flink/pom.xml +++ b/modules/flink/pom.xml @@ -74,6 +74,18 @@ org.apache.zookeeper zookeeper + + commons-beanutils + commons-beanutils + + + commons-beanutils + commons-beanutils-bean-collections + + + commons-codec + commons-codec + diff --git a/modules/flume/pom.xml b/modules/flume/pom.xml index dcee0dbb316df..12da725cabeff 100644 --- a/modules/flume/pom.xml +++ b/modules/flume/pom.xml @@ -45,6 +45,12 @@ org.apache.flume flume-ng-core ${flume.ng.version} + + + commons-codec + commons-codec + + diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml index ba6d484c4957d..d81fea562a968 100644 --- a/modules/gce/pom.xml +++ b/modules/gce/pom.xml @@ -45,6 +45,12 @@ com.google.api-client google-api-client 1.22.0 + + + commons-codec + commons-codec + + diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml index 5ed32789e0c85..a7b4a984bd62b 100644 --- a/modules/hadoop/pom.xml +++ b/modules/hadoop/pom.xml @@ -70,24 +70,56 @@ org.apache.hadoop hadoop-auth ${hadoop.version} + + + commons-codec + commons-codec + + org.apache.hadoop hadoop-common ${hadoop.version} + + + commons-beanutils + commons-beanutils + + + commons-beanutils + commons-beanutils-core + + + commons-codec + commons-codec + + org.apache.hadoop hadoop-hdfs ${hadoop.version} + + + commons-codec + commons-codec + + org.apache.hadoop hadoop-mapreduce-client-common ${hadoop.version} + + + commons-codec + commons-codec + + diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml index 1f40f0f3cda20..c67066131ac26 100644 --- a/modules/spark-2.10/pom.xml +++ b/modules/spark-2.10/pom.xml @@ -95,6 +95,16 @@ org.apache.hadoop hadoop-common ${spark.hadoop.version} + + + commons-beanutils + commons-beanutils + + + commons-beanutils + commons-beanutils-core + + diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml index e338c8823da6c..fb3bcc72bca35 100644 --- a/modules/spark/pom.xml +++ b/modules/spark/pom.xml @@ -71,6 +71,12 @@ org.apache.spark spark-catalyst_2.11 ${spark.version} + + + commons-codec + commons-codec + + @@ -101,6 +107,20 @@ org.apache.hadoop hadoop-common ${spark.hadoop.version} + + + commons-beanutils + commons-beanutils + + + commons-beanutils + commons-beanutils-core + + + commons-codec + commons-codec + + diff --git a/modules/yarn/pom.xml b/modules/yarn/pom.xml index ce7217b0d37e9..c76d3c9b6810b 100644 --- a/modules/yarn/pom.xml +++ b/modules/yarn/pom.xml @@ -43,12 +43,32 @@ org.apache.hadoop hadoop-yarn-client ${hadoop-yarn.version} + + + commons-codec + commons-codec + + org.apache.hadoop hadoop-common ${hadoop-yarn.version} + + + commons-beanutils + commons-beanutils + + + commons-beanutils + commons-beanutils-core + + + commons-codec + commons-codec + + diff --git a/parent/pom.xml b/parent/pom.xml index 490c531427002..27ee390507d7d 100644 --- a/parent/pom.xml +++ b/parent/pom.xml @@ -54,9 +54,9 @@ 1.10.12_1 1.11.75 2.16.0 - 1.8.3_1 - 1.8.3 - 1.6 + 1.9.2_1 + 1.9.2 + 1.11 3.2.2 2.6 2.2.5 From 144db70e618b60f100e84b35529df0be27b6a80d Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Tue, 8 May 2018 10:02:20 +0300 Subject: [PATCH 0245/1463] IGNITE-8434 .NET: Fix service proxy generation * Fix proxies on .NET Core * Fix interface hierarchy handling This closes #3946 (cherry picked from commit c9106fb) --- .../Services/ServicesTest.cs | 49 ++++++++------ .../Apache.Ignite.Core.csproj | 1 - .../Impl/Binary/ReflectionUtils.cs | 50 ++++++++++++-- .../Impl/Services/ServiceMethodHelper.cs | 61 ----------------- .../Services/ServiceProxyTypeGenerator.cs | 66 ++++++++++++++----- 5 files changed, 122 insertions(+), 105 deletions(-) delete mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServiceMethodHelper.cs diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs index c6a8e0bdf57f9..017a580f1a9c0 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs @@ -379,7 +379,7 @@ public void TestGetDynamicServiceProxyLocal() } /// - /// Tests the duck typing: proxy interface can be different from actual service interface, + /// Tests the duck typing: proxy interface can be different from actual service interface, /// only called method signature should be compatible. /// [Test] @@ -389,7 +389,7 @@ public void TestDuckTyping([Values(true, false)] bool local) // Deploy locally or to the remote node var nodeId = (local ? Grid1 : Grid2).GetCluster().GetLocalNode().Id; - + var cluster = Grid1.GetCluster().ForNodeIds(nodeId); cluster.GetServices().DeployNodeSingleton(SvcName, svc); @@ -399,7 +399,7 @@ public void TestDuckTyping([Values(true, false)] bool local) // NodeId signature is the same as in service Assert.AreEqual(nodeId, prx.NodeId); - + // Method signature is different from service signature (object -> object), but is compatible. Assert.AreEqual(15, prx.Method(15)); @@ -434,7 +434,7 @@ public void TestServiceDescriptors() var top = desc.TopologySnapshot; var prx = Services.GetServiceProxy(SvcName); - + Assert.AreEqual(1, top.Count); Assert.AreEqual(prx.NodeId, top.Keys.Single()); Assert.AreEqual(1, top.Values.Single()); @@ -463,7 +463,7 @@ public void TestWithKeepBinaryClient() res = (IBinaryObject) prx.Method(Grid1.GetBinary().ToBinary(obj)); Assert.AreEqual(11, res.Deserialize().Val); } - + /// /// Tests the server binary flag. /// @@ -558,10 +558,10 @@ public void TestDeployAllException([Values(true, false)] bool binarizable) MaxPerNodeCount = 2, TotalCount = 2, NodeFilter = new NodeFilter { NodeId = Grid1.GetCluster().GetLocalNode().Id }, - Service = binarizable ? new TestIgniteServiceBinarizable { TestProperty = i, ThrowInit = throwInit } + Service = binarizable ? new TestIgniteServiceBinarizable { TestProperty = i, ThrowInit = throwInit } : new TestIgniteServiceSerializable { TestProperty = i, ThrowInit = throwInit } }); - } + } var deploymentException = Assert.Throws(() => Services.DeployAll(cfgs)); @@ -569,9 +569,9 @@ public void TestDeployAllException([Values(true, false)] bool binarizable) Assert.IsNotNull(failedCfgs); Assert.AreEqual(2, failedCfgs.Count); - var firstFailedSvc = binarizable ? failedCfgs.ElementAt(0).Service as TestIgniteServiceBinarizable : + var firstFailedSvc = binarizable ? failedCfgs.ElementAt(0).Service as TestIgniteServiceBinarizable : failedCfgs.ElementAt(0).Service as TestIgniteServiceSerializable; - var secondFailedSvc = binarizable ? failedCfgs.ElementAt(1).Service as TestIgniteServiceBinarizable : + var secondFailedSvc = binarizable ? failedCfgs.ElementAt(1).Service as TestIgniteServiceBinarizable : failedCfgs.ElementAt(1).Service as TestIgniteServiceSerializable; Assert.IsNotNull(firstFailedSvc); @@ -680,7 +680,7 @@ private void VerifyDeploymentException(Action deploy, bool var deploymentException = Assert.Throws(() => deploy(services, svc)); - var text = keepBinary + var text = keepBinary ? "Service deployment failed with a binary error. Examine BinaryCause for details." : "Service deployment failed with an exception. Examine InnerException for details."; @@ -759,7 +759,7 @@ public void TestMarshalExceptionOnRead() var ex = Assert.Throws(() => Services.DeployMultiple(SvcName, svc, Grids.Length, 1)); - + Assert.IsNotNull(ex.InnerException); Assert.AreEqual("Expected exception", ex.InnerException.Message); @@ -858,7 +858,7 @@ public void TestCallJavaService() var arr = new [] {10, 11, 12}.Select(x => new PlatformComputeBinarizable {Field = x}).ToArray(); Assert.AreEqual(new[] {11, 12, 13}, svc.testBinarizableCollection(arr) .OfType().Select(x => x.Field).ToArray()); - Assert.AreEqual(new[] {11, 12, 13}, + Assert.AreEqual(new[] {11, 12, 13}, svc.testBinarizableArray(arr).OfType().Select(x => x.Field).ToArray()); // Binary object @@ -1074,12 +1074,22 @@ private static string MakeServiceName(int i) } /// - /// Test service interface for proxying. + /// Test base service. /// - public interface ITestIgniteService + public interface ITestIgniteServiceBase { + /** */ int TestProperty { get; set; } + /** */ + object Method(object arg); + } + + /// + /// Test service interface for proxying. + /// + public interface ITestIgniteService : IService, ITestIgniteServiceBase + { /** */ bool Initialized { get; } @@ -1095,9 +1105,6 @@ public interface ITestIgniteService /** */ string LastCallContextName { get; } - /** */ - object Method(object arg); - /** */ object ErrMethod(object arg); } @@ -1124,7 +1131,7 @@ public interface ITestIgniteServiceProxyInterface /// Test serializable service. /// [Serializable] - private class TestIgniteServiceSerializable : IService, ITestIgniteService + private class TestIgniteServiceSerializable : ITestIgniteService { /** */ [InstanceResource] @@ -1232,7 +1239,7 @@ private void CheckContext(IServiceContext context) if (context.AffinityKey != null && !(context.AffinityKey is int)) { var binaryObj = context.AffinityKey as IBinaryObject; - + var key = binaryObj != null ? binaryObj.Deserialize() : (BinarizableObject) context.AffinityKey; @@ -1279,7 +1286,7 @@ private class TestIgniteServiceBinarizableErr : TestIgniteServiceSerializable, I public void WriteBinary(IBinaryWriter writer) { writer.WriteInt("TestProp", TestProperty); - + if (ThrowOnWrite) throw new Exception("Expected exception"); } @@ -1288,7 +1295,7 @@ public void WriteBinary(IBinaryWriter writer) public void ReadBinary(IBinaryReader reader) { TestProperty = reader.ReadInt("TestProp"); - + throw new Exception("Expected exception"); } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj index f217a0d34abd7..fc67e6cf8ab9a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj @@ -112,7 +112,6 @@ - diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/ReflectionUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/ReflectionUtils.cs index 9f004ae8b853d..6ea9e2fc743c9 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/ReflectionUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/ReflectionUtils.cs @@ -20,6 +20,7 @@ namespace Apache.Ignite.Core.Impl.Binary using System; using System.Collections.Generic; using System.Diagnostics; + using System.Linq; using System.Reflection; /// @@ -27,6 +28,13 @@ namespace Apache.Ignite.Core.Impl.Binary /// internal static class ReflectionUtils { + /** */ + private const BindingFlags BindFlags = + BindingFlags.Public | + BindingFlags.NonPublic | + BindingFlags.Instance | + BindingFlags.DeclaredOnly; + /// /// Gets all fields, including base classes. /// @@ -57,18 +65,48 @@ public static IEnumerable> GetFieldsAndProperties Debug.Assert(type != null); if (type.IsPrimitive) + { yield break; + } - const BindingFlags bindingFlags = BindingFlags.Public | BindingFlags.NonPublic | BindingFlags.Instance | - BindingFlags.DeclaredOnly; - - while (type != typeof(object) && type != null) + foreach (var t in GetSelfAndBaseTypes(type)) { - foreach (var fieldInfo in type.GetFields(bindingFlags)) + foreach (var fieldInfo in t.GetFields(BindFlags)) yield return new KeyValuePair(fieldInfo, fieldInfo.FieldType); - foreach (var propertyInfo in type.GetProperties(bindingFlags)) + foreach (var propertyInfo in t.GetProperties(BindFlags)) yield return new KeyValuePair(propertyInfo, propertyInfo.PropertyType); + } + } + + /// + /// Gets methods, including base classes. + /// + /// The type. + public static IEnumerable GetMethods(Type type) + { + Debug.Assert(type != null); + + if (type.IsInterface) + { + return type.GetInterfaces().Concat(new[] {typeof(object), type}) + .SelectMany(t => t.GetMethods(BindFlags)); + } + + return GetSelfAndBaseTypes(type) + .SelectMany(t => t.GetMethods(BindFlags)); + } + + /// + /// Returns full type hierarchy. + /// + private static IEnumerable GetSelfAndBaseTypes(Type type) + { + Debug.Assert(type != null); + + while (type != typeof(object) && type != null) + { + yield return type; type = type.BaseType; } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServiceMethodHelper.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServiceMethodHelper.cs deleted file mode 100644 index e6fb2c0fe62bf..0000000000000 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServiceMethodHelper.cs +++ /dev/null @@ -1,61 +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. - */ - -namespace Apache.Ignite.Core.Impl.Services -{ - using System; - using System.Collections.Generic; - using System.Diagnostics; - using System.Reflection; - - /// - /// Provides reflection information about types. - /// This class used by ServiceProxyTypeGenerator and by generated proxy (to initialize static field). - /// - internal static class ServiceMethodHelper - { - /// - /// Provides information about virtual methods of the type - /// - /// Type to inspect. - /// List of virtual methods. - public static MethodInfo[] GetVirtualMethods(Type type) - { - Debug.Assert(type != null); - var methods = new List(); - - foreach (var method in type.GetMethods(BindingFlags.Instance | BindingFlags.Public | - BindingFlags.NonPublic | BindingFlags.DeclaredOnly)) - { - if (method.IsVirtual) - methods.Add(method); - } - - if (type.IsInterface) - { - foreach (var method in typeof(object).GetMethods( - BindingFlags.Instance | BindingFlags.Public | BindingFlags.DeclaredOnly)) - { - if (method.IsVirtual) - methods.Add(method); - } - } - - return methods.ToArray(); - } - } -} \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxyTypeGenerator.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxyTypeGenerator.cs index 97de9c7e7dc50..8c84ccd6df749 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxyTypeGenerator.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServiceProxyTypeGenerator.cs @@ -19,8 +19,11 @@ namespace Apache.Ignite.Core.Impl.Services { using System; using System.Diagnostics; + using System.Linq; + using System.Linq.Expressions; using System.Reflection; using System.Reflection.Emit; + using Apache.Ignite.Core.Impl.Binary; using ProxyAction = System.Func; /// @@ -34,6 +37,26 @@ internal static class ServiceProxyTypeGenerator /** */ private static readonly MethodInfo InvokeMethod = ActionType.GetMethod("Invoke"); + /** */ + private static readonly MethodInfo Finalizer = typeof(object) + .GetMethod("Finalize", BindingFlags.Instance | BindingFlags.NonPublic); + + /** Classic .NET Method. */ + private static readonly MethodInfo AppDomainDefineAssembly = typeof(AppDomain).GetMethod( + "DefineDynamicAssembly", + BindingFlags.Public | BindingFlags.Instance, + null, + new[] {typeof(AssemblyName), typeof(AssemblyBuilderAccess)}, + null); + + /** .NET Core Method. */ + private static readonly MethodInfo AssemblyBuilderDefineAssembly = typeof(AssemblyBuilder).GetMethod( + "DefineDynamicAssembly", + BindingFlags.Public | BindingFlags.Static, + null, + new[] {typeof(AssemblyName), typeof(AssemblyBuilderAccess)}, + null); + /** */ private static readonly ModuleBuilder ModuleBuilder = CreateModuleBuilder(); @@ -59,7 +82,12 @@ public static Tuple Generate(Type serviceType) GenerateStaticConstructor(buildContext); GenerateConstructor(buildContext); - buildContext.Methods = ServiceMethodHelper.GetVirtualMethods(buildContext.ServiceType); + Debug.Assert(Finalizer != null); + + buildContext.Methods = ReflectionUtils.GetMethods(buildContext.ServiceType) + .Where(m => m.IsVirtual && m != Finalizer) + .ToArray(); + for (var i = 0; i < buildContext.Methods.Length; i++) { GenerateMethod(buildContext, i); @@ -76,15 +104,21 @@ private static ModuleBuilder CreateModuleBuilder() { var name = Guid.NewGuid().ToString("N"); -#if !NETCOREAPP2_0 - var assemblyBuilder = - AppDomain.CurrentDomain.DefineDynamicAssembly(new AssemblyName(name), - AssemblyBuilderAccess.RunAndCollect); -#else - var assemblyBuilder = - AssemblyBuilder.DefineDynamicAssembly(new AssemblyName(name), - AssemblyBuilderAccess.RunAndCollect); -#endif + var asmName = Expression.Constant(new AssemblyName(name)); + var access = Expression.Constant(AssemblyBuilderAccess.RunAndCollect); + var domain = Expression.Constant(AppDomain.CurrentDomain); + + // AppDomain.DefineDynamicAssembly is not available on .NET Core; + // AssemblyBuilder.DefineDynamicAssembly is not available on .NET 4. + // Both of them can not be called with Reflection. + // So we have to be creative and use expression trees. + var callExpr = AppDomainDefineAssembly != null + ? Expression.Call(domain, AppDomainDefineAssembly, asmName, access) + : Expression.Call(AssemblyBuilderDefineAssembly, asmName, access); + + var callExprLambda = Expression.Lambda>(callExpr); + + var assemblyBuilder = callExprLambda.Compile()(); return assemblyBuilder.DefineDynamicModule(name); } @@ -97,7 +131,7 @@ private static void GenerateFields(ProxyBuildContext buildContext) // Static field - empty object array to optimize calls without parameters. buildContext.EmptyParametersField = buildContext.ProxyType.DefineField("_emptyParameters", typeof(object[]), FieldAttributes.Static | FieldAttributes.Private | FieldAttributes.InitOnly); - + // Instance field for function to invoke. buildContext.ActionField = buildContext.ProxyType.DefineField("_action", ActionType, FieldAttributes.Private | FieldAttributes.InitOnly); @@ -197,10 +231,10 @@ private static void GenerateMethod(ProxyBuildContext buildContext, int methodInd // Load methods array field. gen.Emit(OpCodes.Ldarg_0); gen.Emit(OpCodes.Ldfld, buildContext.MethodsField); - + // Load index of method. gen.Emit(OpCodes.Ldc_I4, methodIndex); - + // Load array element. gen.Emit(OpCodes.Ldelem_Ref); @@ -215,10 +249,10 @@ private static void GenerateMethod(ProxyBuildContext buildContext, int methodInd for (var i = 0; i < parameters.Length; i++) { gen.Emit(OpCodes.Dup); - + // Parameter's index in array. gen.Emit(OpCodes.Ldc_I4, i); - + // Parameter's value. gen.Emit(OpCodes.Ldarg, i + 1); if (parameterTypes[i].IsValueType) @@ -278,4 +312,4 @@ public ProxyBuildContext(TypeBuilder proxyType, Type serviceType) /** */ public MethodInfo[] Methods { get; set; } } } -} \ No newline at end of file +} From 8d7669be99be984da6223e734d7434e9f76335b9 Mon Sep 17 00:00:00 2001 From: Turik Campbell Date: Tue, 8 May 2018 16:46:37 +0300 Subject: [PATCH 0246/1463] IGNITE-8041: Add a GA Grid example that solves 'Knapsack Problem' This closes #3947 (cherry picked from commit 297df32) --- .../examples/ml/genetic/knapsack/Item.java | 90 +++++ .../knapsack/KnapsackFitnessFunction.java | 69 ++++ .../genetic/knapsack/KnapsackGAExample.java | 325 ++++++++++++++++++ .../knapsack/KnapsackTerminateCriteria.java | 100 ++++++ 4 files changed, 584 insertions(+) create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/Item.java create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/KnapsackFitnessFunction.java create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/KnapsackGAExample.java create mode 100644 examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/KnapsackTerminateCriteria.java diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/Item.java b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/Item.java new file mode 100644 index 0000000000000..f64cb173bb091 --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/Item.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.examples.ml.genetic.knapsack; + +import java.io.Serializable; + +/** + * POJO to model an Item + */ +public class Item implements Serializable { + /** weight of item in lbs. */ + private double weight; + /** value of item */ + private double value; + /** name of item */ + private String name; + + /** + * Get the weight + * + * @return Weight + */ + public double getWeight() { + return weight; + } + + /** + * Set the weight + * + * @param weight Weight + */ + public void setWeight(double weight) { + this.weight = weight; + } + + /** + * Get the value + * + * @return Value + */ + public double getValue() { + return value; + } + + /** + * @param value Value + */ + public void setValue(double value) { + this.value = value; + } + + /** + * Get the name + * + * @return Name + */ + public String getName() { + return name; + } + + /** + * Set the name + * + * @param name Name + */ + public void setName(String name) { + this.name = name; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "Item [weight=" + weight + ", value=" + value + ", name=" + name + "]"; + } + +} diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/KnapsackFitnessFunction.java b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/KnapsackFitnessFunction.java new file mode 100644 index 0000000000000..5ccaa8b5b93aa --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/KnapsackFitnessFunction.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.examples.ml.genetic.knapsack; + +import java.util.ArrayList; +import java.util.List; +import org.apache.ignite.ml.genetic.Gene; +import org.apache.ignite.ml.genetic.IFitnessFunction; + +/** + * This example demonstrates how to create a IFitnessFunction + * + * Your IFitnessFunction will vary depending on your particular use case. + * + * For this fitness function, we simply want to calculate the weight and value of + * + * an individual solution relative to other solutions. + * + * + * To do this, we total the weights and values of all the genes within a chromosome. + */ +public class KnapsackFitnessFunction implements IFitnessFunction { + /** weight capacity of knapsack */ + private double maximumWeight = 20; + + /** + * Calculate fitness + * + * @param genes List of Genes + * @return Fitness value + */ + public double evaluate(List genes) { + + double value = 0; + double weight = 0; + + List dups = new ArrayList(); + int badSolution = 1; + + for (Gene agene : genes) { + weight = weight + ((Item)(agene.getValue())).getWeight(); + value = value + ((Item)(agene.getValue())).getValue(); + + if (dups.contains(agene.id()) || (weight > maximumWeight)) { + badSolution = 0; + break; + } + else + dups.add(agene.id()); + } + + return (value * badSolution); + } +} 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 new file mode 100644 index 0000000000000..0029a9e2e69ed --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/KnapsackGAExample.java @@ -0,0 +1,325 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.genetic.knapsack; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.ignite.Ignite; +import org.apache.ignite.Ignition; + +import org.apache.ignite.ml.genetic.GAGrid; +import org.apache.ignite.ml.genetic.Gene; +import org.apache.ignite.ml.genetic.parameter.GAConfiguration; + +/** + * This example demonstrates how to use the GAGrid framework. + * + * Example demonstrates Knapsack Problem: Given a set of 30 items, each with a weight and a value, pack 10 items in + * knapsack so that the total weight is less <= 20 lbs. and the total value is maximized. + * + * + * How To Run: + * + * mvn exec:java -Dexec.mainClass="org.apache.ignite.examples.ml.genetic.knapsack.KnapsackGAExample" + * + *

    Remote nodes should always be started with special configuration file which enables P2P class loading: {@code + * 'ignite.{sh|bat} examples/config/example-ignite.xml'}.

    Alternatively you can run ExampleNodeStartup in + * another JVM which will start node with {@code examples/config/example-ignite.xml} configuration.

    + */ +public class KnapsackGAExample { + /** Ignite instance */ + private static Ignite ignite = null; + /** GAGrid */ + private static GAGrid gaGrid = null; + /** GAConfiguration */ + private static GAConfiguration gaConfig = null; + + /** + * @param args Command line arguments, none required. + */ + public static void main(String args[]) { + System.setProperty("IGNITE_QUIET", "false"); + + try { + + //Create an Ignite instance as you would in any other use case. + ignite = Ignition.start("examples/config/example-ignite.xml"); + + // Create GAConfiguration + gaConfig = new GAConfiguration(); + + // set Gene Pool + List genes = getGenePool(); + + // set the Chromosome Length to '10' since our knapsack may contain a total of 10 items. + gaConfig.setChromosomeLength(10); + + // initialize gene pool + gaConfig.setGenePool(genes); + + // create and set Fitness function + KnapsackFitnessFunction function = new KnapsackFitnessFunction(); + gaConfig.setFitnessFunction(function); + + // create and set TerminateCriteria + KnapsackTerminateCriteria termCriteria = new KnapsackTerminateCriteria(ignite); + gaConfig.setTerminateCriteria(termCriteria); + + ignite.log(); + + gaGrid = new GAGrid(gaConfig, ignite); + // evolve the population + gaGrid.evolve(); + + Ignition.stop(true); + + ignite = null; + + } + catch (Exception e) { + System.out.println(e); + } + + } + + /** + * Helper routine to initialize Gene pool + * + * In typical usecase genes may be stored in database. + * + * @return List + */ + private static List getGenePool() { + List list = new ArrayList(); + + Item item1 = new Item(); + item1.setName("Swiss Army Knife"); + item1.setWeight(0.08125); + item1.setValue(15); + Gene gene1 = new Gene(item1); + + Item item2 = new Item(); + item2.setName("Duct Tape"); + item2.setWeight(1.3); + item2.setValue(3); + Gene gene2 = new Gene(item2); + + Item item3 = new Item(); + item3.setName("Rope (50 feet)"); + item3.setWeight(7); + item3.setValue(10); + Gene gene3 = new Gene(item3); + + Item item4 = new Item(); + item4.setName("Satellite phone"); + item4.setWeight(2); + item4.setValue(8); + Gene gene4 = new Gene(item4); + + Item item5 = new Item(); + item5.setName("Elmer's Glue"); + item5.setWeight(0.25); + item5.setValue(2); + Gene gene5 = new Gene(item5); + + Item item6 = new Item(); + item6.setName("Toilet Paper Roll"); + item6.setWeight(.5); + item6.setValue(4); + Gene gene6 = new Gene(item6); + + Item item7 = new Item(); + item7.setName("Binoculars"); + item7.setWeight(3); + item7.setValue(5); + Gene gene7 = new Gene(item7); + + Item item8 = new Item(); + item8.setName("Compass"); + item8.setWeight(0.0573202); + item8.setValue(15); + Gene gene8 = new Gene(item8); + + Item item9 = new Item(); + item9.setName("Jug (prefilled with water)"); + item9.setWeight(4); + item9.setValue(6); + Gene gene9 = new Gene(item9); + + Item item10 = new Item(); + item10.setName("Flashlight"); + item10.setWeight(2); + item10.setValue(4); + Gene gene10 = new Gene(item10); + + Item item11 = new Item(); + item11.setName("Box of paper clips"); + item11.setWeight(.9); + item11.setValue(2); + Gene gene11 = new Gene(item11); + + Item item12 = new Item(); + item12.setName("Gloves (1 pair)"); + item12.setWeight(.8125); + item12.setValue(3); + Gene gene12 = new Gene(item12); + + Item item13 = new Item(); + item13.setName("Scissors"); + item13.setWeight(0.2); + item13.setValue(2); + Gene gene13 = new Gene(item13); + + Item item14 = new Item(); + item14.setName("Signal Flair (4pk)"); + item14.setWeight(4); + item14.setValue(5); + Gene gene14 = new Gene(item14); + + Item item15 = new Item(); + item15.setName("Water Purifying System"); + item15.setWeight(0.5125); + item15.setValue(4); + Gene gene15 = new Gene(item15); + + Item item16 = new Item(); + item16.setName("Whistle"); + item16.setWeight(0.075); + item16.setValue(2); + Gene gene16 = new Gene(item16); + + Item item17 = new Item(); + item17.setName("Sleeping Bag"); + item17.setWeight(0.38125); + item17.setValue(4); + Gene gene17 = new Gene(item17); + + Item item18 = new Item(); + item18.setName("Insect Repellent"); + item18.setWeight(1.15); + item18.setValue(3); + Gene gene18 = new Gene(item18); + + Item item19 = new Item(); + item19.setName("Trowel"); + item19.setWeight(0.31875); + item19.setValue(3); + Gene gene19 = new Gene(item19); + + Item item20 = new Item(); + item20.setName("Lighter"); + item20.setWeight(.2); + item20.setValue(4); + Gene gene20 = new Gene(item20); + + Item item21 = new Item(); + item21.setName("Safety Horn"); + item21.setWeight(.21); + item21.setValue(3); + Gene gene21 = new Gene(item21); + + Item item22 = new Item(); + item22.setName("Headlamp"); + item22.setWeight(.8); + item22.setValue(4); + Gene gene22 = new Gene(item22); + + Item item23 = new Item(); + item23.setName("Freeze Dried Food Kit"); + item23.setWeight(2); + item23.setValue(6); + Gene gene23 = new Gene(item23); + + Item item24 = new Item(); + item24.setName("Sunscreen"); + item24.setWeight(.5); + item24.setValue(4); + Gene gene24 = new Gene(item24); + + Item item25 = new Item(); + item25.setName("Trekking Pole (Adjustable)"); + item25.setWeight(1.3); + item25.setValue(4); + Gene gene25 = new Gene(item25); + + Item item26 = new Item(); + item26.setName("Counter Assault Bear Spray"); + item26.setWeight(.5); + item26.setValue(4); + Gene gene26 = new Gene(item26); + + Item item27 = new Item(); + item27.setName("Insect Spray"); + item27.setWeight(.5); + item27.setValue(3); + Gene gene27 = new Gene(item27); + + Item item28 = new Item(); + item28.setName("Hand sanitizer"); + item28.setWeight(.625); + item28.setValue(3); + Gene gene28 = new Gene(item28); + + Item item29 = new Item(); + item29.setName("Mirror"); + item29.setWeight(.5); + item29.setValue(3); + Gene gene29 = new Gene(item29); + + Item item30 = new Item(); + item30.setName("First Aid Kit"); + item30.setWeight(3); + item30.setValue(6); + Gene gene30 = new Gene(item30); + + list.add(gene1); + list.add(gene2); + list.add(gene3); + list.add(gene4); + list.add(gene5); + list.add(gene6); + list.add(gene7); + list.add(gene8); + list.add(gene9); + list.add(gene10); + list.add(gene11); + list.add(gene12); + list.add(gene13); + list.add(gene14); + list.add(gene15); + list.add(gene16); + list.add(gene17); + list.add(gene18); + list.add(gene19); + list.add(gene20); + list.add(gene21); + list.add(gene22); + list.add(gene23); + list.add(gene24); + list.add(gene25); + list.add(gene26); + list.add(gene27); + list.add(gene28); + list.add(gene29); + list.add(gene30); + + return list; + } + +} 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 new file mode 100644 index 0000000000000..a10a6fc12754a --- /dev/null +++ b/examples/src/main/java/org/apache/ignite/examples/ml/genetic/knapsack/KnapsackTerminateCriteria.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.examples.ml.genetic.knapsack; + +import java.util.List; + +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; +import org.apache.ignite.ml.genetic.utils.GAGridUtils; + +/** + * Represents the terminate condition for Knapsack Genetic algorithm + * + * Class terminates Genetic algorithm when once GA Grid has performed 30 generations. + */ +public class KnapsackTerminateCriteria implements ITerminateCriteria { + /** Ignite instance */ + private static Ignite ignite = null; + + /** Ignite logger */ + private IgniteLogger igniteLogger = null; + + /** + * @param ignite Ignite + */ + public KnapsackTerminateCriteria(Ignite ignite) { + this.ignite = ignite; + this.igniteLogger = this.ignite.log(); + } + + /** + * @param fittestChromosome Most fit chromosome at for the nth generation + * @param averageFitnessScore Average fitness score as of the nth generation + * @param currentGeneration Current generation + * @return Boolean value + */ + public boolean isTerminationConditionMet(Chromosome fittestChromosome, double averageFitnessScore, + int currentGeneration) { + boolean isTerminate = true; + + igniteLogger.info("##########################################################################################"); + igniteLogger.info("Generation: " + currentGeneration); + igniteLogger.info("Fittest is Chromosome Key: " + fittestChromosome); + igniteLogger.info("Total value is: " + fittestChromosome.getFitnessScore()); + igniteLogger.info("Total weight is: " + calculateTotalWeight(GAGridUtils.getGenesInOrderForChromosome(ignite, fittestChromosome))); + igniteLogger.info("Avg Chromosome Fitness: " + averageFitnessScore); + igniteLogger.info("Chromosome: " + fittestChromosome); + printItems(GAGridUtils.getGenesInOrderForChromosome(ignite, fittestChromosome)); + igniteLogger.info("##########################################################################################"); + + if (!(currentGeneration > 29)) + isTerminate = false; + + return isTerminate; + } + + /** + * @param genes List of Genes + * @return double value + */ + private double calculateTotalWeight(List genes) { + double totalWeight = 0; + for (Gene gene : genes) + totalWeight = totalWeight + ((Item)gene.getValue()).getWeight(); + + return totalWeight; + } + + /** + * Helper to print items in knapsack + * + * @param genes List of Genes + */ + private void printItems(List genes) { + for (Gene gene : genes) { + igniteLogger.info("------------------------------------------------------------------------------------------"); + igniteLogger.info("Name: " + ((Item)gene.getValue()).getName().toString()); + igniteLogger.info("Weight: " + ((Item)gene.getValue()).getWeight()); + igniteLogger.info("Value: " + ((Item)gene.getValue()).getValue()); + } + } +} From 1398572729688e6e31b5d19c540f86f2b5ffae6c Mon Sep 17 00:00:00 2001 From: Andrey Gura Date: Thu, 7 Jun 2018 19:04:46 +0300 Subject: [PATCH 0247/1463] IGNITE-8713 Spring Data dependencies upgraded --- parent/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/parent/pom.xml b/parent/pom.xml index 27ee390507d7d..17e8a405be787 100644 --- a/parent/pom.xml +++ b/parent/pom.xml @@ -112,8 +112,8 @@ 1.6.4 2.6.5 2.2.0 - 1.13.1.RELEASE - 4.3.7.RELEASE + 1.13.11.RELEASE + 4.3.7.RELEASE 4.1.7.RELEASE_1 1.1.1 8.0.23 From 25372068fffd3a39366d467367426db0a4ab3620 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Sat, 9 Jun 2018 18:15:56 +0300 Subject: [PATCH 0248/1463] IGNITE-8764: Fixed issue with Informatica on Windows (cherry picked from commit ec9e44e) --- .../odbc/odbc/OdbcRequestHandler.java | 14 ++++++++-- modules/platforms/cpp/odbc/src/connection.cpp | 26 ++++++++++++++----- 2 files changed, 31 insertions(+), 9 deletions(-) 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 82e93c33fb840..bf0601be52192 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 @@ -643,8 +643,18 @@ private static byte sqlTypeToBinary(int sqlType) { * @return Whether string matches pattern. */ private static boolean matches(String str, String ptrn) { - return str != null && (F.isEmpty(ptrn) || - str.toUpperCase().matches(ptrn.toUpperCase().replace("%", ".*").replace("_", "."))); + if (F.isEmpty(ptrn)) + return true; + + if (str == null) + return false; + + String pattern = ptrn.toUpperCase().replace("%", ".*").replace("_", "."); + + if (pattern.length() >= 2 && pattern.matches("['\"].*['\"]")) + pattern = pattern.substring(1, pattern.length() - 1); + + return str.toUpperCase().matches(pattern); } /** diff --git a/modules/platforms/cpp/odbc/src/connection.cpp b/modules/platforms/cpp/odbc/src/connection.cpp index b96753c2669b0..4a28bbf4dd5f1 100644 --- a/modules/platforms/cpp/odbc/src/connection.cpp +++ b/modules/platforms/cpp/odbc/src/connection.cpp @@ -448,6 +448,15 @@ namespace ignite break; } + case SQL_ATTR_AUTOCOMMIT: + { + SQLUINTEGER *val = reinterpret_cast(buf); + + *val = SQL_AUTOCOMMIT_ON; + + break; + } + default: { AddStatusRecord(SqlState::SHYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, @@ -467,13 +476,6 @@ namespace ignite SqlResult::Type Connection::InternalSetAttribute(int attr, void* value, SQLINTEGER valueLen) { - if (!value) - { - AddStatusRecord(SqlState::SHY009_INVALID_USE_OF_NULL_POINTER, "Value pointer is null."); - - return SqlResult::AI_ERROR; - } - switch (attr) { case SQL_ATTR_CONNECTION_DEAD: @@ -503,6 +505,16 @@ namespace ignite break; } + case SQL_ATTR_AUTOCOMMIT: + { + SQLUINTEGER val = static_cast(reinterpret_cast(value)); + + if (val != SQL_AUTOCOMMIT_ON) + return SqlResult::AI_ERROR; + + break; + } + default: { AddStatusRecord(SqlState::SHYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED, From bd930736639670c8fb3c71eeac79f5cd68eada3e Mon Sep 17 00:00:00 2001 From: Andrei Aleksandrov Date: Sat, 9 Jun 2018 18:31:25 +0300 Subject: [PATCH 0249/1463] IGNITE-7319: Cancelable future task for backup cleaner should be closed on CacheContinuousQueryManager stop Signed-off-by: Andrey Gura --- .../CacheContinuousQueryManager.java | 25 ++++++++++++++++++- ...dCacheContinuousQueryAbstractSelfTest.java | 24 ++++++++++++++++++ 2 files changed, 48 insertions(+), 1 deletion(-) 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 55c44b489d7a7..5e6af2f067d3c 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 @@ -62,6 +62,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.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.CI2; import org.apache.ignite.internal.util.typedef.F; @@ -124,6 +125,28 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter { /** Ordered topic prefix. */ private String topicPrefix; + /** Cancelable future task for backup cleaner */ + private GridTimeoutProcessor.CancelableTask cancelableTask; + + /** {@inheritDoc} */ + @Override protected void stop0(boolean cancel, boolean destroy) { + if (cancelableTask != null) { + cancelableTask.close(); + + cancelableTask = null; + } + } + + /** + * USED ONLY FOR TESTING. + * + * @return Internal cancelable future task for backup cleaner. + */ + /*@java.test.only*/ + protected GridTimeoutProcessor.CancelableTask getCancelableTask() { + return cancelableTask; + } + /** {@inheritDoc} */ @Override protected void start0() throws IgniteCheckedException { // Append cache name to the topic. @@ -140,7 +163,7 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter { } }); - cctx.time().schedule(new BackupCleaner(lsnrs, cctx.kernalContext()), BACKUP_ACK_FREQ, BACKUP_ACK_FREQ); + cancelableTask = cctx.time().schedule(new BackupCleaner(lsnrs, cctx.kernalContext()), BACKUP_ACK_FREQ, BACKUP_ACK_FREQ); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java index 8287a9b8d12f1..9d7cad40c7568 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java @@ -434,6 +434,30 @@ public void testTwoQueryListener() throws Exception { } } + /** + * @throws Exception If failed. + */ + public void testBackupCleanerTaskFinalize() throws Exception { + final String CACHE_NAME = "LOCAL_CACHE"; + + CacheConfiguration cCfg = new CacheConfiguration<>(); + + cCfg.setName(CACHE_NAME); + + cCfg.setCacheMode(cacheMode()); + + IgniteCache cache = grid(0).getOrCreateCache(cCfg); + + CacheContinuousQueryManager qm = grid(0).context().cache(). + internalCache(CACHE_NAME).context().continuousQueries(); + + assertNotNull(qm.getCancelableTask()); + + cache.destroy(); + + assertNull(qm.getCancelableTask()); + } + /** * @throws Exception If failed. */ From 94d67ffc180b9994fcef31fa5b181172e988f5a3 Mon Sep 17 00:00:00 2001 From: Sergey Kosarev Date: Wed, 13 Jun 2018 17:24:42 +0300 Subject: [PATCH 0250/1463] IGNITE-8736 Add transaction label to CU.txString() method output - Fixes #4152. Signed-off-by: Alexey Goncharuk --- .../ignite/internal/processors/cache/GridCacheUtils.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 4f349a4b5c385..2520f65e54311 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 @@ -827,7 +827,9 @@ public static String txString(@Nullable IgniteInternalTx tx) { ", rollbackOnly=" + tx.isRollbackOnly() + ", nodeId=" + tx.nodeId() + ", timeout=" + tx.timeout() + - ", duration=" + (U.currentTimeMillis() - tx.startTime()) + ']'; + ", duration=" + (U.currentTimeMillis() - tx.startTime()) + + (tx instanceof GridNearTxLocal ? ", label=" + ((GridNearTxLocal)tx).label() : "") + + ']'; } /** From ff6d9d03d861757d90d13a252dd93b0d4f5ba16b Mon Sep 17 00:00:00 2001 From: Alexey Kuznetsov Date: Thu, 14 Jun 2018 16:20:19 +0700 Subject: [PATCH 0251/1463] IGNITE-8722 REST: Fixed BinaryObject serialization to JSON. (cherry picked from commit e539a0c) --- .../rest/JettyRestProcessorAbstractSelfTest.java | 14 ++++++++++++++ .../http/jetty/GridJettyObjectMapper.java | 4 ++-- 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java index 96391e95a16b8..d3b328ae2e0f2 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java @@ -483,6 +483,7 @@ public void testGetBinaryObjects() throws Exception { JsonNode json = assertResponseSucceeded(ret, false); assertEquals(ref1.name, json.get("name").asText()); + assertEquals(ref1.ref.toString(), json.get("ref").toString()); ref2.ref(ref1); @@ -2760,6 +2761,19 @@ public CircularRef ref() { public void ref(CircularRef ref) { this.ref = ref; } + + /** {@inheritDoc} */ + @Override public String toString() { + SB sb = new SB(); + + sb.a('{') + .a('"').a("id").a('"').a(':').a(id).a(',') + .a('"').a("name").a('"').a(':').a('"').a(name).a('"').a(',') + .a('"').a("ref").a('"').a(':').a(ref) + .a('}'); + + return sb.toString(); + } } /** diff --git a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyObjectMapper.java b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyObjectMapper.java index d8b79cfd4cfde..92ee5c0df02b7 100644 --- a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyObjectMapper.java +++ b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyObjectMapper.java @@ -246,8 +246,8 @@ private void writeException(Throwable e, JsonGenerator gen) throws IOException { if (ref.hasCircularReferences()) throw ser.mappingException("Failed convert to JSON object for circular references"); } - else - gen.writeObjectField(name, val); + + gen.writeObjectField(name, val); } gen.writeEndObject(); From 39d4b4b491b592432e095e7e3022ed2ba53b0a1d Mon Sep 17 00:00:00 2001 From: Andrey Gura Date: Sat, 9 Jun 2018 16:37:49 +0300 Subject: [PATCH 0252/1463] IGNITE-8751 Failure handler accordingly to segmentation policy should be invoked on node segmentation instead of configured failure handler --- .../ignite/spi/discovery/tcp/ServerImpl.java | 11 ++- .../TcpDiscoverySegmentationPolicyTest.java | 91 +++++++++++++++++++ .../IgniteSpiDiscoverySelfTestSuite.java | 3 + 3 files changed, 101 insertions(+), 4 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySegmentationPolicyTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 35e945208eb65..079058b799287 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -1687,7 +1687,10 @@ private void clearNodeAddedMessage(TcpDiscoveryAbstractMessage msg) { /** {@inheritDoc} */ @Override public void brakeConnection() { - throw new UnsupportedOperationException(); + Socket sock = msgWorker.sock; + + if (sock != null) + U.closeQuiet(sock); } /** {@inheritDoc} */ @@ -2621,7 +2624,7 @@ void addMessage(TcpDiscoveryAbstractMessage msg) { super.body(); } catch (InterruptedException e) { - if (!spi.isNodeStopping0()) + if (!spi.isNodeStopping0() && spiStateCopy() != DISCONNECTING) err = e; throw e; @@ -2657,7 +2660,7 @@ void addMessage(TcpDiscoveryAbstractMessage msg) { throw e; } finally { - if (err == null && !spi.isNodeStopping0()) + if (err == null && !spi.isNodeStopping0() && spiStateCopy() != DISCONNECTING) err = new IllegalStateException("Thread " + getName() + " is terminated unexpectedly."); FailureProcessor failure = ((IgniteEx)spi.ignite()).context().failure(); @@ -5682,7 +5685,7 @@ private class TcpServer extends IgniteSpiThread { throw t; } finally { - if (err == null && !spi.isNodeStopping0()) + if (err == null && !spi.isNodeStopping0() && spiStateCopy() != DISCONNECTING) err = new IllegalStateException("Thread " + getName() + " is terminated unexpectedly."); FailureProcessor failure = ((IgniteEx)spi.ignite()).context().failure(); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySegmentationPolicyTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySegmentationPolicyTest.java new file mode 100644 index 0000000000000..df76afc2317e4 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySegmentationPolicyTest.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.spi.discovery.tcp; + +import java.util.Arrays; +import java.util.Collection; +import org.apache.ignite.Ignite; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.failure.FailureContext; +import org.apache.ignite.failure.FailureHandler; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests for segmentation policy and failure handling in {@link TcpDiscoverySpi}. + */ +public class TcpDiscoverySegmentationPolicyTest extends GridCommonAbstractTest { + /** Nodes count. */ + private static final int NODES_CNT = 3; + + /** Default failure handler invoked. */ + private static volatile boolean dfltFailureHndInvoked; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + if (igniteInstanceName.endsWith("2")) + cfg.setFailureHandler(new TestFailureHandler()); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + public void testStopOnSegmentation() throws Exception { + startGrids(NODES_CNT); + + IgniteEx ignite1 = grid(1); + IgniteEx ignite2 = grid(2); + + ((TcpDiscoverySpi)ignite1.configuration().getDiscoverySpi()).brakeConnection(); + ((TcpDiscoverySpi)ignite2.configuration().getDiscoverySpi()).brakeConnection(); + + waitForTopology(2); + + assertFalse(dfltFailureHndInvoked); + + Collection nodes = ignite1.cluster().forServers().nodes(); + + assertEquals(2, nodes.size()); + assertTrue(nodes.containsAll(Arrays.asList(((IgniteKernal)ignite(0)).localNode(), ((IgniteKernal)ignite(1)).localNode()))); + + System.out.println(); + } + + /** + * Test failure handler. + */ + private static class TestFailureHandler implements FailureHandler { + /** {@inheritDoc} */ + @Override public boolean onFailure(Ignite ignite, FailureContext failureCtx) { + dfltFailureHndInvoked = true; + + return true; + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java index 1f82316e5a99b..ef582a5df7565 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java @@ -35,6 +35,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoveryNodeConfigConsistentIdSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoveryNodeConsistentIdSelfTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoveryRestartTest; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySegmentationPolicyTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySelfTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySnapshotHistoryTest; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpiConfigSelfTest; @@ -96,6 +97,8 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(TcpDiscoveryRestartTest.class)); suite.addTest(new TestSuite(TcpDiscoveryMultiThreadedTest.class)); + suite.addTest(new TestSuite(TcpDiscoverySegmentationPolicyTest.class)); + suite.addTest(new TestSuite(TcpDiscoveryNodeAttributesUpdateOnReconnectTest.class)); suite.addTest(new TestSuite(AuthenticationRestartTest.class)); From 7144617c8fc79afc9a6fd39b3bf579ce76fce4e0 Mon Sep 17 00:00:00 2001 From: Andrey Gura Date: Wed, 13 Jun 2018 17:30:18 +0300 Subject: [PATCH 0253/1463] IGNITE-8781 GridNioServer accepter threads should have different names --- .../ignite/internal/util/nio/GridNioServer.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java index da3438e069414..03870dbb3e0d9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java @@ -338,7 +338,16 @@ private GridNioServer( // This method will throw exception if address already in use. Selector acceptSelector = createSelector(locAddr); - acceptThread = new IgniteThread(new GridNioAcceptWorker(igniteInstanceName, "nio-acceptor", log, acceptSelector)); + String threadName; + + if (srvName == null) + threadName = "nio-acceptor"; + else + threadName = "nio-acceptor-" + srvName; + + GridNioAcceptWorker w = new GridNioAcceptWorker(igniteInstanceName, threadName, log, acceptSelector); + + acceptThread = new IgniteThread(w); } else { locAddr = null; From c12f34709e7c8dd817779d34885b462da78ee80a Mon Sep 17 00:00:00 2001 From: Aleksey Plekhanov Date: Thu, 14 Jun 2018 20:41:31 +0300 Subject: [PATCH 0254/1463] IGNITE-8763 java.nio.file.AccessDeniedException is not handled with default failure handler Signed-off-by: Andrey Gura --- .../GridCacheDatabaseSharedManager.java | 6 ++- .../IgnitePdsCorruptedStoreTest.java | 50 ++++++++++++++++++- 2 files changed, 53 insertions(+), 3 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 6064cfe2775cd..3ee8f28d6d3de 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 @@ -146,6 +146,7 @@ import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.S; @@ -825,8 +826,9 @@ private void unRegistrateMetricsMBean() { notifyMetastorageReadyForReadWrite(); } - catch (StorageException | PersistentStorageIOException e) { - cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e)); + catch (IgniteCheckedException e) { + if (X.hasCause(e, StorageException.class, PersistentStorageIOException.class, IOException.class)) + cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e)); throw e; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedStoreTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedStoreTest.java index d4a135dccca5d..ff95f97a1fc78 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedStoreTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCorruptedStoreTest.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.cache.persistence; import java.io.File; +import java.io.IOException; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; @@ -33,14 +34,18 @@ import org.apache.ignite.failure.FailureContext; import org.apache.ignite.failure.FailureHandler; import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.pagemem.PageIdUtils; import org.apache.ignite.internal.pagemem.wal.StorageException; import org.apache.ignite.internal.processors.cache.IgniteInternalCache; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; +import org.apache.ignite.internal.processors.cache.persistence.file.PersistentStorageIOException; import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage; import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx; 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.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -303,13 +308,56 @@ private void corruptTreeRoot(IgniteEx ignite, PageMemoryEx pageMem, int grpId, i } } + /** + * Test node invalidation when meta store is read only. + */ + public void testReadOnlyMetaStore() throws Exception { + IgniteEx ignite0 = startGrid(0); + + ignite0.cluster().active(true); + + IgniteInternalCache cache = ignite0.cachex(CACHE_NAME1); + + cache.put(1, 1); + + ignite0.cluster().active(false); + + FilePageStoreManager storeMgr = ((FilePageStoreManager)ignite0.context().cache().context().pageStore()); + + File workDir = storeMgr.workDir(); + File metaStoreDir = new File(workDir, MetaStorage.METASTORAGE_CACHE_NAME.toLowerCase()); + File metaStoreFile = new File(metaStoreDir, String.format(FilePageStoreManager.PART_FILE_TEMPLATE, 0)); + + metaStoreFile.setWritable(false); + + try { + IgniteInternalFuture fut = GridTestUtils.runAsync(new Runnable() { + @Override public void run() { + try { + ignite0.cluster().active(true); + } + catch (Exception ignore) { + // No-op. + } + } + }); + + waitFailure(IOException.class); + + fut.cancel(); + } + finally { + metaStoreFile.setWritable(true); + } + } + /** * @param expError Expected error. */ private void waitFailure(Class expError) throws IgniteInterruptedCheckedException { assertTrue(GridTestUtils.waitForCondition(() -> failureHnd.failure(), 5_000L)); - assertTrue(expError.isInstance(failureHnd.error())); + assertTrue(X.hasCause(failureHnd.error(), expError)); } /** From 87686e83d5ee8b16f43c7fe3d4d6e8262f3344fb Mon Sep 17 00:00:00 2001 From: Sergey Chugunov Date: Fri, 15 Jun 2018 11:31:33 +0300 Subject: [PATCH 0255/1463] IGNITE-8657 Simultaneous start of multiple clients may lead to client start hang when exchange history size is too short - Fixes #4102. Signed-off-by: Alexey Goncharuk (cherry-picked from commit #2b928c702b2d6a7c1de8d1b3b0a1e0e65e653f21) --- .../apache/ignite/internal/IgniteKernal.java | 8 +- .../IgniteNeedReconnectException.java | 8 +- .../GridCachePartitionExchangeManager.java | 30 ++- .../GridDhtPartitionsExchangeFuture.java | 40 +++- .../IgniteCacheClientReconnectTest.java | 178 +++++++++++++++++- 5 files changed, 241 insertions(+), 23 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 90edfe337b6e5..112eb8548b493 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 @@ -1104,7 +1104,13 @@ public void start( comp.onKernalStart(active); } catch (IgniteNeedReconnectException e) { - assert ctx.discovery().reconnectSupported(); + ClusterNode locNode = ctx.discovery().localNode(); + + assert CU.clientNode(locNode); + + if (!locNode.isClient()) + throw new IgniteCheckedException("Client node in forceServerMode " + + "is not allowed to reconnect to the cluster and will be stopped."); if (log.isDebugEnabled()) log.debug("Failed to start node components on node start, will wait for reconnect: " + e); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNeedReconnectException.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNeedReconnectException.java index f3849500d3561..c26f4da03107a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNeedReconnectException.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNeedReconnectException.java @@ -29,12 +29,10 @@ public class IgniteNeedReconnectException extends IgniteCheckedException { private static final long serialVersionUID = 0L; /** - * @param locNode Local node. + * @param node Node that should reconnect. * @param cause Cause. */ - public IgniteNeedReconnectException(ClusterNode locNode, @Nullable Throwable cause) { - super("Local node need try to reconnect [locNodeId=" + locNode.id() + ']', cause); - - assert locNode.isClient() : locNode; + public IgniteNeedReconnectException(ClusterNode node, @Nullable Throwable cause) { + super("Node need try to reconnect [nodeId=" + node.id() + ']', cause); } } 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 3a990892bfe8d..38ddaf661de92 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 @@ -1558,6 +1558,20 @@ else if (!grp.isLocal()) if (log.isDebugEnabled()) log.debug("Notifying exchange future about single message: " + exchFut); + if (msg.client() && !exchFut.isDone()) { + if (exchFut.initialVersion().compareTo(readyAffinityVersion()) <= 0) { + U.warn(log, "Client node tries to connect but its exchange " + + "info is cleaned up from exchange history." + + " Consider increasing 'IGNITE_EXCHANGE_HISTORY_SIZE' property " + + "or start clients in smaller batches." + ); + + exchFut.forceClientReconnect(node, msg); + + return; + } + } + exchFut.onReceiveSingleMessage(node, msg); } } @@ -2648,14 +2662,18 @@ else if (r != null) { throw e; } catch (IgniteClientDisconnectedCheckedException | IgniteNeedReconnectException e) { - assert cctx.discovery().reconnectSupported(); - - U.warn(log,"Local node failed to complete partition map exchange due to " + - "network issues, will try to reconnect to cluster", e); + if (cctx.discovery().reconnectSupported()) { + U.warn(log, "Local node failed to complete partition map exchange due to " + + "exception, will try to reconnect to cluster: " + e.getMessage(), e); - cctx.discovery().reconnect(); + cctx.discovery().reconnect(); - reconnectNeeded = true; + reconnectNeeded = true; + } + else + U.warn(log, "Local node received IgniteClientDisconnectedCheckedException or " + + " IgniteNeedReconnectException exception but doesn't support reconnect, stopping node: " + + e.getMessage(), e); return; } 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 0c51835fbfd30..3c85aca08f359 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 @@ -2031,6 +2031,31 @@ private void processMergedMessage(final ClusterNode node, final GridDhtPartition finishExchangeOnCoordinator(null); } + /** + * Method is called on coordinator in situation when initial ExchangeFuture created on client join event was preempted + * from exchange history because of IGNITE_EXCHANGE_HISTORY_SIZE property. + * + * @param node Client node that should try to reconnect to the cluster. + * @param msg Single message received from the client which didn't find original ExchangeFuture. + */ + public void forceClientReconnect(ClusterNode node, GridDhtPartitionsSingleMessage msg) { + Exception e = new IgniteNeedReconnectException(node, null); + + changeGlobalStateExceptions.put(node.id(), e); + + onDone(null, e); + + GridDhtPartitionsFullMessage fullMsg = createPartitionsMessage(true, false); + + fullMsg.setErrorsMap(changeGlobalStateExceptions); + + FinishState finishState0 = new FinishState(cctx.localNodeId(), + initialVersion(), + fullMsg); + + sendAllPartitionsToNode(finishState0, msg, node.id()); + } + /** * Processing of received single message. Actual processing in future may be delayed if init method was not * completed, see {@link #initDone()} @@ -3129,6 +3154,16 @@ private void processFullMessage(boolean checkCrd, ClusterNode node, GridDhtParti return; } else { + if (!F.isEmpty(msg.getErrorsMap())) { + Exception e = msg.getErrorsMap().get(cctx.localNodeId()); + + assert e != null : msg.getErrorsMap(); + + onDone(e); + + return; + } + AffinityTopologyVersion resVer = msg.resultTopologyVersion() != null ? msg.resultTopologyVersion() : initialVersion(); if (log.isInfoEnabled()) { @@ -3751,8 +3786,9 @@ private void onBecomeCoordinator(InitNewCoordinatorFuture newCrdFut) { * @return {@code True} if local node should try reconnect in case of error. */ public boolean reconnectOnError(Throwable e) { - return X.hasCause(e, IOException.class, IgniteClientDisconnectedCheckedException.class) && - cctx.discovery().reconnectSupported(); + return (e instanceof IgniteNeedReconnectException + || X.hasCause(e, IOException.class, IgniteClientDisconnectedCheckedException.class)) + && cctx.discovery().reconnectSupported(); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientReconnectTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientReconnectTest.java index ced1a7d8a78d7..4beb31a9d92e6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientReconnectTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientReconnectTest.java @@ -24,9 +24,17 @@ 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.IgniteSystemProperties; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +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.util.typedef.G; 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; @@ -48,15 +56,24 @@ public class IgniteCacheClientReconnectTest extends GridCommonAbstractTest { /** */ private static final int SRV_CNT = 3; + /** */ + private static final int CLIENTS_CNT = 3; + /** */ private static final int CACHES = 10; + /** */ + private static final int PARTITIONS_CNT = 32; + /** */ private static final long TEST_TIME = 60_000; /** */ private boolean client; + /** */ + private boolean forceServerMode; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); @@ -69,19 +86,22 @@ public class IgniteCacheClientReconnectTest extends GridCommonAbstractTest { CacheConfiguration[] ccfgs = new CacheConfiguration[CACHES]; for (int i = 0; i < CACHES; i++) { - CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME); + CacheConfiguration ccfg = new CacheConfiguration(); ccfg.setCacheMode(PARTITIONED); ccfg.setAtomicityMode(TRANSACTIONAL); ccfg.setBackups(1); ccfg.setName("cache-" + i); ccfg.setWriteSynchronizationMode(FULL_SYNC); + ccfg.setAffinity(new RendezvousAffinityFunction(PARTITIONS_CNT, null)); ccfgs[i] = ccfg; } cfg.setCacheConfiguration(ccfgs); } + else + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(forceServerMode); cfg.setClientMode(client); @@ -89,26 +109,166 @@ public class IgniteCacheClientReconnectTest extends GridCommonAbstractTest { } /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - startGrids(SRV_CNT); + @Override protected long getTestTimeout() { + return TEST_TIME + 60_000; } /** {@inheritDoc} */ - @Override protected void afterTestsStopped() throws Exception { - super.afterTestsStopped(); - + @Override protected void afterTest() throws Exception { stopAllGrids(); } - /** {@inheritDoc} */ - @Override protected long getTestTimeout() { - return TEST_TIME + 60_000; + /** + * If setting IGNITE_EXCHANGE_HISTORY_SIZE is set to small value + * it is possible that bunch of clients simultaneous start (amount > IGNITE_EXCHANGE_HISTORY_SIZE) + * may result in ExchangeFuture for some client being flushed from exchange history. + * + * In that case client attempts to reconnect until success. + * + * After that it should get correct information about topology version and affinity distribution. + * + * @throws Exception If failed + */ + public void testClientReconnectOnExchangeHistoryExhaustion() throws Exception { + System.setProperty(IgniteSystemProperties.IGNITE_EXCHANGE_HISTORY_SIZE, "1"); + + try { + startGrids(SRV_CNT); + + client = true; + + startGridsMultiThreaded(SRV_CNT, CLIENTS_CNT); + + waitForTopology(SRV_CNT + CLIENTS_CNT); + + verifyPartitionToNodeMappings(); + + verifyAffinityTopologyVersions(); + + verifyCacheOperationsOnClients(); + } + finally { + System.clearProperty(IgniteSystemProperties.IGNITE_EXCHANGE_HISTORY_SIZE); + } + } + + /** + * Verifies that in case of exchange history exhaustion + * (refer to javadoc at {@link #testClientReconnectOnExchangeHistoryExhaustion()} for more info about it) + * clients with forceServerMode=true flag don't try to reconnect to the cluster and stop. + * + * @throws Exception If failed + */ + public void testClientInForceServerModeStopsOnExchangeHistoryExhaustion() throws Exception { + System.setProperty(IgniteSystemProperties.IGNITE_EXCHANGE_HISTORY_SIZE, "1"); + + try { + startGrids(SRV_CNT); + + client = true; + + forceServerMode = true; + + int clientNodes = 10; + + try { + startGridsMultiThreaded(SRV_CNT, clientNodes); + } + catch (IgniteCheckedException e) { + //Ignored: it is expected to get exception here + } + + awaitPartitionMapExchange(); + + int topSize = G.allGrids().size(); + + assertTrue("Actual size: " + topSize, topSize < SRV_CNT + clientNodes); + + } + finally { + System.clearProperty(IgniteSystemProperties.IGNITE_EXCHANGE_HISTORY_SIZE); + } + } + + /** + * Verifies basic cache operations from all clients. + */ + private void verifyCacheOperationsOnClients() { + for (int i = SRV_CNT; i < SRV_CNT + CLIENTS_CNT; i++) { + IgniteEx cl = grid(i); + + if (!forceServerMode) + assertTrue(cl.localNode().isClient()); + + for (int j = 0; j < CACHES; j++) { + IgniteCache cache = cl.cache("cache-" + j); + + String keyPrefix = "cl-" + i + "_key-"; + + for (int k = 0; k < 1_000; k++) + cache.put(keyPrefix + k, "val-" + k); + + for (int k = 999; k >= 0; k--) + assertEquals("val-" + k, cache.get(keyPrefix + k)); + } + } + } + + /** + * Verifies that affinity mappings are the same on clients and servers. + */ + private void verifyPartitionToNodeMappings() { + IgniteEx refSrv = grid(0); + String cacheName; + + for (int i = 0; i < CACHES; i++) { + cacheName = "cache-" + i; + + Affinity refAffinity = refSrv.affinity(cacheName); + + for (int j = 0; j < PARTITIONS_CNT; j++) { + ClusterNode refAffNode = refAffinity.mapPartitionToNode(j); + + assertNotNull("Affinity node for " + j + " partition is null", refAffNode); + + for (int k = SRV_CNT; k < SRV_CNT + CLIENTS_CNT; k++) { + ClusterNode clAffNode = grid(k).affinity(cacheName).mapPartitionToNode(j); + + assertNotNull("Affinity node for " + k + " client and " + j + " partition is null", clAffNode); + + assertEquals("Affinity node for " + + k + + " client and " + + j + + " partition is different on client", + refAffNode.id(), + clAffNode.id()); + } + } + } + } + + /** + * Verifies {@link AffinityTopologyVersion}s: one obtained from coordinator and all from each client node. + */ + private void verifyAffinityTopologyVersions() { + IgniteEx srv = grid(0); + + AffinityTopologyVersion srvTopVer = srv.context().discovery().topologyVersionEx(); + + for (int i = SRV_CNT; i < SRV_CNT + CLIENTS_CNT; i++) { + AffinityTopologyVersion clntTopVer = grid(i).context().discovery().topologyVersionEx(); + + assertTrue(clntTopVer.equals(srvTopVer)); + } } /** * @throws Exception If failed. */ public void testClientReconnect() throws Exception { + startGrids(SRV_CNT); + client = true; final AtomicBoolean stop = new AtomicBoolean(false); From 54b750f347ee56b28b53414834e55a7483569d43 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 15 Jun 2018 12:48:24 +0300 Subject: [PATCH 0256/1463] IGNITE-8610 Fixed checkpoint record search for WAL delta rebalancing - Fixes #4090. Signed-off-by: Alexey Goncharuk (cherry picked from commit 10aa02a) --- .../dht/GridDhtLocalPartition.java | 23 +- .../distributed/dht/GridDhtLockFuture.java | 3 +- .../GridDhtPartitionDemandMessage.java | 2 +- .../GridDhtPartitionsExchangeFuture.java | 21 +- .../dht/preloader/GridDhtPreloader.java | 8 +- ...IgniteDhtPartitionHistorySuppliersMap.java | 13 +- .../GridCacheDatabaseSharedManager.java | 1389 +++++------------ .../persistence/GridCacheOffheapManager.java | 5 +- .../IgniteCacheDatabaseSharedManager.java | 26 +- .../checkpoint/CheckpointEntry.java | 366 +++++ .../checkpoint/CheckpointEntryType.java | 29 + .../checkpoint/CheckpointHistory.java | 382 +++++ .../wal/FileWriteAheadLogManager.java | 12 +- .../internal/visor/misc/VisorWalTask.java | 4 +- ...sAtomicCacheHistoricalRebalancingTest.java | 19 + ...IgnitePdsCacheRebalancingAbstractTest.java | 328 ++-- ...tePdsTxCacheHistoricalRebalancingTest.java | 39 - ...alModeChangeDuringRebalancingSelfTest.java | 4 +- .../db/IgnitePdsUnusedWalSegmentsTest.java | 20 +- .../db/wal/IgniteWalRebalanceTest.java | 271 +++- .../wal/WalRecoveryTxLogicalRecordsTest.java | 3 +- .../IgnitePdsWithIndexingCoreTestSuite.java | 2 - 22 files changed, 1696 insertions(+), 1273 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointEntry.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointEntryType.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.java delete mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsTxCacheHistoricalRebalancingTest.java 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 c54a6cb238b52..81249b613b246 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 @@ -32,6 +32,7 @@ import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.pagemem.wal.record.delta.PartitionMetaStateRecord; @@ -812,11 +813,23 @@ public void destroy() { * Awaits completion of partition destroy process in case of {@code EVICTED} partition state. */ public void awaitDestroy() { - try { - if (state() == EVICTED) - rent.get(); - } catch (IgniteCheckedException e) { - log.error("Unable to await partition destroy " + this, e); + if (state() != EVICTED) + return; + + final long timeout = 10_000; + + for (;;) { + try { + rent.get(timeout); + + break; + } + catch (IgniteFutureTimeoutCheckedException ignored) { + U.warn(log, "Failed to await partition destroy within timeout " + this); + } + catch (IgniteCheckedException e) { + throw new IgniteException("Failed to await partition destroy " + this, e); + } } } 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 2869bb6441652..529d96575e827 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 @@ -1309,7 +1309,8 @@ void onResult(GridDhtLockResponse res) { info.version(), info.ttl(), info.expireTime(), - true, topVer, + true, + topVer, replicate ? DR_PRELOAD : DR_NONE, false)) { if (rec && !entry.isInternal()) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java index 0d2cc4bebf5fe..dc6162bc321f2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java @@ -136,7 +136,7 @@ public GridDhtPartitionDemandMessage withNewPartitionsMap(@NotNull IgniteDhtDema /** * @return Partition. */ - IgniteDhtDemandedPartitionsMap partitions() { + public IgniteDhtDemandedPartitionsMap partitions() { return parts; } 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 3c85aca08f359..c00cd0c510704 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 @@ -428,12 +428,15 @@ public void affinityChangeMessage(CacheAffinityChangeMessage affChangeMsg) { } /** + * Retreives the node which has WAL history since {@code cntrSince}. + * * @param grpId Cache group ID. * @param partId Partition ID. + * @param cntrSince Partition update counter since history supplying is requested. * @return ID of history supplier node or null if it doesn't exist. */ - @Nullable public UUID partitionHistorySupplier(int grpId, int partId) { - return partHistSuppliers.getSupplier(grpId, partId); + @Nullable public UUID partitionHistorySupplier(int grpId, int partId, long cntrSince) { + return partHistSuppliers.getSupplier(grpId, partId, cntrSince); } /** @@ -1167,8 +1170,9 @@ private void distributedExchange() throws IgniteCheckedException { In case of persistent store is enabled we first restore partitions presented on disk. We need to guarantee that there are no partition state changes logged to WAL before this callback to make sure that we correctly restored last actual states. */ - cctx.database().beforeExchange(this); + boolean restored = cctx.database().beforeExchange(this); + // Pre-create missing partitions using current affinity. if (!exchCtx.mergeExchanges()) { for (CacheGroupContext grp : cctx.cache().cacheGroups()) { if (grp.isLocal() || cacheGroupStopping(grp.groupId())) @@ -1180,6 +1184,10 @@ private void distributedExchange() throws IgniteCheckedException { } } + // After all partitions have been restored and pre-created it's safe to make first checkpoint. + if (restored) + cctx.database().onStateRestored(); + changeWalModeIfNeeded(); if (crd.isLocal()) { @@ -1745,6 +1753,7 @@ public void finishMerged() { } cctx.database().releaseHistoryForExchange(); + cctx.database().rebuildIndexesIfNeeded(this); if (err == null) { @@ -2410,8 +2419,6 @@ else if (cntr == maxCntr.cnt) maxCntr.nodes.add(cctx.localNodeId()); } - int entryLeft = maxCntrs.size(); - Map> partHistReserved0 = partHistReserved; Map localReserved = partHistReserved0 != null ? partHistReserved0.get(top.groupId()) : null; @@ -2434,7 +2441,7 @@ else if (cntr == maxCntr.cnt) Long localCntr = localReserved.get(p); if (localCntr != null && localCntr <= minCntr && maxCntrObj.nodes.contains(cctx.localNodeId())) { - partHistSuppliers.put(cctx.localNodeId(), top.groupId(), p, minCntr); + partHistSuppliers.put(cctx.localNodeId(), top.groupId(), p, localCntr); haveHistory.add(p); @@ -2446,7 +2453,7 @@ else if (cntr == maxCntr.cnt) Long histCntr = e0.getValue().partitionHistoryCounters(top.groupId()).get(p); if (histCntr != null && histCntr <= minCntr && maxCntrObj.nodes.contains(e0.getKey())) { - partHistSuppliers.put(e0.getKey(), top.groupId(), p, minCntr); + partHistSuppliers.put(e0.getKey(), top.groupId(), p, histCntr); haveHistory.add(p); 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 700f0cf98b7a1..77f48661d588b 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 @@ -39,6 +39,7 @@ 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.dht.atomic.GridNearAtomicAbstractUpdateRequest; +import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -223,6 +224,8 @@ private IgniteCheckedException stopError() { // If partition was destroyed recreate it. if (part.state() == EVICTED) { + part.awaitDestroy(); + part = top.localPartition(p, topVer, true); } @@ -231,7 +234,7 @@ private IgniteCheckedException stopError() { ClusterNode histSupplier = null; if (grp.persistenceEnabled() && exchFut != null) { - UUID nodeId = exchFut.partitionHistorySupplier(grp.groupId(), p); + UUID nodeId = exchFut.partitionHistorySupplier(grp.groupId(), p, part.initialUpdateCounter()); if (nodeId != null) histSupplier = ctx.discovery().node(nodeId); @@ -288,6 +291,9 @@ private IgniteCheckedException stopError() { } } + if (!assignments.isEmpty()) + ctx.database().lastCheckpointInapplicableForWalRebalance(grp.groupId()); + return assignments; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgniteDhtPartitionHistorySuppliersMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgniteDhtPartitionHistorySuppliersMap.java index f8da6a80bf972..6755f287b6974 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgniteDhtPartitionHistorySuppliersMap.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgniteDhtPartitionHistorySuppliersMap.java @@ -47,17 +47,22 @@ public static IgniteDhtPartitionHistorySuppliersMap empty() { } /** - * @param grpId Cache group ID. + * @param grpId Group ID. * @param partId Partition ID. + * @param cntrSince Partition update counter since history supplying is requested. * @return Supplier UUID. */ - @Nullable public synchronized UUID getSupplier(int grpId, int partId) { + @Nullable public synchronized UUID getSupplier(int grpId, int partId, long cntrSince) { if (map == null) return null; for (Map.Entry, Long>> e : map.entrySet()) { - if (e.getValue().containsKey(new T2<>(grpId, partId))) - return e.getKey(); + UUID supplierNode = e.getKey(); + + Long historyCounter = e.getValue().get(new T2<>(grpId, partId)); + + if (historyCounter != null && historyCounter <= cntrSince) + return supplierNode; } return null; 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 3ee8f28d6d3de..7d30181c9bf46 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 @@ -18,10 +18,8 @@ package org.apache.ignite.internal.processors.cache.persistence; import java.io.File; -import java.io.FileFilter; import java.io.IOException; import java.io.RandomAccessFile; -import java.lang.ref.SoftReference; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.channels.FileChannel; @@ -42,20 +40,16 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.NavigableMap; import java.util.Set; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.LongAdder; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.regex.Matcher; @@ -118,6 +112,9 @@ 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.persistence.checkpoint.CheckpointEntry; +import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointEntryType; +import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointHistory; 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.FilePageStore; @@ -165,7 +162,6 @@ import org.jsr166.ConcurrentLinkedHashMap; import static java.nio.file.StandardOpenOption.READ; -import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_SKIP_CRC; import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD; import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR; @@ -222,64 +218,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan /** 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 patter. */ - private static final Pattern NODE_STARTED_FILE_NAME_PATTERN = Pattern.compile("(\\d+)-node-started\\.bin"); - /** Node started file suffix. */ public static final String NODE_STARTED_FILE_NAME_SUFFIX = "-node-started.bin"; - /** */ - private static final FileFilter CP_FILE_FILTER = new FileFilter() { - @Override public boolean accept(File f) { - return CP_FILE_NAME_PATTERN.matcher(f.getName()).matches(); - } - }; - - /** */ - private static final FileFilter NODE_STARTED_FILE_FILTER = new FileFilter() { - @Override public boolean accept(File f) { - return f.getName().endsWith(NODE_STARTED_FILE_NAME_SUFFIX); - } - }; - - /** */ - private static final Comparator ASC_PART_COMPARATOR = new Comparator() { - @Override public int compare(GridDhtLocalPartition a, GridDhtLocalPartition b) { - return Integer.compare(a.id(), b.id()); - } - }; - - /** */ - private static final Comparator CP_TS_COMPARATOR = new Comparator() { - /** {@inheritDoc} */ - @Override public int compare(File o1, File o2) { - Matcher m1 = CP_FILE_NAME_PATTERN.matcher(o1.getName()); - Matcher m2 = CP_FILE_NAME_PATTERN.matcher(o2.getName()); - - boolean s1 = m1.matches(); - boolean s2 = m2.matches(); - - assert s1 : "Failed to match CP file: " + o1.getAbsolutePath(); - assert s2 : "Failed to match CP file: " + o2.getAbsolutePath(); - - long ts1 = Long.parseLong(m1.group(1)); - long ts2 = Long.parseLong(m2.group(1)); - - int res = Long.compare(ts1, ts2); - - if (res == 0) { - CheckpointEntryType type1 = CheckpointEntryType.valueOf(m1.group(3)); - CheckpointEntryType type2 = CheckpointEntryType.valueOf(m2.group(3)); - - assert type1 != type2 : "o1=" + o1.getAbsolutePath() + ", o2=" + o2.getAbsolutePath(); - - res = type1 == CheckpointEntryType.START ? -1 : 1; - } - - return res; - } - }; - /** */ private static final String MBEAN_NAME = "DataStorageMetrics"; @@ -289,12 +230,15 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan /** WAL marker prefix for meta store. */ private static final String WAL_KEY_PREFIX = "grp-wal-"; - /** WAL marker prefix for meta store. */ + /** Prefix for meta store records which means that WAL was disabled globally for some group. */ private static final String WAL_GLOBAL_KEY_PREFIX = WAL_KEY_PREFIX + "disabled-"; - /** WAL marker prefix for meta store. */ + /** Prefix for meta store records which means that WAL was disabled locally for some group. */ private static final String WAL_LOCAL_KEY_PREFIX = WAL_KEY_PREFIX + "local-disabled-"; + /** Prefix for meta store records which means that checkpoint entry for some group is not applicable for WAL rebalance. */ + private static final String CHECKPOINT_INAPPLICABLE_FOR_REBALANCE = "cp-wal-rebalance-inapplicable-"; + /** WAL marker predicate for meta store. */ private static final IgnitePredicate WAL_KEY_PREFIX_PRED = new IgnitePredicate() { @Override public boolean apply(String key) { @@ -320,6 +264,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan /** */ private long checkpointFreq; + /** */ + private CheckpointHistory cpHistory; + /** */ private FilePageStoreManager storeMgr; @@ -335,9 +282,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan /** */ private final Collection lsnrs = new CopyOnWriteArrayList<>(); - /** Checkpoint history. */ - private final CheckpointHistory checkpointHist = new CheckpointHistory(); - /** */ private boolean stopping; @@ -360,7 +304,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan private final long lockWaitTime; /** */ - private final int maxCpHistMemSize; + private final boolean truncateWalOnCpFinish; /** */ private Map>> reservedForExchange; @@ -416,6 +360,8 @@ public GridCacheDatabaseSharedManager(GridKernalContext ctx) { checkpointFreq = persistenceCfg.getCheckpointFrequency(); + truncateWalOnCpFinish = persistenceCfg.getWalHistorySize() != Integer.MAX_VALUE; + lockWaitTime = persistenceCfg.getLockWaitTime(); persStoreMetrics = new DataStorageMetricsImpl( @@ -426,9 +372,6 @@ public GridCacheDatabaseSharedManager(GridKernalContext ctx) { metastorageLifecycleLsnrs = ctx.internalSubscriptionProcessor().getMetastorageSubscribers(); - maxCpHistMemSize = Math.min(persistenceCfg.getWalHistorySize(), - IgniteSystemProperties.getInteger(IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE, 100)); - ioFactory = persistenceCfg.getFileIOFactory(); } @@ -515,6 +458,8 @@ private DataRegionConfiguration createDataRegionConfiguration(DataStorageConfigu if (!kernalCtx.clientNode()) { checkpointer = new Checkpointer(cctx.igniteInstanceName(), "db-checkpoint-thread", log); + cpHistory = new CheckpointHistory(kernalCtx); + IgnitePageStoreManager store = cctx.pageStore(); assert store instanceof FilePageStoreManager : "Invalid page store manager was created: " + store; @@ -531,6 +476,7 @@ private DataRegionConfiguration createDataRegionConfiguration(DataStorageConfigu final FileLockHolder preLocked = kernalCtx.pdsFolderResolver() .resolveFolders() .getLockedFileLockHolder(); + if (preLocked == null) fileLockHolder = new FileLockHolder(storeMgr.workDir().getPath(), kernalCtx, log); @@ -546,12 +492,10 @@ private DataRegionConfiguration createDataRegionConfiguration(DataStorageConfigu */ private void cleanupTempCheckpointDirectory() throws IgniteCheckedException { try { - try (DirectoryStream files = Files.newDirectoryStream(cpDir.toPath(), new DirectoryStream.Filter() { - @Override - public boolean accept(Path path) throws IOException { - return path.endsWith(FILE_TMP_SUFFIX); - } - })) { + try (DirectoryStream files = Files.newDirectoryStream( + cpDir.toPath(), + path -> path.endsWith(FILE_TMP_SUFFIX)) + ) { for (Path path : files) Files.delete(path); } @@ -577,18 +521,83 @@ public void cleanupCheckpointDirectory() throws IgniteCheckedException { } /** + * Retreives checkpoint history form specified {@code dir}. * + * @return List of checkpoints. */ - private void initDataBase() { - if (persistenceCfg.getCheckpointThreads() > 1) - asyncRunner = new IgniteThreadPoolExecutor( - "checkpoint-runner", - cctx.igniteInstanceName(), - persistenceCfg.getCheckpointThreads(), - persistenceCfg.getCheckpointThreads(), - 30_000, - new LinkedBlockingQueue() - ); + private List retreiveHistory() throws IgniteCheckedException { + if (!cpDir.exists()) + return Collections.emptyList(); + + try (DirectoryStream cpFiles = Files.newDirectoryStream( + cpDir.toPath(), + path -> CP_FILE_NAME_PATTERN.matcher(path.toFile().getName()).matches()) + ) { + List checkpoints = new ArrayList<>(); + + ByteBuffer buf = ByteBuffer.allocate(16); + buf.order(ByteOrder.nativeOrder()); + + for (Path cpFile : cpFiles) { + CheckpointEntry cp = parseFromFile(buf, cpFile.toFile()); + + if (cp != null) + checkpoints.add(cp); + } + + return checkpoints; + } + catch (IOException e) { + throw new IgniteCheckedException("Failed to load checkpoint history.", e); + } + } + + /** + * Parses checkpoint entry from given file. + * + * @param buf Temporary byte buffer. + * @param file Checkpoint file. + */ + @Nullable private CheckpointEntry parseFromFile(ByteBuffer buf, File file) throws IgniteCheckedException { + Matcher matcher = CP_FILE_NAME_PATTERN.matcher(file.getName()); + + if (!matcher.matches()) + return null; + + CheckpointEntryType type = CheckpointEntryType.valueOf(matcher.group(3)); + + if (type != CheckpointEntryType.START) + return null; + + long cpTs = Long.parseLong(matcher.group(1)); + UUID cpId = UUID.fromString(matcher.group(2)); + + WALPointer ptr = readPointer(file, buf); + + return createCheckPointEntry(cpTs, ptr, cpId, null, CheckpointEntryType.START); + } + + /** + * Removes checkpoint start/end files belongs to given {@code cpEntry}. + * + * @param cpEntry Checkpoint entry. + * + * @throws IgniteCheckedException If failed to delete. + */ + private void removeCheckpointFiles(CheckpointEntry cpEntry) throws IgniteCheckedException { + Path startFile = new File(cpDir.getAbsolutePath(), checkpointFileName(cpEntry, CheckpointEntryType.START)).toPath(); + Path endFile = new File(cpDir.getAbsolutePath(), checkpointFileName(cpEntry, CheckpointEntryType.END)).toPath(); + + try { + if (Files.exists(startFile)) + Files.delete(startFile); + + if (Files.exists(endFile)) + Files.delete(endFile); + } + catch (IOException e) { + throw new PersistentStorageIOException("Failed to delete stale checkpoint files: " + cpEntry, e); + } } /** */ @@ -712,6 +721,21 @@ else if (regCfg.getMaxSize() < 8 * GB) } } + /** + * + */ + private void initDataBase() { + if (persistenceCfg.getCheckpointThreads() > 1) + asyncRunner = new IgniteThreadPoolExecutor( + "checkpoint-runner", + cctx.igniteInstanceName(), + persistenceCfg.getCheckpointThreads(), + persistenceCfg.getCheckpointThreads(), + 30_000, + new LinkedBlockingQueue() + ); + } + /** * Try to register Metrics MBean. * @@ -883,49 +907,49 @@ private void nodeStart(WALPointer ptr) throws IgniteCheckedException { * Tuples are sorted by timestamp. * * @return Sorted list of tuples (node started timestamp, memory recovery pointer). - * @throws IgniteCheckedException + * + * @throws IgniteCheckedException If failed. */ public List> nodeStartedPointers() throws IgniteCheckedException { List> res = new ArrayList<>(); - File[] files = cpDir.listFiles(NODE_STARTED_FILE_FILTER); - - Arrays.sort(files, new Comparator() { - @Override public int compare(File o1, File o2) { - String n1 = o1.getName(); - String n2 = o2.getName(); - - long ts1 = Long.valueOf(n1.substring(0, n1.length() - NODE_STARTED_FILE_NAME_SUFFIX.length())); - long ts2 = Long.valueOf(n2.substring(0, n2.length() - NODE_STARTED_FILE_NAME_SUFFIX.length())); - - return Long.compare(ts1, ts2); - } - }); + try (DirectoryStream nodeStartedFiles = Files.newDirectoryStream( + cpDir.toPath(), + path -> path.toFile().getName().endsWith(NODE_STARTED_FILE_NAME_SUFFIX)) + ) { + ByteBuffer buf = ByteBuffer.allocate(20); + buf.order(ByteOrder.nativeOrder()); - ByteBuffer buf = ByteBuffer.allocate(20); - buf.order(ByteOrder.nativeOrder()); + for (Path path : nodeStartedFiles) { + File f = path.toFile(); - for (File f : files){ - String name = f.getName(); + String name = f.getName(); - Long ts = Long.valueOf(name.substring(0, name.length() - NODE_STARTED_FILE_NAME_SUFFIX.length())); + Long ts = Long.valueOf(name.substring(0, name.length() - NODE_STARTED_FILE_NAME_SUFFIX.length())); - try (FileIO io = ioFactory.create(f, READ)) { - io.read(buf); + try (FileIO io = ioFactory.create(f, READ)) { + io.read(buf); - buf.flip(); + buf.flip(); - FileWALPointer ptr = new FileWALPointer( - buf.getLong(), buf.getInt(), buf.getInt()); + FileWALPointer ptr = new FileWALPointer( + buf.getLong(), buf.getInt(), buf.getInt()); - res.add(new T2(ts, ptr)); + res.add(new T2<>(ts, ptr)); - buf.clear(); - } - catch (IOException e) { - throw new IgniteCheckedException("Failed to read node started marker file: " + f.getAbsolutePath(), e); + buf.clear(); + } + catch (IOException e) { + throw new PersistentStorageIOException("Failed to read node started marker file: " + f.getAbsolutePath(), e); + } } } + catch (IOException e) { + throw new PersistentStorageIOException("Failed to retreive node started files.", e); + } + + // Sort start markers by file timestamp. + res.sort(Comparator.comparingLong(IgniteBiTuple::get1)); return res; } @@ -1182,7 +1206,7 @@ private long[] calculateFragmentSizes(int concLvl, long cacheSize, long chpBufSi * @param partFile Partition file. */ private int resolvePageSizeFromPartitionFile(Path partFile) throws IOException, IgniteCheckedException { - try (FileIO fileIO = persistenceCfg.getFileIOFactory().create(partFile.toFile())) { + try (FileIO fileIO = ioFactory.create(partFile.toFile())) { int minimalHdr = FilePageStore.HEADER_SIZE; if (fileIO.size() < minimalHdr) @@ -1269,7 +1293,7 @@ private void shutdownCheckpointer(boolean cancel) { } /** {@inheritDoc} */ - @Override public void beforeExchange(GridDhtPartitionsExchangeFuture fut) throws IgniteCheckedException { + @Override public boolean beforeExchange(GridDhtPartitionsExchangeFuture fut) throws IgniteCheckedException { DiscoveryEvent discoEvt = fut.firstEvent(); boolean joinEvt = discoEvt.type() == EventType.EVT_NODE_JOINED; @@ -1280,9 +1304,13 @@ private void shutdownCheckpointer(boolean cancel) { boolean clusterInTransitionStateToActive = fut.activateCluster(); + boolean restored = false; + // In case of cluster activation or local join restore, restore whole manager state. if (clusterInTransitionStateToActive || (joinEvt && locNode && isSrvNode)) { restoreState(); + + restored = true; } // In case of starting groups, restore partition states only for these groups. else if (fut.exchangeActions() != null && !F.isEmpty(fut.exchangeActions().cacheGroupsToStart())) { @@ -1307,6 +1335,8 @@ else if (acts.localJoinContext() != null && !F.isEmpty(acts.localJoinContext().c } } } + + return restored; } /** @@ -1386,13 +1416,7 @@ private void prepareIndexRebuildFuture(int cacheId) { PageMemoryEx pageMem = (PageMemoryEx)gctx.dataRegion().pageMemory(); - Collection grpIds = destroyed.get(pageMem); - - if (grpIds == null) { - grpIds = new HashSet<>(); - - destroyed.put(pageMem, grpIds); - } + Collection grpIds = destroyed.computeIfAbsent(pageMem, k -> new HashSet<>()); grpIds.add(tup.get1().groupId()); @@ -1549,97 +1573,91 @@ private void restoreState() throws IgniteCheckedException { } snapshotMgr.restoreState(); - - new IgniteThread(cctx.igniteInstanceName(), "db-checkpoint-thread", checkpointer).start(); - - CheckpointProgressSnapshot chp = checkpointer.wakeupForCheckpoint(0, "node started"); - - if (chp != null) - chp.cpBeginFut.get(); } catch (StorageException e) { throw new IgniteCheckedException(e); } } + /** + * Called when all partitions have been fully restored and pre-created on node start. + * + * Starts checkpointing process and initiates first checkpoint. + * + * @throws IgniteCheckedException If first checkpoint has failed. + */ + @Override public void onStateRestored() throws IgniteCheckedException { + new IgniteThread(cctx.igniteInstanceName(), "db-checkpoint-thread", checkpointer).start(); + + CheckpointProgressSnapshot chp = checkpointer.wakeupForCheckpoint(0, "node started"); + + if (chp != null) + chp.cpBeginFut.get(); + } + /** {@inheritDoc} */ @Override public synchronized Map> reserveHistoryForExchange() { assert reservedForExchange == null : reservedForExchange; reservedForExchange = new HashMap<>(); - Map> parts4CheckpointHistSearch = partsForCheckpointHistorySearch(); + Map> applicableGroupsAndPartitions = partitionsApplicableForWalRebalance(); - Map> lastCheckpointEntry4Grp = - searchLastCheckpointEntryPerPartition(parts4CheckpointHistSearch); + Map> earliestValidCheckpoints; - Map> grpPartsWithCnts = new HashMap<>(); + checkpointReadLock(); try { - for (Map.Entry> e : lastCheckpointEntry4Grp.entrySet()) { - Integer grpId = e.getKey(); - - for (Map.Entry e0 : e.getValue().entrySet()) { - CheckpointEntry cpEntry = e0.getValue(); + earliestValidCheckpoints = cpHistory.searchAndReserveCheckpoints(applicableGroupsAndPartitions); + } + finally { + checkpointReadUnlock(); + } - Integer partId = e0.getKey(); + Map> grpPartsWithCnts = new HashMap<>(); - if (cctx.wal().reserve(cpEntry.cpMark)) { - Map> grpChpState = reservedForExchange.get(grpId); + for (Map.Entry> e : earliestValidCheckpoints.entrySet()) { + int grpId = e.getKey(); - Map grpCnts = grpPartsWithCnts.get(grpId); + for (Map.Entry e0 : e.getValue().entrySet()) { + CheckpointEntry cpEntry = e0.getValue(); - if (grpChpState == null) { - reservedForExchange.put(grpId, grpChpState = new HashMap<>()); + int partId = e0.getKey(); - grpPartsWithCnts.put(grpId, grpCnts = new HashMap<>()); - } + assert cctx.wal().reserved(cpEntry.checkpointMark()) + : "WAL segment for checkpoint " + cpEntry + " has not reserved"; - Long partCnt = cpEntry.partitionCounter(cctx, grpId, partId); + Long updCntr = cpEntry.partitionCounter(cctx, grpId, partId); - if (partCnt != null) { - grpChpState.put(partId, new T2<>(partCnt, cpEntry.cpMark)); + if (updCntr != null) { + reservedForExchange.computeIfAbsent(grpId, k -> new HashMap<>()) + .put(partId, new T2<>(updCntr, cpEntry.checkpointMark())); - grpCnts.put(partId, partCnt); - } - else - cctx.wal().release(cpEntry.cpMark); - } + grpPartsWithCnts.computeIfAbsent(grpId, k -> new HashMap<>()).put(partId, updCntr); } } } - catch (IgniteCheckedException ex) { - U.error(log, "Error while trying to reserve history", ex); - } return grpPartsWithCnts; } /** - * - * @return Map of group id -> Set parts. + * @return Map of group id -> Set of partitions which can be used as suppliers for WAL rebalance. */ - private Map> partsForCheckpointHistorySearch() { - Map> part4CheckpointHistSearch = new HashMap<>(); + private Map> partitionsApplicableForWalRebalance() { + Map> res = new HashMap<>(); for (CacheGroupContext grp : cctx.cache().cacheGroups()) { if (grp.isLocal()) continue; - for (GridDhtLocalPartition part : grp.topology().currentLocalPartitions()) { - if (part.state() != GridDhtPartitionState.OWNING || part.dataStore().fullSize() <= walRebalanceThreshold) - continue; - - Set parts = part4CheckpointHistSearch.get(grp.groupId()); - - if (parts == null) - part4CheckpointHistSearch.put(grp.groupId(), parts = new HashSet<>()); - - parts.add(part.id()); + for (GridDhtLocalPartition locPart : grp.topology().currentLocalPartitions()) { + if (locPart.state() == GridDhtPartitionState.OWNING && locPart.fullSize() > walRebalanceThreshold) + res.computeIfAbsent(grp.groupId(), k -> new HashSet<>()).add(locPart.id()); } } - return part4CheckpointHistSearch; + return res; } /** {@inheritDoc} */ @@ -1647,28 +1665,41 @@ private Map> partsForCheckpointHistorySearch() { if (reservedForExchange == null) return; + FileWALPointer earliestPtr = null; + for (Map.Entry>> e : reservedForExchange.entrySet()) { for (Map.Entry> e0 : e.getValue().entrySet()) { - try { - cctx.wal().release(e0.getValue().get2()); - } - catch (IgniteCheckedException ex) { - U.error(log, "Could not release history lock", ex); - } + FileWALPointer ptr = (FileWALPointer) e0.getValue().get2(); + + if (earliestPtr == null || ptr.index() < earliestPtr.index()) + earliestPtr = ptr; } } reservedForExchange = null; + + if (earliestPtr == null) + return; + + assert cctx.wal().reserved(earliestPtr) + : "Earliest checkpoint WAL pointer is not reserved for exchange: " + earliestPtr; + + try { + cctx.wal().release(earliestPtr); + } + catch (IgniteCheckedException e) { + log.error("Failed to release earliest checkpoint WAL pointer: " + earliestPtr, e); + } } /** {@inheritDoc} */ @Override public boolean reserveHistoryForPreloading(int grpId, int partId, long cntr) { - CheckpointEntry cpEntry = searchCheckpointEntry(grpId, partId, cntr); + CheckpointEntry cpEntry = cpHistory.searchCheckpointEntry(grpId, partId, cntr); if (cpEntry == null) return false; - WALPointer ptr = cpEntry.cpMark; + WALPointer ptr = cpEntry.checkpointMark(); if (ptr == null) return false; @@ -1706,13 +1737,6 @@ private Map> partsForCheckpointHistorySearch() { reservedForPreloading.clear(); } - /** - * For debugging only. TODO: remove. - */ - public Map, T2> reservedForPreloading() { - return reservedForPreloading; - } - /** * */ @@ -1758,150 +1782,6 @@ public Map, T2> reservedForPreloading() { return cp.wakeupForCheckpoint(0, reason); } - /** - * Tries to search for a WAL pointer for the given partition counter start. - * - * @return Checkpoint entry or {@code null} if failed to search. - */ - private Map> searchLastCheckpointEntryPerPartition( - final Map> part4reserve - ) { - final Map> res = new HashMap<>(); - - if (F.isEmpty(part4reserve)) - return res; - - for (Long cpTs : checkpointHist.checkpoints()) { - CheckpointEntry chpEntry = null; - - try { - chpEntry = checkpointHist.entry(cpTs); - - Map grpsState = chpEntry.groupState(cctx); - - if (F.isEmpty(grpsState)) { - res.clear(); - - continue; - } - - for (Map.Entry> grps : part4reserve.entrySet()) { - Integer grpId = grps.getKey(); - - Map partToCheckPntEntry = res.get(grpId); - - CheckpointEntry.GroupState grpState = grpsState.get(grpId); - - if (grpState == null) { - res.remove(grpId); - - continue; - } - - if (partToCheckPntEntry == null) - res.put(grpId, partToCheckPntEntry = new HashMap<>()); - - for (Integer partId : grps.getValue()) { - int idx = grpState.indexByPartition(partId); - - if (idx < 0) - partToCheckPntEntry.remove(partId); - else { - if (partToCheckPntEntry.containsKey(partId)) - continue; - - partToCheckPntEntry.put(partId, chpEntry); - } - } - } - } - catch (IgniteCheckedException ex) { - String msg = chpEntry != null ? - ", chpId=" + chpEntry.cpId + " ptr=" + chpEntry.cpMark + " ts=" + chpEntry.cpTs : ""; - - U.error(log, "Failed to read checkpoint entry" + msg, ex); - - // Treat exception the same way as a gap. - res.clear(); - } - } - - return res; - } - - /** - * Tries to search for a WAL pointer for the given partition counter start. - * - * @param grpId Cache group ID. - * @param part Partition ID. - * @param partCntrSince Partition counter or {@code null} to search for minimal counter. - * @return Checkpoint entry or {@code null} if failed to search. - */ - @Nullable public WALPointer searchPartitionCounter(int grpId, int part, @Nullable Long partCntrSince) { - CheckpointEntry entry = searchCheckpointEntry(grpId, part, partCntrSince); - - if (entry == null) - return null; - - return entry.cpMark; - } - - /** - * Tries to search for a WAL pointer for the given partition counter start. - * - * @param grpId Cache group ID. - * @param part Partition ID. - * @param partCntrSince Partition counter or {@code null} to search for minimal counter. - * @return Checkpoint entry or {@code null} if failed to search. - */ - @Nullable private CheckpointEntry searchCheckpointEntry(int grpId, int part, @Nullable Long partCntrSince) { - boolean hasGap = false; - CheckpointEntry first = null; - - for (Long cpTs : checkpointHist.checkpoints()) { - try { - CheckpointEntry entry = checkpointHist.entry(cpTs); - - Long foundCntr = entry.partitionCounter(cctx, grpId, part); - - if (foundCntr != null) { - if (partCntrSince == null) { - if (hasGap) { - first = entry; - - hasGap = false; - } - - if (first == null) - first = entry; - } - else if (foundCntr <= partCntrSince) { - first = entry; - - hasGap = false; - } - else - return hasGap ? null : first; - } - else - hasGap = true; - } - catch (IgniteCheckedException ignore) { - // Treat exception the same way as a gap. - hasGap = true; - } - } - - return hasGap ? null : first; - } - - /** - * @return Checkpoint history. For tests only. - */ - public CheckpointHistory checkpointHistory() { - return checkpointHist; - } - /** * @return Checkpoint directory. */ @@ -1944,19 +1824,7 @@ private CheckpointStatus readCheckpointStatus() throws IgniteCheckedException { File dir = cpDir; if (!dir.exists()) { - // TODO: remove excessive logging after GG-12116 fix. - File[] files = dir.listFiles(); - - if (files != null && files.length > 0) { - log.warning("Read checkpoint status: cpDir.exists() is false, cpDir.listFiles() is: " + - Arrays.toString(files)); - } - - if (Files.exists(dir.toPath())) - log.warning("Read checkpoint status: cpDir.exists() is false, Files.exists(cpDir) is true."); - - if (log.isInfoEnabled()) - log.info("Read checkpoint status: checkpoint directory is not found."); + log.warning("Read checkpoint status: checkpoint directory is not found."); return new CheckpointStatus(0, startId, startPtr, endId, endPtr); } @@ -2230,7 +2098,7 @@ else if (!F.eq(cpRec.checkpointId(), status.cpEndId)) finalizeCheckpointOnRecovery(status.cpStartTs, status.cpStartId, status.startPtr); } - checkpointHist.loadHistory(cpDir); + cpHistory.initialize(retreiveHistory()); return lastRead == null ? null : lastRead.next(); } @@ -2552,8 +2420,11 @@ else if (restore != null) { * * @param highBound WALPointer. */ - public void onWalTruncated(WALPointer highBound) { - checkpointHist.onWalTruncated(highBound); + public void onWalTruncated(WALPointer highBound) throws IgniteCheckedException { + List removedFromHistory = cpHistory.onWalTruncated(highBound); + + for (CheckpointEntry cp : removedFromHistory) + removeCheckpointFiles(cp); } /** @@ -2686,7 +2557,7 @@ private void finalizeCheckpointOnRecovery(long cpTs, UUID cpId, WALPointer walPt for (IgniteBiTuple> e : cpEntities) ((PageMemoryEx)e.get1()).finishCheckpoint(); - writeCheckpointEntry( + CheckpointEntry cp = prepareCheckpointEntry( tmpWriteBuf, cpTs, cpId, @@ -2694,6 +2565,8 @@ private void finalizeCheckpointOnRecovery(long cpTs, UUID cpId, WALPointer walPt null, CheckpointEntryType.END); + writeCheckpointEntry(tmpWriteBuf, cp, CheckpointEntryType.END); + cctx.pageStore().finishRecover(); if (log.isInfoEnabled()) @@ -2708,43 +2581,61 @@ private void finalizeCheckpointOnRecovery(long cpTs, UUID cpId, WALPointer walPt } /** - * Writes into specified file checkpoint entry containing WAL pointer to checkpoint record. + * Prepares checkpoint entry containing WAL pointer to checkpoint record. + * Writes into given {@code ptrBuf} WAL pointer content. * - * @param cpId Checkpoint ID. - * @param ptr Wal pointer of current checkpoint. + * @param entryBuf Buffer to fill + * @param cpTs Checkpoint timestamp. + * @param cpId Checkpoint id. + * @param ptr WAL pointer containing record. + * @param rec Checkpoint WAL record. + * @param type Checkpoint type. + * @return Checkpoint entry. */ - private CheckpointEntry writeCheckpointEntry( - ByteBuffer tmpWriteBuf, + private CheckpointEntry prepareCheckpointEntry( + ByteBuffer entryBuf, long cpTs, UUID cpId, WALPointer ptr, - CheckpointRecord rec, + @Nullable CheckpointRecord rec, CheckpointEntryType type - ) throws IgniteCheckedException { + ) { assert ptr instanceof FileWALPointer; FileWALPointer filePtr = (FileWALPointer)ptr; - String fileName = checkpointFileName(cpTs, cpId, type); - String tmpFileName = fileName + FILE_TMP_SUFFIX; + entryBuf.rewind(); - try { - try (FileIO io = ioFactory.create(Paths.get(cpDir.getAbsolutePath(), skipSync ? fileName : tmpFileName).toFile(), - StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE)) { + entryBuf.putLong(filePtr.index()); - tmpWriteBuf.rewind(); + entryBuf.putInt(filePtr.fileOffset()); - tmpWriteBuf.putLong(filePtr.index()); + entryBuf.putInt(filePtr.length()); - tmpWriteBuf.putInt(filePtr.fileOffset()); + entryBuf.flip(); - tmpWriteBuf.putInt(filePtr.length()); + return createCheckPointEntry(cpTs, ptr, cpId, rec, type); + } - tmpWriteBuf.flip(); + /** + * Writes checkpoint entry buffer {@code entryBuf} to specified checkpoint file with 2-phase protocol. + * + * @param entryBuf Checkpoint entry buffer to write. + * @param cp Checkpoint entry. + * @param type Checkpoint entry type. + * @throws PersistentStorageIOException If failed to write checkpoint entry. + */ + public void writeCheckpointEntry(ByteBuffer entryBuf, CheckpointEntry cp, CheckpointEntryType type) throws PersistentStorageIOException { + String fileName = checkpointFileName(cp, type); + String tmpFileName = fileName + FILE_TMP_SUFFIX; + + try { + try (FileIO io = ioFactory.create(Paths.get(cpDir.getAbsolutePath(), skipSync ? fileName : tmpFileName).toFile(), + StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE)) { - io.write(tmpWriteBuf); + io.write(entryBuf); - tmpWriteBuf.clear(); + entryBuf.clear(); if (!skipSync) io.force(true); @@ -2752,14 +2643,12 @@ private CheckpointEntry writeCheckpointEntry( if (!skipSync) Files.move(Paths.get(cpDir.getAbsolutePath(), tmpFileName), Paths.get(cpDir.getAbsolutePath(), fileName)); - - return createCheckPointEntry(cpTs, ptr, cpId, rec, type); } catch (IOException e) { - throw new PersistentStorageIOException("Failed to write checkpoint entry [ptr=" + filePtr - + ", cpTs=" + cpTs - + ", cpId=" + cpId - + ", type=" + type + "]", e); + throw new PersistentStorageIOException("Failed to write checkpoint entry [ptr=" + cp.checkpointMark() + + ", cpTs=" + cp.timestamp() + + ", cpId=" + cp.checkpointId() + + ", type=" + type + "]", e); } } @@ -2793,6 +2682,14 @@ private static String checkpointFileName(long cpTs, UUID cpId, CheckpointEntryTy return cpTs + "-" + cpId + "-" + type + ".bin"; } + /** + * @param cp Checkpoint entry. + * @param type Checkpoint type. + */ + private static String checkpointFileName(CheckpointEntry cp, CheckpointEntryType type) { + return checkpointFileName(cp.timestamp(), cp.checkpointId(), type); + } + /** * Replace thread local with buffers. Thread local should provide direct buffer with one page in length. * @@ -2811,7 +2708,7 @@ public void setThreadBuf(final ThreadLocal threadBuf) { * * @return Checkpoint entry. */ - private CheckpointEntry createCheckPointEntry( + public CheckpointEntry createCheckPointEntry( long cpTs, WALPointer ptr, UUID cpId, @@ -2823,18 +2720,23 @@ private CheckpointEntry createCheckPointEntry( assert cpId != null; assert type != null; - if (type != CheckpointEntryType.START) - return null; - Map cacheGrpStates = null; - // Create lazy checkpoint entry. - if ((checkpointHist.histMap.size() + 1 < maxCpHistMemSize) && rec != null) + // Do not hold groups state in-memory if there is no space in the checkpoint history to prevent possible OOM. + // In this case the actual group states will be readed from WAL by demand. + if (rec != null && cpHistory.hasSpace()) cacheGrpStates = rec.cacheGroupStates(); return new CheckpointEntry(cpTs, ptr, cpId, cacheGrpStates); } + /** + * @return Checkpoint history. + */ + @Nullable public CheckpointHistory checkpointHistory() { + return cpHistory; + } + /** * Adds given partition to checkpointer destroy queue. * @@ -3275,8 +3177,18 @@ private void doCheckpoint() { success = true; } finally { - if (success) - markCheckpointEnd(chp); + if (success) { + try { + markCheckpointEnd(chp); + } + catch (IgniteCheckedException e) { + chp.progress.cpFinishFut.onDone(e); + + cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e)); + + return; + } + } } tracker.onEnd(); @@ -3486,14 +3398,27 @@ private Checkpoint markCheckpointBegin(CheckpointMetricsTracker tracker) throws final CheckpointProgress curr; + CheckpointEntry cp = null; + IgniteBiTuple>, Integer> cpPagesTuple; tracker.onLockWaitStart(); boolean hasPages; + boolean hasPartitionsToDestroy; + IgniteFuture snapFut = null; + long cpTs = System.currentTimeMillis(); + + // This can happen in an unlikely event of two checkpoints happening + // within a currentTimeMillis() granularity window. + if (cpTs == lastCpTs) + cpTs++; + + lastCpTs = cpTs; + checkpointLock.writeLock().lock(); try { @@ -3564,13 +3489,27 @@ private Checkpoint markCheckpointBegin(CheckpointMetricsTracker tracker) throws hasPages = hasPageForWrite(cpPagesTuple.get1()); - if (hasPages || curr.nextSnapshot || !curr.destroyQueue.pendingReqs.isEmpty()) { + hasPartitionsToDestroy = !curr.destroyQueue.pendingReqs.isEmpty(); + + if (hasPages || curr.nextSnapshot || hasPartitionsToDestroy) { // No page updates for this checkpoint are allowed from now on. cpPtr = cctx.wal().log(cpRec); if (cpPtr == null) cpPtr = CheckpointStatus.NULL_PTR; } + + if (hasPages || hasPartitionsToDestroy) { + cp = prepareCheckpointEntry( + tmpWriteBuf, + cpTs, + cpRec.checkpointId(), + cpPtr, + cpRec, + CheckpointEntryType.START); + + cpHistory.addCheckpoint(cp); + } } finally { checkpointLock.writeLock().unlock(); @@ -3590,8 +3529,9 @@ private Checkpoint markCheckpointBegin(CheckpointMetricsTracker tracker) throws } } - if (hasPages || !curr.destroyQueue.pendingReqs.isEmpty()) { + if (hasPages || hasPartitionsToDestroy) { assert cpPtr != null; + assert cp != null; tracker.onWalCpRecordFsyncStart(); @@ -3600,24 +3540,7 @@ private Checkpoint markCheckpointBegin(CheckpointMetricsTracker tracker) throws tracker.onWalCpRecordFsyncEnd(); - long cpTs = System.currentTimeMillis(); - - // This can happen in an unlikely event of two checkpoints happening - // within a currentTimeMillis() granularity window. - if (cpTs == lastCpTs) - cpTs++; - - lastCpTs = cpTs; - - CheckpointEntry cpEntry = writeCheckpointEntry( - tmpWriteBuf, - cpTs, - cpRec.checkpointId(), - cpPtr, - cpRec, - CheckpointEntryType.START); - - checkpointHist.addCheckpointEntry(cpEntry); + writeCheckpointEntry(tmpWriteBuf, cp, CheckpointEntryType.START); GridMultiCollectionWrapper cpPages = splitAndSortCpPagesIfNeeded(cpPagesTuple); @@ -3634,7 +3557,7 @@ private Checkpoint markCheckpointBegin(CheckpointMetricsTracker tracker) throws curr.reason) ); - return new Checkpoint(cpEntry, cpPages, curr); + return new Checkpoint(cp, cpPages, curr); } else { if (curr.nextSnapshot) @@ -3714,19 +3637,27 @@ private void markCheckpointEnd(Checkpoint chp) throws IgniteCheckedException { ((PageMemoryEx)memPlc.pageMemory()).finishCheckpoint(); } - if (chp.hasDelta()) - writeCheckpointEntry( - tmpWriteBuf, - chp.cpEntry.checkpointTimestamp(), - chp.cpEntry.checkpointId(), - chp.cpEntry.checkpointMark(), - null, - CheckpointEntryType.END); - currCheckpointPagesCnt = 0; } - checkpointHist.onCheckpointFinished(chp); + if (chp.hasDelta()) { + CheckpointEntry cp = prepareCheckpointEntry( + tmpWriteBuf, + chp.cpEntry.timestamp(), + chp.cpEntry.checkpointId(), + chp.cpEntry.checkpointMark(), + null, + CheckpointEntryType.END); + + writeCheckpointEntry(tmpWriteBuf, cp, CheckpointEntryType.END); + + cctx.wal().allowCompressionUntil(chp.cpEntry.checkpointMark()); + } + + List removedFromHistory = cpHistory.onCheckpointFinished(chp, truncateWalOnCpFinish); + + for (CheckpointEntry cp : removedFromHistory) + removeCheckpointFiles(cp); if (chp.progress != null) chp.progress.cpFinishFut.onDone(); @@ -3930,18 +3861,7 @@ private WriteCheckpointPages( /** * */ - private enum CheckpointEntryType { - /** */ - START, - - /** */ - END - } - - /** - * - */ - private static class Checkpoint { + public static class Checkpoint { /** Checkpoint entry. */ @Nullable private final CheckpointEntry cpEntry; @@ -3977,9 +3897,16 @@ private Checkpoint( /** * @return {@code true} if this checkpoint contains at least one dirty page. */ - private boolean hasDelta() { + public boolean hasDelta() { return pagesSize != 0; } + + /** + * @param walFilesDeleted Wal files deleted. + */ + public void walFilesDeleted(int walFilesDeleted) { + this.walFilesDeleted = walFilesDeleted; + } } /** @@ -4037,16 +3964,16 @@ public String toString() { } /** - * + * Data class representing the state of running/scheduled checkpoint. */ private static class CheckpointProgress { - /** */ + /** Scheduled time of checkpoint. */ private volatile long nextCpTs; - /** */ + /** Checkpoint begin phase future. */ private GridFutureAdapter cpBeginFut = new GridFutureAdapter<>(); - /** */ + /** Checkpoint finish phase future. */ private GridFutureAdapter cpFinishFut = new GridFutureAdapter() { @Override protected boolean onDone(@Nullable Void res, @Nullable Throwable err, boolean cancel) { if (err != null && !cpBeginFut.isDone()) @@ -4056,13 +3983,13 @@ private static class CheckpointProgress { } }; - /** */ + /** Flag indicates that snapshot operation will be performed after checkpoint. */ private volatile boolean nextSnapshot; - /** */ + /** Flag indicates that checkpoint is started. */ private volatile boolean started; - /** */ + /** Snapshot operation that should be performed if {@link #nextSnapshot} set to true. */ private volatile SnapshotOperation snapshotOperation; /** Partitions destroy queue. */ @@ -4110,540 +4037,6 @@ private static class CheckpointProgressSnapshot implements CheckpointFuture { } } - /** - * Checkpoint history. Holds chronological ordered map with {@link GridCacheDatabaseSharedManager.CheckpointEntry - * CheckpointEntries}. Data is loaded from corresponding checkpoint directory. This directory holds files for - * checkpoint start and end. - */ - @SuppressWarnings("PublicInnerClass") - public class CheckpointHistory { - /** - * Maps checkpoint's timestamp (from CP file name) to CP entry. - * Using TS provides historical order of CP entries in map ( first is oldest ) - */ - private final NavigableMap histMap = new ConcurrentSkipListMap<>(); - - /** - * Load history form checkpoint directory. - * - * @param dir Checkpoint state dir. - */ - private void loadHistory(File dir) throws IgniteCheckedException { - if (!dir.exists()) - return; - - File[] files = dir.listFiles(CP_FILE_FILTER); - - if (!F.isEmpty(files)) { - Arrays.sort(files, CP_TS_COMPARATOR); - - ByteBuffer buf = ByteBuffer.allocate(16); - buf.order(ByteOrder.nativeOrder()); - - for (File file : files) { - Matcher matcher = CP_FILE_NAME_PATTERN.matcher(file.getName()); - - if (matcher.matches()) { - CheckpointEntryType type = CheckpointEntryType.valueOf(matcher.group(3)); - - if (type == CheckpointEntryType.START) { - long cpTs = Long.parseLong(matcher.group(1)); - UUID cpId = UUID.fromString(matcher.group(2)); - - WALPointer ptr = readPointer(file, buf); - - if (ptr == null) - continue; - - CheckpointEntry entry = createCheckPointEntry(cpTs, ptr, cpId, null, type); - - histMap.put(cpTs, entry); - } - } - } - } - } - - /** - * @param cpTs Checkpoint timestamp. - * @return Initialized entry. - * @throws IgniteCheckedException If failed to initialize entry. - */ - private CheckpointEntry entry(Long cpTs) throws IgniteCheckedException { - CheckpointEntry entry = histMap.get(cpTs); - - if (entry == null) - throw new IgniteCheckedException("Checkpoint entry was removed: " + cpTs); - - return entry; - } - - /** - * @return First checkpoint entry if exists. Otherwise {@code null}. - */ - private CheckpointEntry firstEntry() { - Map.Entry entry = histMap.firstEntry(); - - return entry != null ? entry.getValue() : null; - } - - /** - * Get WAL pointer to low checkpoint bound. - * - * @return WAL pointer to low checkpoint bound. - */ - public WALPointer lowCheckpointBound() { - CheckpointEntry entry = firstEntry(); - - return entry != null ? entry.cpMark : null; - } - - /** - * @return Collection of checkpoint timestamps. - */ - public Collection checkpoints() { - return histMap.keySet(); - } - - /** - * Adds checkpoint entry after the corresponding WAL record has been written to WAL. The checkpoint itself - * is not finished yet. - * - * @param entry Entry to ad. - */ - private void addCheckpointEntry(CheckpointEntry entry) { - histMap.put(entry.checkpointTimestamp(), entry); - } - - /** - * Callback on truncate wal. - */ - private void onWalTruncated(WALPointer ptr) { - FileWALPointer highBound = (FileWALPointer)ptr; - - List cpToRemove = new ArrayList<>(); - - for (CheckpointEntry cpEntry : histMap.values()) { - FileWALPointer cpPnt = (FileWALPointer)cpEntry.checkpointMark(); - - if (highBound.compareTo(cpPnt) <= 0) - break; - - if (cctx.wal().reserved(cpEntry.checkpointMark())) { - U.warn(log, "Could not clear historyMap due to WAL reservation on cpEntry " + cpEntry.cpId + - ", history map size is " + histMap.size()); - - break; - } - - if (!removeCheckpointFiles(cpEntry)) - cpToRemove.add(cpEntry); - } - - for (CheckpointEntry cpEntry : cpToRemove) - histMap.remove(cpEntry.cpTs); - } - - /** - * Clears checkpoint history. - */ - private void onCheckpointFinished(Checkpoint chp) { - int deleted = 0; - - boolean dropWal = persistenceCfg.getWalHistorySize() != Integer.MAX_VALUE; - - while (histMap.size() > maxCpHistMemSize) { - Map.Entry entry = histMap.firstEntry(); - - CheckpointEntry cpEntry = entry.getValue(); - - if (cctx.wal().reserved(cpEntry.checkpointMark())) { - U.warn(log, "Could not clear historyMap due to WAL reservation on cpEntry " + cpEntry.cpId + - ", history map size is " + histMap.size()); - - break; - } - - boolean fail = removeCheckpointFiles(cpEntry); - - if (!fail) { - if (dropWal) - deleted += cctx.wal().truncate(null, cpEntry.checkpointMark()); - - histMap.remove(entry.getKey()); - } - else - break; - } - - chp.walFilesDeleted = deleted; - - if (!chp.cpPages.isEmpty()) - cctx.wal().allowCompressionUntil(chp.cpEntry.checkpointMark()); - } - - /** - * @param cpEntry Checkpoint entry. - * @return {True} if delete fail. - */ - private boolean removeCheckpointFiles(CheckpointEntry cpEntry) { - File startFile = new File(cpDir.getAbsolutePath(), cpEntry.startFile()); - File endFile = new File(cpDir.getAbsolutePath(), cpEntry.endFile()); - - boolean rmvdStart = !startFile.exists() || startFile.delete(); - boolean rmvdEnd = !endFile.exists() || endFile.delete(); - - boolean fail = !rmvdStart || !rmvdEnd; - - if (fail) { - U.warn(log, "Failed to remove stale checkpoint files [startFile=" + startFile.getAbsolutePath() + - ", endFile=" + endFile.getAbsolutePath() + ']'); - - if (histMap.size() > 2 * maxCpHistMemSize) { - U.error(log, "Too many stale checkpoint entries in the map, will truncate WAL archive anyway."); - - fail = false; - } - } - - return fail; - } - } - - /** - * Checkpoint entry. - */ - private static class CheckpointEntry { - /** Checkpoint timestamp. */ - private long cpTs; - - /** Checkpoint end mark. */ - private WALPointer cpMark; - - /** Checkpoint ID. */ - private UUID cpId; - - /** */ - private volatile SoftReference grpStateLazyStore; - - /** - * Checkpoint entry constructor. - * - * If {@code grpStates} is null then it will be inited lazy from wal pointer. - * - * @param cpTs Checkpoint timestamp. - * @param cpMark Checkpoint mark pointer. - * @param cpId Checkpoint ID. - * @param cacheGrpStates Cache groups states. - */ - private CheckpointEntry( - long cpTs, - WALPointer cpMark, - UUID cpId, - @Nullable Map cacheGrpStates - ) { - this.cpTs = cpTs; - this.cpMark = cpMark; - this.cpId = cpId; - this.grpStateLazyStore = new SoftReference<>(new GroupStateLazyStore(cacheGrpStates)); - } - - /** - * @return Checkpoint timestamp. - */ - private long checkpointTimestamp() { - return cpTs; - } - - /** - * @return Checkpoint ID. - */ - private UUID checkpointId() { - return cpId; - } - - /** - * @return Checkpoint mark. - */ - private WALPointer checkpointMark() { - return cpMark; - } - - /** - * @return Start file name. - */ - private String startFile() { - return checkpointFileName(cpTs, cpId, CheckpointEntryType.START); - } - - /** - * @return End file name. - */ - private String endFile() { - return checkpointFileName(cpTs, cpId, CheckpointEntryType.END); - } - - /** - * @param cctx Cache shred context. - */ - public Map groupState( - GridCacheSharedContext cctx - ) throws IgniteCheckedException { - GroupStateLazyStore store = initIfNeeded(cctx); - - return store.grpStates; - } - - /** - * @param cctx Cache shred context. - * @return Group lazy store. - */ - private GroupStateLazyStore initIfNeeded(GridCacheSharedContext cctx) throws IgniteCheckedException { - GroupStateLazyStore store = grpStateLazyStore.get(); - - if (store == null) { - store = new GroupStateLazyStore(); - - grpStateLazyStore = new SoftReference<>(store); - } - - store.initIfNeeded(cctx, cpMark); - - return store; - } - - /** - * @param cctx Cache shared context. - * @param grpId Cache group ID. - * @param part Partition ID. - * @return Partition counter or {@code null} if not found. - */ - private Long partitionCounter(GridCacheSharedContext cctx, int grpId, int part) { - GroupStateLazyStore store; - - try { - store = initIfNeeded(cctx); - } - catch (IgniteCheckedException e) { - return null; - } - - return store.partitionCounter(grpId, part); - } - - /** - * - */ - private static class GroupState { - /** */ - private int[] parts; - - /** */ - private long[] cnts; - - /** */ - private int idx; - - /** - * @param partsCnt Partitions count. - */ - private GroupState(int partsCnt) { - parts = new int[partsCnt]; - cnts = new long[partsCnt]; - } - - /** - * @param partId Partition ID to add. - * @param cntr Partition counter. - */ - public void addPartitionCounter(int partId, long cntr) { - if (idx == parts.length) - throw new IllegalStateException("Failed to add new partition to the partitions state " + - "(no enough space reserved) [partId=" + partId + ", reserved=" + parts.length + ']'); - - if (idx > 0) { - if (parts[idx - 1] >= partId) - throw new IllegalStateException("Adding partition in a wrong order [prev=" + parts[idx - 1] + - ", cur=" + partId + ']'); - } - - parts[idx] = partId; - - cnts[idx] = cntr; - - idx++; - } - - /** - * Gets partition counter by partition ID. - * - * @param partId Partition ID. - * @return Partition update counter (will return {@code -1} if partition is not present in the record). - */ - public long counterByPartition(int partId) { - int idx = indexByPartition(partId); - - return idx >= 0 ? cnts[idx] : 0; - } - - public long size(){ - return idx; - } - - /** - * @param partId Partition ID to search. - * @return Non-negative index of partition if found or negative value if not found. - */ - private int indexByPartition(int partId) { - return Arrays.binarySearch(parts, 0, idx, partId); - } - - /** {@inheritDoc} */ - @Override public String toString() { - return "GroupState [cap=" + parts.length + ", size=" + idx + ']'; - } - } - - /** - * Group state lazy store. - */ - private static class GroupStateLazyStore { - /** */ - private static final AtomicIntegerFieldUpdater initGuardUpdater = - AtomicIntegerFieldUpdater.newUpdater(GroupStateLazyStore.class, "initGuard"); - - /** Cache states. Initialized lazily. */ - private volatile Map grpStates; - - /** */ - private final CountDownLatch latch; - - /** */ - @SuppressWarnings("unused") - private volatile int initGuard; - - /** Initialization exception. */ - private IgniteCheckedException initEx; - - /** - * Default constructor. - */ - private GroupStateLazyStore() { - this(null); - } - - /** - * @param cacheGrpStates Cache group state. - */ - private GroupStateLazyStore(Map cacheGrpStates) { - CountDownLatch latch; - - if (cacheGrpStates != null) { - initGuard = 1; - - this.latch = new CountDownLatch(0); - } - else - this.latch = new CountDownLatch(1); - - this.grpStates = remap(cacheGrpStates); - } - - /** - * @param stateRec Cache group state. - */ - private Map remap(Map stateRec) { - if (stateRec == null) - return null; - - Map grpStates = new HashMap<>(stateRec.size()); - - for (Integer grpId : stateRec.keySet()) { - CacheState recState = stateRec.get(grpId); - - GroupState groupState = new GroupState(recState.size()); - - for (int i = 0; i < recState.size(); i++) { - groupState.addPartitionCounter( - recState.partitionByIndex(i), - recState.partitionCounterByIndex(i) - ); - } - - grpStates.put(grpId, groupState); - } - - return grpStates; - } - - /** - * @param grpId Group id. - * @param part Partition id. - * @return Partition counter. - */ - private Long partitionCounter(int grpId, int part) { - assert initGuard != 0 : initGuard; - - if (initEx != null || grpStates == null) - return null; - - GroupState state = grpStates.get(grpId); - - if (state != null) { - long cntr = state.counterByPartition(part); - - return cntr < 0 ? null : cntr; - } - - return null; - } - - /** - * @param cctx Cache shared context. - * @param ptr Checkpoint wal pointer. - * @throws IgniteCheckedException If failed to read WAL entry. - */ - private void initIfNeeded( - GridCacheSharedContext cctx, - WALPointer ptr - ) throws IgniteCheckedException { - if (initGuardUpdater.compareAndSet(this, 0, 1)) { - try (WALIterator it = cctx.wal().replay(ptr)) { - if (it.hasNextX()) { - IgniteBiTuple tup = it.nextX(); - - CheckpointRecord rec = (CheckpointRecord)tup.get2(); - - Map stateRec = rec.cacheGroupStates(); - - if (stateRec != null) - this.grpStates = remap(stateRec); - else - grpStates = Collections.emptyMap(); - } - else - initEx = new IgniteCheckedException( - "Failed to find checkpoint record at the given WAL pointer: " + ptr); - } - catch (IgniteCheckedException e) { - initEx = e; - - throw e; - } - finally { - latch.countDown(); - } - } - else { - U.await(latch); - - if (initEx != null) - throw initEx; - } - } - } - } - /** * */ @@ -4853,8 +4246,11 @@ public DataStorageMetricsImpl persistentStoreMetricsImpl() { try { if (enabled) metaStorage.remove(key); - else + else { metaStorage.write(key, true); + + lastCheckpointInapplicableForWalRebalance(grpId); + } } catch (IgniteCheckedException e) { throw new IgniteException("Failed to write cache group WAL state [grpId=" + grpId + @@ -4865,6 +4261,41 @@ public DataStorageMetricsImpl persistentStoreMetricsImpl() { } } + /** + * Checks that checkpoint with timestamp {@code cpTs} is inapplicable as start point for WAL rebalance for given group {@code grpId}. + * + * @param cpTs Checkpoint timestamp. + * @param grpId Group ID. + * @return {@code true} if checkpoint {@code cpTs} is inapplicable as start point for WAL rebalance for {@code grpId}. + * @throws IgniteCheckedException If failed to check. + */ + public boolean isCheckpointInapplicableForWalRebalance(Long cpTs, int grpId) throws IgniteCheckedException { + return metaStorage.read(checkpointInapplicableCpAndGroupIdToKey(cpTs, grpId)) != null; + } + + /** + * Set last checkpoint as inapplicable for WAL rebalance for given group {@code grpId}. + * + * @param grpId Group ID. + */ + @Override public void lastCheckpointInapplicableForWalRebalance(int grpId) { + checkpointReadLock(); + + try { + CheckpointEntry lastCp = cpHistory.lastCheckpoint(); + long lastCpTs = lastCp != null ? lastCp.timestamp() : 0; + + if (lastCpTs != 0) + metaStorage.write(checkpointInapplicableCpAndGroupIdToKey(lastCpTs, grpId), true); + } + catch (IgniteCheckedException e) { + log.error("Failed to mark last checkpoint as inapplicable for WAL rebalance for group: " + grpId, e); + } + finally { + checkpointReadUnlock(); + } + } + /** * */ @@ -4880,6 +4311,9 @@ private void fillWalDisabledGroups() { for (String key : keys) { T2 t2 = walKeyToGroupIdAndLocalFlag(key); + if (t2 == null) + continue; + if (t2.get2()) initiallyLocalWalDisabledGrps.add(t2.get1()); else @@ -4905,6 +4339,17 @@ private static String walGroupIdToKey(int grpId, boolean local) { return WAL_GLOBAL_KEY_PREFIX + grpId; } + /** + * Convert checkpoint timestamp and cache group ID to key for {@link #CHECKPOINT_INAPPLICABLE_FOR_REBALANCE} metastorage records. + * + * @param cpTs Checkpoint timestamp. + * @param grpId Group ID. + * @return Key. + */ + private static String checkpointInapplicableCpAndGroupIdToKey(long cpTs, int grpId) { + return CHECKPOINT_INAPPLICABLE_FOR_REBALANCE + cpTs + "-" + grpId; + } + /** * Convert WAL state key to cache group ID. * @@ -4914,7 +4359,9 @@ private static String walGroupIdToKey(int grpId, boolean local) { private static T2 walKeyToGroupIdAndLocalFlag(String key) { if (key.startsWith(WAL_LOCAL_KEY_PREFIX)) return new T2<>(Integer.parseInt(key.substring(WAL_LOCAL_KEY_PREFIX.length())), true); - else + else if (key.startsWith(WAL_GLOBAL_KEY_PREFIX)) return new T2<>(Integer.parseInt(key.substring(WAL_GLOBAL_KEY_PREFIX.length())), false); + else + return 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 9445a43ab2265..8173e48ac07f6 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 @@ -748,7 +748,8 @@ private Metas getOrAllocateCacheMetas() throws IgniteCheckedException { int p = partCntrs.partitionAt(i); long initCntr = partCntrs.initialUpdateCounterAt(i); - FileWALPointer startPtr = (FileWALPointer)database.searchPartitionCounter(grp.groupId(), p, initCntr); + FileWALPointer startPtr = (FileWALPointer)database.checkpointHistory().searchPartitionCounter( + grp.groupId(), p, initCntr); if (startPtr == null) throw new IgniteCheckedException("Could not find start pointer for partition [part=" + p + ", partCntrSince=" + initCntr + "]"); @@ -1009,7 +1010,7 @@ private void advance() { long from = partMap.initialUpdateCounterAt(idx); long to = partMap.updateCounterAt(idx); - if (entry.partitionCounter() >= from && entry.partitionCounter() <= to) { + if (entry.partitionCounter() > from && entry.partitionCounter() <= to) { if (entry.partitionCounter() == to) reachedPartitionEnd = true; 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 e3ce04d8809ac..06f1c5762a983 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 @@ -766,14 +766,25 @@ public void waitForCheckpoint(String reason) throws IgniteCheckedException { /** * @param discoEvt Before exchange for the given discovery event. + * + * @return {@code True} if partitions have been restored from persistent storage. */ - public void beforeExchange(GridDhtPartitionsExchangeFuture discoEvt) throws IgniteCheckedException { - // No-op. + public boolean beforeExchange(GridDhtPartitionsExchangeFuture discoEvt) throws IgniteCheckedException { + return false; } /** - * @param fut Partition exchange future. + * Called when all partitions have been fully restored and pre-created on node start. + * + * @throws IgniteCheckedException If failed. */ + public void onStateRestored() throws IgniteCheckedException { + // No-op. + } + + /** + * @param fut Partition exchange future. + */ public void rebuildIndexesIfNeeded(GridDhtPartitionsExchangeFuture fut) { // No-op. } @@ -1090,4 +1101,13 @@ public boolean walEnabled(int grpId, boolean local) { public void walEnabled(int grpId, boolean enabled, boolean local) { // No-op. } + + /** + * Marks last checkpoint as inapplicable for WAL rebalance for given group {@code grpId}. + * + * @param grpId Group id. + */ + public void lastCheckpointInapplicableForWalRebalance(int grpId) { + // No-op. + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointEntry.java new file mode 100644 index 0000000000000..f6433e156070b --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointEntry.java @@ -0,0 +1,366 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.persistence.checkpoint; + +import java.lang.ref.SoftReference; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.pagemem.wal.WALIterator; +import org.apache.ignite.internal.pagemem.wal.WALPointer; +import org.apache.ignite.internal.pagemem.wal.record.CacheState; +import org.apache.ignite.internal.pagemem.wal.record.CheckpointRecord; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiTuple; +import org.jetbrains.annotations.Nullable; + +/** + * Class represents checkpoint state. + */ +public class CheckpointEntry { + /** Checkpoint timestamp. */ + private final long cpTs; + + /** Checkpoint end mark. */ + private final WALPointer cpMark; + + /** Checkpoint ID. */ + private final UUID cpId; + + /** State of groups and partitions snapshotted at the checkpoint begin. */ + private volatile SoftReference grpStateLazyStore; + + /** + * Checkpoint entry constructor. + * + * If {@code grpStates} is null then it will be inited lazy from wal pointer. + * + * @param cpTs Checkpoint timestamp. + * @param cpMark Checkpoint mark pointer. + * @param cpId Checkpoint ID. + * @param cacheGrpStates Cache groups states. + */ + public CheckpointEntry( + long cpTs, + WALPointer cpMark, + UUID cpId, + @Nullable Map cacheGrpStates + ) { + this.cpTs = cpTs; + this.cpMark = cpMark; + this.cpId = cpId; + this.grpStateLazyStore = new SoftReference<>(new GroupStateLazyStore(cacheGrpStates)); + } + + /** + * @return Checkpoint timestamp. + */ + public long timestamp() { + return cpTs; + } + + /** + * @return Checkpoint ID. + */ + public UUID checkpointId() { + return cpId; + } + + /** + * @return Checkpoint mark. + */ + public WALPointer checkpointMark() { + return cpMark; + } + + /** + * @param cctx Cache shred context. + */ + public Map groupState( + GridCacheSharedContext cctx + ) throws IgniteCheckedException { + GroupStateLazyStore store = initIfNeeded(cctx); + + return store.grpStates; + } + + /** + * @param cctx Cache shred context. + * @return Group lazy store. + */ + private GroupStateLazyStore initIfNeeded(GridCacheSharedContext cctx) throws IgniteCheckedException { + GroupStateLazyStore store = grpStateLazyStore.get(); + + if (store == null) { + store = new GroupStateLazyStore(); + + grpStateLazyStore = new SoftReference<>(store); + } + + store.initIfNeeded(cctx, cpMark); + + return store; + } + + /** + * @param cctx Cache shared context. + * @param grpId Cache group ID. + * @param part Partition ID. + * @return Partition counter or {@code null} if not found. + */ + public Long partitionCounter(GridCacheSharedContext cctx, int grpId, int part) { + GroupStateLazyStore store; + + try { + store = initIfNeeded(cctx); + } + catch (IgniteCheckedException e) { + return null; + } + + return store.partitionCounter(grpId, part); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "CheckpointEntry [id=" + cpId + ", timestamp=" + cpTs + ", ptr=" + cpMark + "]"; + } + + /** + * + */ + public static class GroupState { + /** */ + private int[] parts; + + /** */ + private long[] cnts; + + /** */ + private int idx; + + /** + * @param partsCnt Partitions count. + */ + private GroupState(int partsCnt) { + parts = new int[partsCnt]; + cnts = new long[partsCnt]; + } + + /** + * @param partId Partition ID to add. + * @param cntr Partition counter. + */ + public void addPartitionCounter(int partId, long cntr) { + if (idx == parts.length) + throw new IllegalStateException("Failed to add new partition to the partitions state " + + "(no enough space reserved) [partId=" + partId + ", reserved=" + parts.length + ']'); + + if (idx > 0) { + if (parts[idx - 1] >= partId) + throw new IllegalStateException("Adding partition in a wrong order [prev=" + parts[idx - 1] + + ", cur=" + partId + ']'); + } + + parts[idx] = partId; + + cnts[idx] = cntr; + + idx++; + } + + /** + * Gets partition counter by partition ID. + * + * @param partId Partition ID. + * @return Partition update counter (will return {@code -1} if partition is not present in the record). + */ + public long counterByPartition(int partId) { + int idx = indexByPartition(partId); + + return idx >= 0 ? cnts[idx] : -1; + } + + /** + * + */ + public long size(){ + return idx; + } + + /** + * @param partId Partition ID to search. + * @return Non-negative index of partition if found or negative value if not found. + */ + public int indexByPartition(int partId) { + return Arrays.binarySearch(parts, 0, idx, partId); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return "GroupState [cap=" + parts.length + ", size=" + idx + ']'; + } + } + + /** + * Group state lazy store. + */ + public static class GroupStateLazyStore { + /** */ + private static final AtomicIntegerFieldUpdater initGuardUpdater = + AtomicIntegerFieldUpdater.newUpdater(GroupStateLazyStore.class, "initGuard"); + + /** Cache states. Initialized lazily. */ + private volatile Map grpStates; + + /** */ + private final CountDownLatch latch; + + /** */ + @SuppressWarnings("unused") + private volatile int initGuard; + + /** Initialization exception. */ + private IgniteCheckedException initEx; + + /** + * Default constructor. + */ + private GroupStateLazyStore() { + this(null); + } + + /** + * @param cacheGrpStates Cache group state. + */ + private GroupStateLazyStore(Map cacheGrpStates) { + if (cacheGrpStates != null) { + initGuard = 1; + + latch = new CountDownLatch(0); + } + else + latch = new CountDownLatch(1); + + grpStates = remap(cacheGrpStates); + } + + /** + * @param stateRec Cache group state. + */ + private Map remap(Map stateRec) { + if (stateRec == null) + return Collections.emptyMap(); + + Map grpStates = new HashMap<>(stateRec.size()); + + for (Integer grpId : stateRec.keySet()) { + CacheState recState = stateRec.get(grpId); + + GroupState grpState = new GroupState(recState.size()); + + for (int i = 0; i < recState.size(); i++) { + byte partState = recState.stateByIndex(i); + + if (GridDhtPartitionState.fromOrdinal(partState) != GridDhtPartitionState.OWNING) + continue; + + grpState.addPartitionCounter( + recState.partitionByIndex(i), + recState.partitionCounterByIndex(i) + ); + } + + grpStates.put(grpId, grpState); + } + + return grpStates; + } + + /** + * @param grpId Group id. + * @param part Partition id. + * @return Partition counter. + */ + private Long partitionCounter(int grpId, int part) { + assert initGuard != 0 : initGuard; + + if (initEx != null || grpStates == null) + return null; + + GroupState state = grpStates.get(grpId); + + if (state != null) { + long cntr = state.counterByPartition(part); + + return cntr < 0 ? null : cntr; + } + + return null; + } + + /** + * @param cctx Cache shared context. + * @param ptr Checkpoint wal pointer. + * @throws IgniteCheckedException If failed to read WAL entry. + */ + private void initIfNeeded( + GridCacheSharedContext cctx, + WALPointer ptr + ) throws IgniteCheckedException { + if (initGuardUpdater.compareAndSet(this, 0, 1)) { + try (WALIterator it = cctx.wal().replay(ptr)) { + if (it.hasNextX()) { + IgniteBiTuple tup = it.nextX(); + + CheckpointRecord rec = (CheckpointRecord)tup.get2(); + + Map stateRec = rec.cacheGroupStates(); + + grpStates = remap(stateRec); + } + else + initEx = new IgniteCheckedException( + "Failed to find checkpoint record at the given WAL pointer: " + ptr); + } + catch (IgniteCheckedException e) { + initEx = e; + + throw e; + } + finally { + latch.countDown(); + } + } + else { + U.await(latch); + + if (initEx != null) + throw initEx; + } + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointEntryType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointEntryType.java new file mode 100644 index 0000000000000..66619bda84de8 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointEntryType.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.persistence.checkpoint; + +/** + * Checkpoint entry types. + */ +public enum CheckpointEntryType { + /** */ + START, + + /** */ + END +} 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 new file mode 100644 index 0000000000000..d6cc297d0fd4f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointHistory.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.persistence.checkpoint; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Set; +import java.util.concurrent.ConcurrentSkipListMap; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.configuration.DataStorageConfiguration; +import org.apache.ignite.internal.GridKernalContext; +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.wal.FileWALPointer; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE; + +/** + * Checkpoint history. Holds chronological ordered map with {@link CheckpointEntry CheckpointEntries}. + * Data is loaded from corresponding checkpoint directory. + * This directory holds files for checkpoint start and end. + */ +public class CheckpointHistory { + /** Logger. */ + private final IgniteLogger log; + + /** Cache shared context. */ + private final GridCacheSharedContext cctx; + + /** + * Maps checkpoint's timestamp (from CP file name) to CP entry. + * Using TS provides historical order of CP entries in map ( first is oldest ) + */ + private final NavigableMap histMap = new ConcurrentSkipListMap<>(); + + /** The maximal number of checkpoints hold in memory. */ + private final int maxCpHistMemSize; + + /** + * Constructor. + * + * @param ctx Context. + */ + public CheckpointHistory(GridKernalContext ctx) { + cctx = ctx.cache().context(); + log = ctx.log(getClass()); + + DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration(); + + maxCpHistMemSize = Math.min(dsCfg.getWalHistorySize(), + IgniteSystemProperties.getInteger(IGNITE_PDS_MAX_CHECKPOINT_MEMORY_HISTORY_SIZE, 100)); + } + + /** + * @param checkpoints Checkpoints. + */ + public void initialize(List checkpoints) { + for (CheckpointEntry e : checkpoints) + histMap.put(e.timestamp(), e); + } + + /** + * @param cpTs Checkpoint timestamp. + * @return Initialized entry. + * @throws IgniteCheckedException If failed to initialize entry. + */ + private CheckpointEntry entry(Long cpTs) throws IgniteCheckedException { + CheckpointEntry entry = histMap.get(cpTs); + + if (entry == null) + throw new IgniteCheckedException("Checkpoint entry was removed: " + cpTs); + + return entry; + } + + /** + * @return First checkpoint entry if exists. Otherwise {@code null}. + */ + public CheckpointEntry firstCheckpoint() { + Map.Entry entry = histMap.firstEntry(); + + return entry != null ? entry.getValue() : null; + } + + /** + * @return Last checkpoint entry if exists. Otherwise {@code null}. + */ + public CheckpointEntry lastCheckpoint() { + Map.Entry entry = histMap.lastEntry(); + + return entry != null ? entry.getValue() : null; + } + + /** + * @return First checkpoint WAL pointer if exists. Otherwise {@code null}. + */ + public WALPointer firstCheckpointPointer() { + CheckpointEntry entry = firstCheckpoint(); + + return entry != null ? entry.checkpointMark() : null; + } + + /** + * @return Collection of checkpoint timestamps. + */ + public Collection checkpoints(boolean descending) { + if (descending) + return histMap.descendingKeySet(); + + return histMap.keySet(); + } + + /** + * + */ + public Collection checkpoints() { + return checkpoints(false); + } + + /** + * Adds checkpoint entry after the corresponding WAL record has been written to WAL. The checkpoint itself + * is not finished yet. + * + * @param entry Entry to add. + */ + public void addCheckpoint(CheckpointEntry entry) { + histMap.put(entry.timestamp(), entry); + } + + /** + * @return {@code true} if there is space for next checkpoint. + */ + public boolean hasSpace() { + return histMap.size() + 1 <= maxCpHistMemSize; + } + + /** + * Clears checkpoint history after WAL truncation. + * + * @return List of checkpoint entries removed from history. + */ + public List onWalTruncated(WALPointer ptr) { + List removed = new ArrayList<>(); + + FileWALPointer highBound = (FileWALPointer)ptr; + + for (CheckpointEntry cpEntry : histMap.values()) { + FileWALPointer cpPnt = (FileWALPointer)cpEntry.checkpointMark(); + + if (highBound.compareTo(cpPnt) <= 0) + break; + + if (cctx.wal().reserved(cpEntry.checkpointMark())) { + U.warn(log, "Could not clear historyMap due to WAL reservation on cp: " + cpEntry + + ", history map size is " + histMap.size()); + + break; + } + + histMap.remove(cpEntry.timestamp()); + + removed.add(cpEntry); + } + + return removed; + } + + /** + * Clears checkpoint history after checkpoint finish. + * + * @return List of checkpoints removed from history. + */ + public List onCheckpointFinished(GridCacheDatabaseSharedManager.Checkpoint chp, boolean truncateWal) { + List removed = new ArrayList<>(); + + int deleted = 0; + + while (histMap.size() > maxCpHistMemSize) { + Map.Entry entry = histMap.firstEntry(); + + CheckpointEntry cpEntry = entry.getValue(); + + 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()); + + break; + } + + if (truncateWal) + deleted += cctx.wal().truncate(null, cpEntry.checkpointMark()); + + histMap.remove(entry.getKey()); + + removed.add(cpEntry); + } + + chp.walFilesDeleted(deleted); + + return removed; + } + + /** + * Tries to search for a WAL pointer for the given partition counter start. + * + * @param grpId Cache group ID. + * @param part Partition ID. + * @param partCntrSince Partition counter or {@code null} to search for minimal counter. + * @return Checkpoint entry or {@code null} if failed to search. + */ + @Nullable public WALPointer searchPartitionCounter(int grpId, int part, long partCntrSince) { + CheckpointEntry entry = searchCheckpointEntry(grpId, part, partCntrSince); + + if (entry == null) + return null; + + return entry.checkpointMark(); + } + + /** + * Tries to search for a WAL pointer for the given partition counter start. + * + * @param grpId Cache group ID. + * @param part Partition ID. + * @param partCntrSince Partition counter or {@code null} to search for minimal counter. + * @return Checkpoint entry or {@code null} if failed to search. + */ + @Nullable public CheckpointEntry searchCheckpointEntry(int grpId, int part, long partCntrSince) { + for (Long cpTs : checkpoints(true)) { + try { + CheckpointEntry entry = entry(cpTs); + + Long foundCntr = entry.partitionCounter(cctx, grpId, part); + + if (foundCntr != null && foundCntr <= partCntrSince) + return entry; + } + catch (IgniteCheckedException ignore) { + break; + } + } + + return null; + } + + /** + * Finds and reserves earliest valid checkpoint for each of given groups and partitions. + * + * @param groupsAndPartitions Groups and partitions to find and reserve earliest valid checkpoint. + * + * @return Map (groupId, Map (partitionId, earliest valid checkpoint to history search)). + */ + public Map> searchAndReserveCheckpoints( + final Map> groupsAndPartitions + ) { + if (F.isEmpty(groupsAndPartitions)) + return Collections.emptyMap(); + + final Map> res = new HashMap<>(); + + CheckpointEntry prevReserved = null; + + // Iterate over all possible checkpoints starting from latest and moving to earliest. + for (Long cpTs : checkpoints(true)) { + CheckpointEntry chpEntry = null; + + try { + chpEntry = entry(cpTs); + + boolean reserved = cctx.wal().reserve(chpEntry.checkpointMark()); + + // If checkpoint WAL history can't be reserved, stop searching. + if (!reserved) + break; + + for (Integer grpId : groupsAndPartitions.keySet()) + if (!isCheckpointApplicableForGroup(grpId, chpEntry)) + groupsAndPartitions.remove(grpId); + + // All groups are no more applicable, release history and stop searching. + if (groupsAndPartitions.isEmpty()) { + cctx.wal().release(chpEntry.checkpointMark()); + + break; + } + + // Release previous checkpoint marker. + if (prevReserved != null) + cctx.wal().release(prevReserved.checkpointMark()); + + prevReserved = chpEntry; + + for (Map.Entry state : chpEntry.groupState(cctx).entrySet()) { + int grpId = state.getKey(); + CheckpointEntry.GroupState cpGroupState = state.getValue(); + + Set applicablePartitions = groupsAndPartitions.get(grpId); + + if (F.isEmpty(applicablePartitions)) + continue; + + Set inapplicablePartitions = null; + + for (Integer partId : applicablePartitions) { + int pIdx = cpGroupState.indexByPartition(partId); + + if (pIdx >= 0) + res.computeIfAbsent(grpId, k -> new HashMap<>()).put(partId, chpEntry); + else { + if (inapplicablePartitions == null) + inapplicablePartitions = new HashSet<>(); + + // Partition is no more applicable for history search, exclude partition from searching. + inapplicablePartitions.add(partId); + } + } + + if (!F.isEmpty(inapplicablePartitions)) + for (Integer partId : inapplicablePartitions) + applicablePartitions.remove(partId); + } + + // Remove groups from search with empty set of applicable partitions. + for (Map.Entry> e : groupsAndPartitions.entrySet()) + if (e.getValue().isEmpty()) + groupsAndPartitions.remove(e.getKey()); + } + catch (IgniteCheckedException ex) { + U.error(log, "Failed to process checkpoint: " + (chpEntry != null ? chpEntry : "none"), ex); + } + } + + return res; + } + + /** + * Checkpoint is not applicable when: + * 1) WAL was disabled somewhere after given checkpoint. + * 2) Checkpoint doesn't contain specified {@code grpId}. + * + * @param grpId Group ID. + * @param cp Checkpoint. + */ + private boolean isCheckpointApplicableForGroup(int grpId, CheckpointEntry cp) throws IgniteCheckedException { + GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager) cctx.database(); + + if (dbMgr.isCheckpointInapplicableForWalRebalance(cp.timestamp(), grpId)) + return false; + + if (!cp.groupState(cctx).containsKey(grpId)) + return false; + + return true; + } +} 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 5efd5ee6c44b2..15b9109a8f01a 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 @@ -832,7 +832,7 @@ private void checkWalRolloverRequiredDuringInactivityPeriod() { } /** {@inheritDoc} */ - @Override public void release(WALPointer start) throws IgniteCheckedException { + @Override public void release(WALPointer start) { assert start != null && start instanceof FileWALPointer : "Invalid start pointer: " + start; if (mode == WALMode.NONE) @@ -1997,14 +1997,8 @@ private void deleteObsoleteRawSegments() { Thread.currentThread().interrupt(); } finally { - try { - if (currReservedSegment != -1) - release(new FileWALPointer(currReservedSegment, 0, 0)); - } - catch (IgniteCheckedException e) { - U.error(log, "Can't release raw WAL segment [idx=" + currReservedSegment + - "] after compression", e); - } + if (currReservedSegment != -1) + release(new FileWALPointer(currReservedSegment, 0, 0)); } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTask.java index 7edcea9d459b2..bdadf970bbd81 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/misc/VisorWalTask.java @@ -186,7 +186,7 @@ Collection getUnusedWalSegments( GridCacheDatabaseSharedManager dbMgr, FileWriteAheadLogManager wal ) throws IgniteCheckedException{ - WALPointer lowBoundForTruncate = dbMgr.checkpointHistory().lowCheckpointBound(); + WALPointer lowBoundForTruncate = dbMgr.checkpointHistory().firstCheckpointPointer(); if (lowBoundForTruncate == null) return Collections.emptyList(); @@ -227,7 +227,7 @@ Collection deleteUnusedWalSegments( GridCacheDatabaseSharedManager dbMgr, FileWriteAheadLogManager wal ) throws IgniteCheckedException { - WALPointer lowBoundForTruncate = dbMgr.checkpointHistory().lowCheckpointBound(); + WALPointer lowBoundForTruncate = dbMgr.checkpointHistory().firstCheckpointPointer(); if (lowBoundForTruncate == null) return Collections.emptyList(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsAtomicCacheHistoricalRebalancingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsAtomicCacheHistoricalRebalancingTest.java index a090381ff9863..f06494b965235 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsAtomicCacheHistoricalRebalancingTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsAtomicCacheHistoricalRebalancingTest.java @@ -18,11 +18,22 @@ package org.apache.ignite.internal.processors.cache.persistence; import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalRebalanceTest; /** * */ public class IgnitePdsAtomicCacheHistoricalRebalancingTest extends IgnitePdsAtomicCacheRebalancingTest { + /** {@inheritDoc */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setCommunicationSpi(new IgniteWalRebalanceTest.WalRebalanceCheckingCommunicationSpi()); + + return cfg; + } + /** {@inheritDoc */ @Override protected void beforeTest() throws Exception { // Use rebalance from WAL if possible. @@ -33,8 +44,16 @@ public class IgnitePdsAtomicCacheHistoricalRebalancingTest extends IgnitePdsAtom /** {@inheritDoc */ @Override protected void afterTest() throws Exception { + boolean walRebalanceInvoked = !IgniteWalRebalanceTest.WalRebalanceCheckingCommunicationSpi.allRebalances() + .isEmpty(); + + IgniteWalRebalanceTest.WalRebalanceCheckingCommunicationSpi.cleanup(); + System.clearProperty(IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD); super.afterTest(); + + if (!walRebalanceInvoked) + throw new AssertionError("WAL rebalance hasn't been invoked."); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java index c0c2be9f86943..347412df79965 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java @@ -18,9 +18,11 @@ package org.apache.ignite.internal.processors.cache.persistence; import java.io.Serializable; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; @@ -47,7 +49,6 @@ import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; -import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; @@ -66,12 +67,21 @@ public abstract class IgnitePdsCacheRebalancingAbstractTest extends GridCommonAb /** */ private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); - /** Cache name. */ - private static final String cacheName = "cache"; + /** Default cache. */ + private static final String CACHE = "cache"; + + /** Cache with node filter. */ + private static final String FILTERED_CACHE = "filtered"; + + /** Cache with enabled indexes. */ + private static final String INDEXED_CACHE = "indexed"; /** */ protected boolean explicitTx; + /** Set to enable filtered cache on topology. */ + private boolean filteredCacheEnabled; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); @@ -80,18 +90,19 @@ public abstract class IgnitePdsCacheRebalancingAbstractTest extends GridCommonAb cfg.setRebalanceThreadPoolSize(2); - CacheConfiguration ccfg1 = cacheConfiguration(cacheName) + CacheConfiguration ccfg1 = cacheConfiguration(CACHE) .setPartitionLossPolicy(PartitionLossPolicy.READ_WRITE_SAFE) - .setBackups(1) + .setBackups(2) .setRebalanceMode(CacheRebalanceMode.ASYNC) .setIndexedTypes(Integer.class, Integer.class) .setAffinity(new RendezvousAffinityFunction(false, 32)) .setRebalanceBatchesPrefetchCount(2) .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); - CacheConfiguration ccfg2 = cacheConfiguration("indexed"); - ccfg2.setBackups(1); - ccfg2.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + CacheConfiguration ccfg2 = cacheConfiguration(INDEXED_CACHE) + .setBackups(2) + .setAffinity(new RendezvousAffinityFunction(false, 32)) + .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); QueryEntity qryEntity = new QueryEntity(Integer.class.getName(), TestValue.class.getName()); @@ -108,36 +119,34 @@ public abstract class IgnitePdsCacheRebalancingAbstractTest extends GridCommonAb ccfg2.setQueryEntities(Collections.singleton(qryEntity)); - // Do not start filtered cache on coordinator. - if (gridName.endsWith("0")) { - cfg.setCacheConfiguration(ccfg1, ccfg2); - } - else { - CacheConfiguration ccfg3 = cacheConfiguration("filtered"); - ccfg3.setPartitionLossPolicy(PartitionLossPolicy.READ_ONLY_SAFE); - ccfg3.setBackups(1); - ccfg3.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); - ccfg3.setNodeFilter(new CoordinatorNodeFilter()); - - cfg.setCacheConfiguration(ccfg1, ccfg2, ccfg3); - } - - DataStorageConfiguration memCfg = new DataStorageConfiguration(); + List cacheCfgs = new ArrayList<>(); + cacheCfgs.add(ccfg1); + cacheCfgs.add(ccfg2); - memCfg.setConcurrencyLevel(Runtime.getRuntime().availableProcessors() * 4); - memCfg.setPageSize(1024); - memCfg.setWalMode(WALMode.LOG_ONLY); + if (filteredCacheEnabled && !gridName.endsWith("0")) { + CacheConfiguration ccfg3 = cacheConfiguration(FILTERED_CACHE) + .setPartitionLossPolicy(PartitionLossPolicy.READ_ONLY_SAFE) + .setBackups(2) + .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC) + .setNodeFilter(new CoordinatorNodeFilter()); - DataRegionConfiguration memPlcCfg = new DataRegionConfiguration(); + cacheCfgs.add(ccfg3); + } - memPlcCfg.setName("dfltDataRegion"); - memPlcCfg.setMaxSize(150 * 1024 * 1024); - memPlcCfg.setInitialSize(100 * 1024 * 1024); - memPlcCfg.setPersistenceEnabled(true); + cfg.setCacheConfiguration(asArray(cacheCfgs)); - memCfg.setDefaultDataRegionConfiguration(memPlcCfg); + DataStorageConfiguration dsCfg = new DataStorageConfiguration() + .setConcurrencyLevel(Runtime.getRuntime().availableProcessors() * 4) + .setPageSize(1024) + .setCheckpointFrequency(10 * 1000) + .setWalMode(WALMode.LOG_ONLY) + .setDefaultDataRegionConfiguration(new DataRegionConfiguration() + .setName("dfltDataRegion") + .setPersistenceEnabled(true) + .setMaxSize(512 * 1024 * 1024) + ); - cfg.setDataStorageConfiguration(memCfg); + cfg.setDataStorageConfiguration(dsCfg); cfg.setDiscoverySpi( new TcpDiscoverySpi() @@ -147,6 +156,17 @@ public abstract class IgnitePdsCacheRebalancingAbstractTest extends GridCommonAb return cfg; } + /** + * @param cacheCfgs Cache cfgs. + */ + private static CacheConfiguration[] asArray(List cacheCfgs) { + CacheConfiguration[] res = new CacheConfiguration[cacheCfgs.size()]; + for (int i = 0; i < res.length; i++) + res[i] = cacheCfgs.get(i); + + return res; + } + /** {@inheritDoc} */ @Override protected long getTestTimeout() { return 20 * 60 * 1000; @@ -185,15 +205,15 @@ public abstract class IgnitePdsCacheRebalancingAbstractTest extends GridCommonAb public void testRebalancingOnRestart() throws Exception { Ignite ignite0 = startGrid(0); - ignite0.cluster().active(true); - startGrid(1); IgniteEx ignite2 = startGrid(2); + ignite0.cluster().active(true); + awaitPartitionMapExchange(); - IgniteCache cache1 = ignite0.cache(cacheName); + IgniteCache cache1 = ignite0.cache(CACHE); for (int i = 0; i < 5000; i++) cache1.put(i, i); @@ -222,7 +242,7 @@ public void testRebalancingOnRestart() throws Exception { awaitPartitionMapExchange(); - IgniteCache cache3 = ignite2.cache(cacheName); + IgniteCache cache3 = ignite2.cache(CACHE); for (int i = 0; i < 100; i++) assertEquals(String.valueOf(i), (Integer)(i * 2), cache3.get(i)); @@ -243,28 +263,21 @@ public void testRebalancingOnRestartAfterCheckpoint() throws Exception { ignite0.cluster().active(true); - ignite0.cache(cacheName).rebalance().get(); - ignite1.cache(cacheName).rebalance().get(); - ignite2.cache(cacheName).rebalance().get(); - ignite3.cache(cacheName).rebalance().get(); - awaitPartitionMapExchange(); - IgniteCache cache1 = ignite0.cache(cacheName); + IgniteCache cache1 = ignite0.cache(CACHE); for (int i = 0; i < 1000; i++) cache1.put(i, i); - ignite0.context().cache().context().database().waitForCheckpoint("test"); - ignite1.context().cache().context().database().waitForCheckpoint("test"); + forceCheckpoint(ignite0); + forceCheckpoint(ignite1); info("++++++++++ After checkpoint"); ignite2.close(); ignite3.close(); - resetBaselineTopology(); - ignite0.resetLostPartitions(Collections.singletonList(cache1.getName())); assert cache1.lostPartitions().isEmpty(); @@ -281,116 +294,20 @@ public void testRebalancingOnRestartAfterCheckpoint() throws Exception { info(">>> Done puts..."); - ignite2 = startGrid(2); - ignite3 = startGrid(3); - - ignite2.cache(cacheName).rebalance().get(); - ignite3.cache(cacheName).rebalance().get(); - - IgniteCache cache2 = ignite2.cache(cacheName); - IgniteCache cache3 = ignite3.cache(cacheName); - - for (int i = 0; i < 100; i++) { - assertEquals(String.valueOf(i), (Integer)(i * 2), cache2.get(i)); - assertEquals(String.valueOf(i), (Integer)(i * 2), cache3.get(i)); - } - } - - /** - * Test that all data is correctly restored after non-graceful restart. - * - * @throws Exception If fails. - */ - public void testDataCorrectnessAfterRestart() throws Exception { - IgniteEx ignite1 = (IgniteEx)G.start(getConfiguration("test1")); - IgniteEx ignite2 = (IgniteEx)G.start(getConfiguration("test2")); - IgniteEx ignite3 = (IgniteEx)G.start(getConfiguration("test3")); - IgniteEx ignite4 = (IgniteEx)G.start(getConfiguration("test4")); - - ignite1.cluster().active(true); + startGrid(2); + startGrid(3); awaitPartitionMapExchange(); - IgniteCache cache1 = ignite1.cache(cacheName); - - for (int i = 0; i < 100; i++) - cache1.put(i, i); - - ignite1.close(); - ignite2.close(); - ignite3.close(); - ignite4.close(); - - ignite1 = (IgniteEx)G.start(getConfiguration("test1")); - ignite2 = (IgniteEx)G.start(getConfiguration("test2")); - ignite3 = (IgniteEx)G.start(getConfiguration("test3")); - ignite4 = (IgniteEx)G.start(getConfiguration("test4")); + ignite2 = grid(2); + ignite3 = grid(3); - ignite1.cluster().active(true); - - awaitPartitionMapExchange(); - - cache1 = ignite1.cache(cacheName); - IgniteCache cache2 = ignite2.cache(cacheName); - IgniteCache cache3 = ignite3.cache(cacheName); - IgniteCache cache4 = ignite4.cache(cacheName); + IgniteCache cache2 = ignite2.cache(CACHE); + IgniteCache cache3 = ignite3.cache(CACHE); for (int i = 0; i < 100; i++) { - assert cache1.get(i).equals(i); - assert cache2.get(i).equals(i); - assert cache3.get(i).equals(i); - assert cache4.get(i).equals(i); - } - } - - /** - * Test that partitions are marked as lost when all owners leave cluster, but recover after nodes rejoin. - * - * @throws Exception If fails. - */ - public void testPartitionLossAndRecover() throws Exception { - Ignite ignite1 = startGrid(0); - Ignite ignite2 = startGrid(1); - Ignite ignite3 = startGrid(2); - Ignite ignite4 = startGrid(3); - - ignite1.cluster().active(true); - - awaitPartitionMapExchange(); - - IgniteCache cache1 = ignite1.cache(cacheName); - - final int offset = 10; - - for (int i = 0; i < 100; i++) - cache1.put(String.valueOf(i), String.valueOf(i + offset)); - - ignite3.close(); - ignite4.close(); - - awaitPartitionMapExchange(); - - assert !ignite1.cache(cacheName).lostPartitions().isEmpty(); - - ignite3 = startGrid(2); - ignite4 = startGrid(3); - - ignite1.resetLostPartitions(Collections.singletonList(cacheName)); - - IgniteCache cache2 = ignite2.cache(cacheName); - IgniteCache cache3 = ignite3.cache(cacheName); - IgniteCache cache4 = ignite4.cache(cacheName); - - //Thread.sleep(5_000); - - for (int i = 0; i < 100; i++) { - String key = String.valueOf(i); - String expected = String.valueOf(i + offset); - - assertEquals(expected, cache1.get(key)); - assertEquals(expected, cache2.get(key)); - assertEquals(expected, cache3.get(key)); - assertEquals(expected, cache4.get(key)); + assertEquals(String.valueOf(i), (Integer)(i * 2), cache2.get(i)); + assertEquals(String.valueOf(i), (Integer)(i * 2), cache3.get(i)); } } @@ -401,26 +318,26 @@ public void testTopologyChangesWithConstantLoad() throws Exception { final long timeOut = U.currentTimeMillis() + 10 * 60 * 1000; final int entriesCnt = 10_000; - int maxNodesCount = 4; - int topChanges = 20; - final String cacheName = "indexed"; + final int maxNodesCnt = 4; + final int topChanges = 50; final AtomicBoolean stop = new AtomicBoolean(); + final AtomicBoolean suspend = new AtomicBoolean(); final ConcurrentMap map = new ConcurrentHashMap<>(); - Ignite ignite = startGrid(0); + Ignite ignite = startGridsMultiThreaded(4); ignite.cluster().active(true); - IgniteCache cache = ignite.cache(cacheName); + IgniteCache cache = ignite.cache(INDEXED_CACHE); for (int i = 0; i < entriesCnt; i++) { cache.put(i, new TestValue(i, i)); map.put(i, new TestValue(i, i)); } - final AtomicInteger nodesCnt = new AtomicInteger(); + final AtomicInteger nodesCnt = new AtomicInteger(4); IgniteInternalFuture fut = runMultiThreadedAsync(new Callable() { @Override public Void call() throws Exception { @@ -428,6 +345,12 @@ public void testTopologyChangesWithConstantLoad() throws Exception { if (stop.get()) return null; + if (suspend.get()) { + U.sleep(10); + + continue; + } + int k = ThreadLocalRandom.current().nextInt(entriesCnt); int v1 = ThreadLocalRandom.current().nextInt(); int v2 = ThreadLocalRandom.current().nextInt(); @@ -456,7 +379,7 @@ public void testTopologyChangesWithConstantLoad() throws Exception { tx = ignite.transactions().txStart(); try { - ignite.cache(cacheName).put(k, new TestValue(v1, v2)); + ignite.cache(INDEXED_CACHE).put(k, new TestValue(v1, v2)); } catch (Exception ignored) { success = false; @@ -478,8 +401,10 @@ public void testTopologyChangesWithConstantLoad() throws Exception { } }, 1, "load-runner"); + boolean[] changes = new boolean[] {false, false, true, true}; + try { - for (int i = 0; i < topChanges; i++) { + for (int it = 0; it < topChanges; it++) { if (U.currentTimeMillis() > timeOut) break; @@ -487,21 +412,30 @@ public void testTopologyChangesWithConstantLoad() throws Exception { boolean add; - if (nodesCnt.get() <= maxNodesCount / 2) + if (it < changes.length) + add = changes[it]; + else if (nodesCnt.get() <= maxNodesCnt / 2) add = true; - else if (nodesCnt.get() > maxNodesCount) + else if (nodesCnt.get() >= maxNodesCnt) add = false; else // More chance that node will be added add = ThreadLocalRandom.current().nextInt(3) <= 1; if (add) - startGrid(nodesCnt.incrementAndGet()); + startGrid(nodesCnt.getAndIncrement()); else - stopGrid(nodesCnt.getAndDecrement()); + stopGrid(nodesCnt.decrementAndGet()); awaitPartitionMapExchange(); - cache.rebalance().get(); + suspend.set(true); + + U.sleep(200); + + for (Map.Entry entry : map.entrySet()) + assertEquals(it + " " + Integer.toString(entry.getKey()), entry.getValue(), cache.get(entry.getKey())); + + suspend.set(false); } } finally { @@ -520,14 +454,21 @@ else if (nodesCnt.get() > maxNodesCount) * @throws Exception If failed. */ public void testForceRebalance() throws Exception { - testForceRebalance(cacheName); + testForceRebalance(CACHE); } /** * @throws Exception If failed. */ public void testForceRebalanceClientTopology() throws Exception { - testForceRebalance("filtered"); + filteredCacheEnabled = true; + + try { + testForceRebalance(FILTERED_CACHE); + } + finally { + filteredCacheEnabled = false; + } } /** @@ -579,63 +520,52 @@ public void testPartitionCounterConsistencyOnUnstableTopology() throws Exception ig.cluster().active(true); - int k = 0; + int keys = 0; - try (IgniteDataStreamer ds = ig.dataStreamer(cacheName)) { + try (IgniteDataStreamer ds = ig.dataStreamer(CACHE)) { ds.allowOverwrite(true); - for (int k0 = k; k < k0 + 10_000; k++) - ds.addData(k, k); + for (; keys < 10_000; keys++) + ds.addData(keys, keys); } - for (int t = 0; t < 5; t++) { - int t0 = t; + for (int it = 0; it < 10; it++) { + final int it0 = it; + IgniteInternalFuture fut = GridTestUtils.runAsync(() -> { try { stopGrid(3); - forceCheckpoint(); - U.sleep(500); // Wait for data load. - IgniteEx ig0 = startGrid(3); + startGrid(3); - U.sleep(2000); // Wait for node join. + U.sleep(500); // Wait for data load. - if (t0 % 2 == 1) { + if (it0 % 2 != 0) { stopGrid(2); - awaitPartitionMapExchange(); - - forceCheckpoint(); + U.sleep(500); // Wait for data load. startGrid(2); - - awaitPartitionMapExchange(); } - ig0.cache(cacheName).rebalance().get(); + awaitPartitionMapExchange(); } catch (Exception e) { error("Unable to start/stop grid", e); + throw new RuntimeException(e); } }); - try (IgniteDataStreamer ds = ig.dataStreamer(cacheName)) { - ds.allowOverwrite(true); - - while (!fut.isDone()) { - int k0 = k; + IgniteCache cache = ig.cache(CACHE); - for (;k < k0 + 3; k++) - ds.addData(k, k); + while (!fut.isDone()) { + int nextKeys = keys + 10; - U.sleep(10); - } - } - catch (Exception e) { - log.error("Unable to write data", e); + for (;keys < nextKeys; keys++) + cache.put(keys, keys); } fut.get(); @@ -647,18 +577,18 @@ public void testPartitionCounterConsistencyOnUnstableTopology() throws Exception for (int g = 0; g < 4; g++) { IgniteEx ig0 = grid(g); - for (GridDhtLocalPartition part : ig0.cachex(cacheName).context().topology().currentLocalPartitions()) { + for (GridDhtLocalPartition part : ig0.cachex(CACHE).context().topology().currentLocalPartitions()) { if (cntrs.containsKey(part.id())) assertEquals(String.valueOf(part.id()), (long) cntrs.get(part.id()), part.updateCounter()); else cntrs.put(part.id(), part.updateCounter()); } - for (int k0 = 0; k0 < k; k0++) - assertEquals(String.valueOf(k0) + " " + g, k0, ig0.cache(cacheName).get(k0)); + for (int k0 = 0; k0 < keys; k0++) + assertEquals(String.valueOf(k0) + " " + g, k0, ig0.cache(CACHE).get(k0)); } - assertEquals(ig.affinity(cacheName).partitions(), cntrs.size()); + assertEquals(ig.affinity(CACHE).partitions(), cntrs.size()); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsTxCacheHistoricalRebalancingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsTxCacheHistoricalRebalancingTest.java deleted file mode 100644 index 179c8e0fcf399..0000000000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsTxCacheHistoricalRebalancingTest.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.ignite.internal.processors.cache.persistence; - -import org.apache.ignite.IgniteSystemProperties; - -/** - * - */ -public class IgnitePdsTxCacheHistoricalRebalancingTest extends IgnitePdsTxCacheRebalancingTest { - /** {@inheritDoc */ - @Override protected void beforeTest() throws Exception { - // Use rebalance from WAL if possible. - System.setProperty(IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD, "0"); - - super.beforeTest(); - } - - /** {@inheritDoc */ - @Override protected void afterTest() throws Exception { - System.clearProperty(IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD); - - super.afterTest(); - } -} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java index ca46a75e09181..f365448d40f75 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java @@ -24,7 +24,6 @@ import java.nio.file.OpenOption; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; -import com.sun.org.apache.regexp.internal.RE; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteException; @@ -39,6 +38,7 @@ import org.apache.ignite.internal.managers.communication.GridIoMessage; import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage; +import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointHistory; 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.util.lang.GridAbsPredicate; @@ -214,7 +214,7 @@ private void doTestSimple() throws Exception { IgniteEx newIgnite = startGrid(3); - final GridCacheDatabaseSharedManager.CheckpointHistory cpHistory = + final CheckpointHistory cpHistory = ((GridCacheDatabaseSharedManager)newIgnite.context().cache().context().database()).checkpointHistory(); GridTestUtils.waitForCondition(new GridAbsPredicate() { 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/IgnitePdsUnusedWalSegmentsTest.java index 0f522544107e4..06a9ec271ef25 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/IgnitePdsUnusedWalSegmentsTest.java @@ -27,6 +27,7 @@ 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.FileWALPointer; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; @@ -49,6 +50,8 @@ public class IgnitePdsUnusedWalSegmentsTest extends GridCommonAbstractTest { IgniteConfiguration cfg = super.getConfiguration(gridName); + cfg.setConsistentId(gridName); + cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER)); CacheConfiguration ccfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME); @@ -102,7 +105,7 @@ public void testWalManagerRangeReservation() throws Exception { assertTrue("Expected that at least resIdx greater than 0, real is " + resIdx, resIdx > 0); - FileWALPointer lowPtr = (FileWALPointer)dbMgr.checkpointHistory().lowCheckpointBound(); + FileWALPointer lowPtr = (FileWALPointer)dbMgr.checkpointHistory().firstCheckpointPointer(); assertTrue("Expected that dbMbr returns valid resIdx", lowPtr.index() == resIdx); @@ -136,7 +139,7 @@ public void testUnusedWalTruncate() throws Exception { assertTrue("Expected that at least resIdx greater than 0, real is " + resIdx, resIdx > 0); - FileWALPointer lowPtr = (FileWALPointer) dbMgr.checkpointHistory().lowCheckpointBound(); + FileWALPointer lowPtr = (FileWALPointer) dbMgr.checkpointHistory().firstCheckpointPointer(); assertTrue("Expected that dbMbr returns valid resIdx", lowPtr.index() == resIdx); @@ -186,17 +189,10 @@ private IgniteEx prepareGrid(int cnt) throws Exception { * Get index of reserved WAL segment by checkpointer. * * @param dbMgr Database shared manager. - * @throws Exception If failed. */ - private long getReservedWalSegmentIndex(GridCacheDatabaseSharedManager dbMgr) throws Exception{ - GridCacheDatabaseSharedManager.CheckpointHistory cpHist = dbMgr.checkpointHistory(); - - Object histMap = GridTestUtils.getFieldValue(cpHist, "histMap"); - - Object cpEntry = GridTestUtils.getFieldValue(GridTestUtils.invoke(histMap, "firstEntry"), "value"); - - FileWALPointer walPtr = GridTestUtils.getFieldValue(cpEntry, "cpMark"); + private long getReservedWalSegmentIndex(GridCacheDatabaseSharedManager dbMgr) { + CheckpointHistory cpHist = dbMgr.checkpointHistory(); - return walPtr.index(); + return ((FileWALPointer) cpHist.firstCheckpointPointer()).index(); } } 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 23dda265fb5eb..28801cd6c20bf 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,32 +17,52 @@ package org.apache.ignite.internal.processors.cache.persistence.db.wal; +import java.util.Collections; +import java.util.HashMap; +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.IgniteException; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.CacheRebalanceMode; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; 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.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.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtDemandedPartitionsMap; +import org.apache.ignite.internal.util.GridConcurrentHashSet; 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.plugin.extensions.communication.Message; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.Assert; import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD; /** - * Historic WAL rebalance base test. + * Historical WAL rebalance base test. */ public class IgniteWalRebalanceTest extends GridCommonAbstractTest { /** Cache name. */ private static final String CACHE_NAME = "cache"; + /** Partitions count. */ + private static final int PARTS_CNT = 32; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { System.setProperty(IGNITE_PDS_WAL_REBALANCE_THRESHOLD, "0"); //to make all rebalance wal-based @@ -51,25 +71,24 @@ public class IgniteWalRebalanceTest extends GridCommonAbstractTest { cfg.setConsistentId(gridName); - CacheConfiguration ccfg = new CacheConfiguration<>(CACHE_NAME); - - ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); - - ccfg.setRebalanceMode(CacheRebalanceMode.ASYNC); - - ccfg.setCacheMode(CacheMode.REPLICATED); - - ccfg.setAffinity(new RendezvousAffinityFunction(false, 32)); + CacheConfiguration ccfg = new CacheConfiguration<>(CACHE_NAME) + .setAtomicityMode(CacheAtomicityMode.ATOMIC) + .setRebalanceMode(CacheRebalanceMode.ASYNC) + .setCacheMode(CacheMode.REPLICATED) + .setAffinity(new RendezvousAffinityFunction(false, PARTS_CNT)); cfg.setCacheConfiguration(ccfg); DataStorageConfiguration dbCfg = new DataStorageConfiguration() .setWalHistorySize(Integer.MAX_VALUE) .setWalMode(WALMode.LOG_ONLY) + .setCheckpointFrequency(15 * 60 * 1000) .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setPersistenceEnabled(true)); cfg.setDataStorageConfiguration(dbCfg); + cfg.setCommunicationSpi(new WalRebalanceCheckingCommunicationSpi()); + return cfg; } @@ -83,10 +102,19 @@ public class IgniteWalRebalanceTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { System.clearProperty(IGNITE_PDS_WAL_REBALANCE_THRESHOLD); + System.clearProperty(IgniteSystemProperties.IGNITE_DISABLE_WAL_DURING_REBALANCING); + + boolean walRebalanceInvoked = !IgniteWalRebalanceTest.WalRebalanceCheckingCommunicationSpi.allRebalances() + .isEmpty(); + + IgniteWalRebalanceTest.WalRebalanceCheckingCommunicationSpi.cleanup(); stopAllGrids(); cleanPersistenceDir(); + + if (!walRebalanceInvoked) + throw new AssertionError("WAL rebalance hasn't been invoked."); } /** @@ -97,7 +125,8 @@ public class IgniteWalRebalanceTest extends GridCommonAbstractTest { public void testSimple() throws Exception { IgniteEx ig0 = startGrid(0); IgniteEx ig1 = startGrid(1); - final int entryCnt = 10_000; + + final int entryCnt = PARTS_CNT * 100; ig0.cluster().active(true); @@ -135,7 +164,8 @@ public void testSimple() throws Exception { public void testRebalanceRemoves() throws Exception { IgniteEx ig0 = startGrid(0); IgniteEx ig1 = startGrid(1); - final int entryCnt = 10_000; + + final int entryCnt = PARTS_CNT * 100; ig0.cluster().active(true); @@ -173,6 +203,164 @@ public void testRebalanceRemoves() throws Exception { } } + /** + * Test that WAL rebalance is not invoked if there are gaps in WAL history due to temporary WAL disabling. + * + * @throws Exception If failed. + */ + public void testWithLocalWalChange() throws Exception { + System.setProperty(IgniteSystemProperties.IGNITE_DISABLE_WAL_DURING_REBALANCING, "true"); + + IgniteEx crd = (IgniteEx) startGrids(4); + + 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)); + } + + stopAllGrids(); + + IgniteEx ig0 = (IgniteEx) startGrids(2); + + ig0.cluster().active(true); + + IgniteCache cache = ig0.cache(CACHE_NAME); + + int grpId = ig0.cachex(CACHE_NAME).context().groupId(); + + for (int k = 0; k < entryCnt; k++) + cache.put(k, new IndexedObject(k)); + + // This node should rebalance data from other nodes and shouldn't have WAL history. + Ignite ignite = startGrid(2); + + awaitPartitionMapExchange(); + + Set topVers = ((WalRebalanceCheckingCommunicationSpi) ignite.configuration().getCommunicationSpi()) + .walRebalanceVersions(grpId); + + Assert.assertTrue(topVers.contains(ignite.cluster().topologyVersion())); + + // Rewrite some data. + for (int k = 0; k < entryCnt; k++) { + if (k % 3 == 0) + cache.put(k, new IndexedObject(k + 1)); + else if (k % 3 == 1) // Spread removes across all partitions. + cache.remove(k); + } + + // Stop grids which have actual WAL history. + stopGrid(0); + + stopGrid(1); + + // Start new node which should rebalance all data from node(2) without using WAL, + // because node(2) doesn't have full history for rebalance. + ignite = startGrid(3); + + awaitPartitionMapExchange(); + + topVers = ((WalRebalanceCheckingCommunicationSpi) ignite.configuration().getCommunicationSpi()) + .walRebalanceVersions(grpId); + + Assert.assertFalse(topVers.contains(ignite.cluster().topologyVersion())); + + // Check data consistency. + for (Ignite ig : G.allGrids()) { + IgniteCache cache1 = ig.cache(CACHE_NAME); + + for (int k = 0; k < entryCnt; k++) { + if (k % 3 == 0) + assertEquals(new IndexedObject(k + 1), cache1.get(k)); + else if (k % 3 == 1) + assertNull(cache1.get(k)); + else + assertEquals(new IndexedObject(k), cache1.get(k)); + } + } + } + + /** + * Test that WAL rebalance is not invoked if there are gaps in WAL history due to global WAL disabling. + * + * @throws Exception If failed. + */ + public void testWithGlobalWalChange() 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)); + } + + stopAllGrids(); + + // Rewrite data with globally disabled WAL. + crd = (IgniteEx) startGrids(2); + + crd.cluster().active(true); + + crd.cluster().disableWal(CACHE_NAME); + + IgniteCache cache = crd.cache(CACHE_NAME); + + int grpId = crd.cachex(CACHE_NAME).context().groupId(); + + for (int k = 0; k < entryCnt; k++) + cache.put(k, new IndexedObject(k)); + + crd.cluster().enableWal(CACHE_NAME); + + // This node shouldn't rebalance data using WAL, because it was disabled on other nodes. + IgniteEx ignite = startGrid(2); + + awaitPartitionMapExchange(); + + Set topVers = ((WalRebalanceCheckingCommunicationSpi) ignite.configuration().getCommunicationSpi()) + .walRebalanceVersions(grpId); + + Assert.assertFalse(topVers.contains(ignite.cluster().topologyVersion())); + + stopGrid(2); + + // Fix actual state to have start point in WAL to rebalance from. + forceCheckpoint(); + + // After another rewriting data with enabled WAL, node should rebalance this diff using WAL rebalance. + for (int k = 0; k < entryCnt; k++) + cache.put(k, new IndexedObject(k + 1)); + + ignite = startGrid(2); + + awaitPartitionMapExchange(); + + topVers = ((WalRebalanceCheckingCommunicationSpi) ignite.configuration().getCommunicationSpi()) + .walRebalanceVersions(grpId); + + Assert.assertTrue(topVers.contains(ignite.cluster().topologyVersion())); + + // 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 + 1), cache1.get(k)); + } + } + /** * */ @@ -214,4 +402,63 @@ private IndexedObject(int iVal) { return S.toString(IndexedObject.class, this); } } + + /** + * Wrapper of communication spi to detect on what topology versions WAL rebalance has happened. + */ + public static class WalRebalanceCheckingCommunicationSpi extends TcpCommunicationSpi { + /** (Group ID, Set of topology versions). */ + private static final Map> topVers = new HashMap<>(); + + /** Lock object. */ + private static final Object mux = new Object(); + + /** + * @param grpId Group ID. + * @return Set of topology versions where WAL history has been used for rebalance. + */ + Set walRebalanceVersions(int grpId) { + synchronized (mux) { + return Collections.unmodifiableSet(topVers.getOrDefault(grpId, Collections.emptySet())); + } + } + + /** + * @return All topology versions for all groups where WAL rebalance has been used. + */ + public static Map> allRebalances() { + synchronized (mux) { + return Collections.unmodifiableMap(topVers); + } + } + + /** + * Cleans all rebalances history. + */ + public static void cleanup() { + synchronized (mux) { + topVers.clear(); + } + } + + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure ackC) throws IgniteSpiException { + if (((GridIoMessage)msg).message() instanceof GridDhtPartitionDemandMessage) { + GridDhtPartitionDemandMessage demandMsg = (GridDhtPartitionDemandMessage) ((GridIoMessage)msg).message(); + + IgniteDhtDemandedPartitionsMap map = demandMsg.partitions(); + + if (!map.historicalMap().isEmpty()) { + int grpId = demandMsg.groupId(); + long topVer = demandMsg.topologyVersion().topologyVersion(); + + synchronized (mux) { + topVers.computeIfAbsent(grpId, v -> new HashSet<>()).add(topVer); + } + } + } + + super.sendMessage(node, msg, ackC); + } + } } \ No newline at end of file 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 7a50c0d55a44b..808e737244101 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 @@ -54,6 +54,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtDemandedPartitionsMap; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; 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.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cache.persistence.freelist.AbstractFreeList; import org.apache.ignite.internal.processors.cache.persistence.freelist.CacheFreeListImpl; @@ -500,7 +501,7 @@ public void testCheckpointHistory() throws Exception { dbMgr.waitForCheckpoint("test"); } - GridCacheDatabaseSharedManager.CheckpointHistory hist = dbMgr.checkpointHistory(); + CheckpointHistory hist = dbMgr.checkpointHistory(); assertTrue(hist.checkpoints().size() <= WAL_HIST_SIZE); 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 d33b20b6f6cf5..571576010995b 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 @@ -23,7 +23,6 @@ import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsBinarySortObjectFieldsTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsCorruptedIndexTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsMarshallerMappingRestoreOnNodeStartTest; -import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsTxCacheHistoricalRebalancingTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsTxCacheRebalancingTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePersistentStoreCacheGroupsTest; import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsMultiNodePutGetRestartTest; @@ -65,7 +64,6 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgnitePdsTxCacheRebalancingTest.class); suite.addTestSuite(IgnitePdsAtomicCacheHistoricalRebalancingTest.class); - suite.addTestSuite(IgnitePdsTxCacheHistoricalRebalancingTest.class); suite.addTestSuite(IgniteWalRebalanceTest.class); suite.addTestSuite(IgniteWalRecoveryPPCTest.class); From 1d585d780cf88cd56728718f66a405e43902782b Mon Sep 17 00:00:00 2001 From: Andrey Gura Date: Thu, 7 Jun 2018 15:53:11 +0300 Subject: [PATCH 0257/1463] IGNITE-8565 Client marshalling improvements --- .../client/suite/IgniteClientTestSuite.java | 2 + .../TcpRestUnmarshalVulnerabilityTest.java | 269 ++++++++++++++++++ .../client/ClientMarshallerBenchmarkTest.java | 16 +- .../apache/ignite/internal/IgniteKernal.java | 108 +------ .../jdk/GridClientJdkMarshaller.java | 51 +++- .../impl/GridTcpRouterNioListenerAdapter.java | 14 +- .../protocols/tcp/GridTcpRestProtocol.java | 13 +- .../ignite/marshaller/MarshallerUtils.java | 117 ++++++++ .../config/class_list_exploit_included.txt | 3 +- .../hadoop/mapreduce/MapReduceClient.java | 16 +- 10 files changed, 494 insertions(+), 115 deletions(-) create mode 100644 modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/TcpRestUnmarshalVulnerabilityTest.java diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java index 163f89acb7a35..e5378c33f61df 100644 --- a/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java +++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java @@ -58,6 +58,7 @@ import org.apache.ignite.internal.processors.rest.RestProcessorMultiStartSelfTest; import org.apache.ignite.internal.processors.rest.RestProcessorStartSelfTest; import org.apache.ignite.internal.processors.rest.TaskCommandHandlerSelfTest; +import org.apache.ignite.internal.processors.rest.TcpRestUnmarshalVulnerabilityTest; import org.apache.ignite.internal.processors.rest.protocols.tcp.TcpRestParserSelfTest; import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.RedisProtocolConnectSelfTest; import org.apache.ignite.internal.processors.rest.protocols.tcp.redis.RedisProtocolServerSelfTest; @@ -84,6 +85,7 @@ public static TestSuite suite() { // Test custom binary protocol with test client. suite.addTestSuite(RestBinaryProtocolSelfTest.class); + suite.addTestSuite(TcpRestUnmarshalVulnerabilityTest.class); // Test jetty rest processor suite.addTestSuite(JettyRestProcessorSignedSelfTest.class); diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/TcpRestUnmarshalVulnerabilityTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/TcpRestUnmarshalVulnerabilityTest.java new file mode 100644 index 0000000000000..92d824be329db --- /dev/null +++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/TcpRestUnmarshalVulnerabilityTest.java @@ -0,0 +1,269 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.rest; + +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.ObjectInputStream; +import java.io.OutputStream; +import java.net.InetAddress; +import java.net.Socket; +import java.nio.ByteBuffer; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.ignite.configuration.ConnectorConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.client.marshaller.jdk.GridClientJdkMarshaller; +import org.apache.ignite.internal.processors.rest.client.message.GridClientHandshakeRequest; +import org.apache.ignite.internal.processors.rest.client.message.GridClientMessage; +import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_MARSHALLER_BLACKLIST; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_MARSHALLER_WHITELIST; +import static org.apache.ignite.internal.processors.rest.protocols.tcp.GridMemcachedMessage.IGNITE_HANDSHAKE_FLAG; +import static org.apache.ignite.internal.processors.rest.protocols.tcp.GridMemcachedMessage.IGNITE_REQ_FLAG; + +/** + * Tests for whitelist and blacklist ot avoiding deserialization vulnerability. + */ +public class TcpRestUnmarshalVulnerabilityTest extends GridCommonAbstractTest { + /** Marshaller. */ + private static final GridClientJdkMarshaller MARSH = new GridClientJdkMarshaller(); + + /** Shared value. */ + private static final AtomicBoolean SHARED = new AtomicBoolean(); + + /** Port. */ + private static int port; + + /** Host. */ + private static String host; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + ConnectorConfiguration connCfg = new ConnectorConfiguration(); + + port = connCfg.getPort(); + host = connCfg.getHost(); + + cfg.setConnectorConfiguration(connCfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + SHARED.set(false); + + System.clearProperty(IGNITE_MARSHALLER_WHITELIST); + System.clearProperty(IGNITE_MARSHALLER_BLACKLIST); + + IgniteUtils.clearClassCache(); + } + + /** + * @throws Exception If failed. + */ + public void testNoLists() throws Exception { + testExploit(true); + } + + /** + * @throws Exception If failed. + */ + public void testWhiteListIncluded() throws Exception { + String path = U.resolveIgnitePath("modules/core/src/test/config/class_list_exploit_included.txt").getPath(); + + System.setProperty(IGNITE_MARSHALLER_WHITELIST, path); + + testExploit(true); + } + + /** + * @throws Exception If failed. + */ + public void testWhiteListExcluded() throws Exception { + String path = U.resolveIgnitePath("modules/core/src/test/config/class_list_exploit_excluded.txt").getPath(); + + System.setProperty(IGNITE_MARSHALLER_WHITELIST, path); + + testExploit(false); + } + + /** + * @throws Exception If failed. + */ + public void testBlackListIncluded() throws Exception { + String path = U.resolveIgnitePath("modules/core/src/test/config/class_list_exploit_included.txt").getPath(); + + System.setProperty(IGNITE_MARSHALLER_BLACKLIST, path); + + testExploit(false); + } + + /** + * @throws Exception If failed. + */ + public void testBlackListExcluded() throws Exception { + String path = U.resolveIgnitePath("modules/core/src/test/config/class_list_exploit_excluded.txt").getPath(); + + System.setProperty(IGNITE_MARSHALLER_BLACKLIST, path); + + testExploit(true); + } + + /** + * @throws Exception If failed. + */ + public void testBothListIncluded() throws Exception { + String path = U.resolveIgnitePath("modules/core/src/test/config/class_list_exploit_included.txt").getPath(); + + System.setProperty(IGNITE_MARSHALLER_WHITELIST, path); + System.setProperty(IGNITE_MARSHALLER_BLACKLIST, path); + + testExploit(false); + } + + /** + * @param positive Positive. + */ + private void testExploit(boolean positive) throws Exception { + try { + startGrid(); + + attack(marshal(new Exploit()).array()); + + boolean res = GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return SHARED.get(); + } + }, 3000L); + + if (positive) + assertTrue(res); + else + assertFalse(res); + } + finally { + stopAllGrids(); + } + } + + /** + * @param obj Object. + */ + private static ByteBuffer marshal(Object obj) throws IOException { + return MARSH.marshal(obj, 0); + } + + /** + * @param data Data. + */ + private void attack(byte[] data) throws IOException { + InetAddress addr = InetAddress.getByName(host); + + try ( + Socket sock = new Socket(addr, port); + OutputStream os = new BufferedOutputStream(sock.getOutputStream()) + ) { + // Handshake request. + os.write(IGNITE_HANDSHAKE_FLAG); + + GridClientHandshakeRequest req = new GridClientHandshakeRequest(); + req.marshallerId(GridClientJdkMarshaller.ID); + os.write(req.rawBytes()); + os.flush(); + + // Handshake response + InputStream is = new BufferedInputStream(sock.getInputStream()); + + is.read(new byte[146]); // Read handshake response. + + int len = data.length + 40; + + os.write(IGNITE_REQ_FLAG); // Package type. + os.write((byte)(len >> 24)); // Package length. + os.write((byte)(len >> 16)); + os.write((byte)(len >> 8)); + os.write((byte)(len)); + os.write(new byte[40]); // Stream header. + os.write(data); // Exploit. + os.flush(); + } + } + + /** */ + private static class Exploit implements GridClientMessage { + /** + * @param is Input stream. + */ + private void readObject(ObjectInputStream is) throws ClassNotFoundException, IOException { + SHARED.set(true); + } + + /** {@inheritDoc} */ + @Override public long requestId() { + return 0; + } + + /** {@inheritDoc} */ + @Override public void requestId(long reqId) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public UUID clientId() { + return null; + } + + /** {@inheritDoc} */ + @Override public void clientId(UUID id) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public UUID destinationId() { + return null; + } + + /** {@inheritDoc} */ + @Override public void destinationId(UUID id) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public byte[] sessionToken() { + return new byte[0]; + } + + /** {@inheritDoc} */ + @Override public void sessionToken(byte[] sesTok) { + // No-op. + } + } +} \ No newline at end of file diff --git a/modules/clients/src/test/java/org/apache/ignite/loadtests/client/ClientMarshallerBenchmarkTest.java b/modules/clients/src/test/java/org/apache/ignite/loadtests/client/ClientMarshallerBenchmarkTest.java index 570678fc81d22..08c2cbe375213 100644 --- a/modules/clients/src/test/java/org/apache/ignite/loadtests/client/ClientMarshallerBenchmarkTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/loadtests/client/ClientMarshallerBenchmarkTest.java @@ -22,11 +22,14 @@ import java.util.HashMap; import java.util.Map; import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; import org.apache.ignite.internal.client.marshaller.GridClientMarshaller; import org.apache.ignite.internal.client.marshaller.jdk.GridClientJdkMarshaller; import org.apache.ignite.internal.client.marshaller.optimized.GridClientOptimizedMarshaller; import org.apache.ignite.internal.processors.rest.client.message.GridClientCacheRequest; import org.apache.ignite.internal.util.typedef.X; +import org.apache.ignite.marshaller.MarshallerUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import static org.apache.ignite.internal.processors.rest.client.message.GridClientCacheRequest.GridCacheOperation.CAS; @@ -41,10 +44,15 @@ public class ClientMarshallerBenchmarkTest extends GridCommonAbstractTest { /** */ public ClientMarshallerBenchmarkTest() { - marshallers = new GridClientMarshaller[] { - new GridClientJdkMarshaller(), - new GridClientOptimizedMarshaller() - }; + try { + marshallers = new GridClientMarshaller[] { + new GridClientJdkMarshaller(MarshallerUtils.classNameFilter(this.getClass().getClassLoader())), + new GridClientOptimizedMarshaller() + }; + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } } /** 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 112eb8548b493..cfbefec3e31d4 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 @@ -17,14 +17,10 @@ package org.apache.ignite.internal; -import java.io.BufferedReader; import java.io.Externalizable; import java.io.File; -import java.io.FileInputStream; -import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; -import java.io.InputStreamReader; import java.io.InvalidObjectException; import java.io.ObjectInput; import java.io.ObjectOutput; @@ -197,6 +193,7 @@ import org.apache.ignite.lifecycle.LifecycleBean; import org.apache.ignite.lifecycle.LifecycleEventType; import org.apache.ignite.marshaller.MarshallerExclusions; +import org.apache.ignite.marshaller.MarshallerUtils; import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.apache.ignite.mxbean.ClusterMetricsMXBean; import org.apache.ignite.mxbean.IgniteMXBean; @@ -274,8 +271,6 @@ import static org.apache.ignite.internal.IgniteVersionUtils.VER_STR; import static org.apache.ignite.lifecycle.LifecycleEventType.AFTER_NODE_START; import static org.apache.ignite.lifecycle.LifecycleEventType.BEFORE_NODE_START; -import static org.apache.ignite.marshaller.MarshallerUtils.CLS_NAMES_FILE; -import static org.apache.ignite.marshaller.MarshallerUtils.JDK_CLS_NAMES_FILE; /** * Ignite kernal. @@ -901,7 +896,7 @@ public void start( schemaExecSvc, customExecSvcs, plugins, - classNameFilter() + MarshallerUtils.classNameFilter(this.getClass().getClassLoader()) ); cfg.getMarshaller().setContext(ctx.marshallerContext()); @@ -1767,105 +1762,6 @@ private void startProcessor(GridProcessor proc) throws IgniteCheckedException { } } - /** - * Returns class name filter for marshaller. - * - * @return Class name filter for marshaller. - */ - private IgnitePredicate classNameFilter() throws IgniteCheckedException { - ClassSet whiteList = classWhiteList(); - ClassSet blackList = classBlackList(); - - return new IgnitePredicate() { - @Override public boolean apply(String s) { - // Allows all primitive arrays and checks arrays' type. - if ((blackList != null || whiteList != null) && s.charAt(0) == '[') { - if (s.charAt(1) == 'L' && s.length() > 2) - s = s.substring(2, s.length() - 1); - else - return true; - } - - return (blackList == null || !blackList.contains(s)) && (whiteList == null || whiteList.contains(s)); - } - }; - } - - /** - * @return White list of classes. - */ - private ClassSet classWhiteList() throws IgniteCheckedException { - ClassSet clsSet = null; - - String fileName = IgniteSystemProperties.getString(IgniteSystemProperties.IGNITE_MARSHALLER_WHITELIST); - - if (fileName != null) { - clsSet = new ClassSet(); - - addClassNames(JDK_CLS_NAMES_FILE, clsSet); - addClassNames(CLS_NAMES_FILE, clsSet); - addClassNames(fileName, clsSet); - } - - return clsSet; - } - - /** - * @return Black list of classes. - */ - private ClassSet classBlackList() throws IgniteCheckedException { - ClassSet clsSet = null; - - String blackListFileName = IgniteSystemProperties.getString(IgniteSystemProperties.IGNITE_MARSHALLER_BLACKLIST); - - if (blackListFileName != null) - addClassNames(blackListFileName, clsSet = new ClassSet()); - - return clsSet; - } - - /** - * Reads class names from resource referred by given system property name and returns set of classes. - * - * @param fileName File name containing list of classes. - * @param clsSet Class set for update. - * @return Set of classes. - */ - private void addClassNames(String fileName, ClassSet clsSet) throws IgniteCheckedException { - InputStream is = this.getClass().getClassLoader().getResourceAsStream(fileName); - - if (is == null) { - try { - is = new FileInputStream(new File(fileName)); - } - catch (FileNotFoundException e) { - throw new IgniteCheckedException("File " + fileName + " not found."); - } - } - - try (BufferedReader reader = new BufferedReader(new InputStreamReader(is))) { - String line; - - for (int i = 1; (line = reader.readLine()) != null; i++) { - String s = line.trim(); - - if (!s.isEmpty() && s.charAt(0) != '#' && s.charAt(0) != '[') { - try { - clsSet.add(s); - } - catch (IllegalArgumentException e) { - throw new IgniteCheckedException("Exception occurred while reading list of classes" + - "[path=" + fileName + ", row=" + i + ", line=" + s + ']', e); - } - } - } - } - catch (IOException e) { - throw new IgniteCheckedException("Exception occurred while reading and creating list of classes " + - "[path=" + fileName + ']', e); - } - } - /** * Add helper. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/jdk/GridClientJdkMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/jdk/GridClientJdkMarshaller.java index 87bc7aa0de9b0..bd89e80b5d383 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/jdk/GridClientJdkMarshaller.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/jdk/GridClientJdkMarshaller.java @@ -19,13 +19,16 @@ import java.io.ByteArrayInputStream; import java.io.IOException; +import java.io.InputStream; import java.io.ObjectInput; import java.io.ObjectInputStream; import java.io.ObjectOutput; import java.io.ObjectOutputStream; +import java.io.ObjectStreamClass; import java.nio.ByteBuffer; import org.apache.ignite.internal.client.marshaller.GridClientMarshaller; import org.apache.ignite.internal.util.io.GridByteArrayOutputStream; +import org.apache.ignite.lang.IgnitePredicate; /** * Simple marshaller that utilize JDK serialization features. @@ -34,6 +37,23 @@ public class GridClientJdkMarshaller implements GridClientMarshaller { /** ID. */ public static final byte ID = 2; + /** Class name filter. */ + private final IgnitePredicate clsFilter; + + /** + * Default constructor. + */ + public GridClientJdkMarshaller() { + this(null); + } + + /** + * @param clsFilter Class filter. + */ + public GridClientJdkMarshaller(IgnitePredicate clsFilter) { + this.clsFilter = clsFilter; + } + /** {@inheritDoc} */ @Override public ByteBuffer marshal(Object obj, int off) throws IOException { GridByteArrayOutputStream bOut = new GridByteArrayOutputStream(); @@ -60,7 +80,7 @@ public class GridClientJdkMarshaller implements GridClientMarshaller { @Override public T unmarshal(byte[] bytes) throws IOException { ByteArrayInputStream tmp = new ByteArrayInputStream(bytes); - ObjectInput in = new ObjectInputStream(tmp); + ObjectInput in = new ClientJdkInputStream(tmp, clsFilter); try { return (T)in.readObject(); @@ -69,4 +89,33 @@ public class GridClientJdkMarshaller implements GridClientMarshaller { throw new IOException("Failed to unmarshal target object: " + e.getMessage(), e); } } + + /** + * Wrapper with class resolving control. + */ + private static class ClientJdkInputStream extends ObjectInputStream { + /** Class name filter. */ + private final IgnitePredicate clsFilter; + + + /** + * @param in Input stream. + * @param clsFilter Class filter. + */ + public ClientJdkInputStream(InputStream in, IgnitePredicate clsFilter) throws IOException { + super(in); + + this.clsFilter = clsFilter; + } + + /** {@inheritDoc} */ + @Override protected Class resolveClass(ObjectStreamClass desc) throws IOException, ClassNotFoundException { + String clsName = desc.getName(); + + if (clsFilter != null && !clsFilter.apply(clsName)) + throw new RuntimeException("Deserialization of class " + clsName + " is disallowed."); + + return super.resolveClass(desc); + } + } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java index 75aa6f29ccc3f..364e044ab677f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterNioListenerAdapter.java @@ -23,6 +23,8 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.failure.FailureType; import org.apache.ignite.internal.client.GridClientException; @@ -42,6 +44,8 @@ import org.apache.ignite.internal.util.nio.GridNioServerListener; import org.apache.ignite.internal.util.nio.GridNioSession; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.marshaller.MarshallerUtils; import org.apache.ignite.plugin.PluginProvider; import org.jetbrains.annotations.Nullable; @@ -87,7 +91,15 @@ public GridTcpRouterNioListenerAdapter(IgniteLogger log, GridRouterClientImpl cl marshMap.put(GridClientOptimizedMarshaller.ID, optdMarsh); marshMap.put(GridClientZipOptimizedMarshaller.ID, new GridClientZipOptimizedMarshaller(optdMarsh, providers)); - marshMap.put(GridClientJdkMarshaller.ID, new GridClientJdkMarshaller()); + + try { + IgnitePredicate clsFilter = MarshallerUtils.classNameFilter(this.getClass().getClassLoader()); + + marshMap.put(GridClientJdkMarshaller.ID, new GridClientJdkMarshaller(clsFilter)); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } init(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java index 0049dbc5d72f3..f08cc254c545d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java @@ -27,6 +27,7 @@ import javax.net.ssl.SSLContext; import javax.net.ssl.SSLException; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; import org.apache.ignite.configuration.ConnectorConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.GridKernalContext; @@ -46,6 +47,8 @@ import org.apache.ignite.internal.util.nio.GridNioServerListener; import org.apache.ignite.internal.util.nio.ssl.GridNioSslFilter; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.marshaller.MarshallerUtils; import org.apache.ignite.plugin.PluginProvider; import org.apache.ignite.spi.IgnitePortProtocol; import org.jetbrains.annotations.Nullable; @@ -149,7 +152,15 @@ else if (depFactory != null) marshMap.put(GridClientOptimizedMarshaller.ID, optMarsh); marshMap.put(GridClientZipOptimizedMarshaller.ID, new GridClientZipOptimizedMarshaller(optMarsh, providers)); - marshMap.put(GridClientJdkMarshaller.ID, new GridClientJdkMarshaller()); + + try { + IgnitePredicate clsFilter = MarshallerUtils.classNameFilter(this.getClass().getClassLoader()); + + marshMap.put(GridClientJdkMarshaller.ID, new GridClientJdkMarshaller(clsFilter)); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } lsnr.marshallers(marshMap); } 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 bec1f57ccd9f2..54d0390b118b1 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 @@ -17,6 +17,17 @@ package org.apache.ignite.marshaller; +import java.io.BufferedReader; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.internal.ClassSet; +import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.jetbrains.annotations.Nullable; @@ -83,4 +94,110 @@ public static void jobSenderVersion(IgniteProductVersion ver) { public static IgniteProductVersion jobSenderVersion() { return JOB_SND_NODE_VER.get(); } + + /** + * Returns class name filter for marshaller. + * + * @return Class name filter for marshaller. + */ + public static IgnitePredicate classNameFilter(ClassLoader clsLdr) throws IgniteCheckedException { + ClassSet whiteList = classWhiteList(clsLdr); + ClassSet blackList = classBlackList(clsLdr); + + return new IgnitePredicate() { + @Override public boolean apply(String s) { + // Allows all primitive arrays and checks arrays' type. + if ((blackList != null || whiteList != null) && s.charAt(0) == '[') { + if (s.charAt(1) == 'L' && s.length() > 2) + s = s.substring(2, s.length() - 1); + else + return true; + } + + return (blackList == null || !blackList.contains(s)) && (whiteList == null || whiteList.contains(s)); + } + }; + } + + /** + * @param clsLdr Class loader. + * @return White list of classes. + */ + private static ClassSet classWhiteList(ClassLoader clsLdr) throws IgniteCheckedException { + ClassSet clsSet = null; + + String fileName = IgniteSystemProperties.getString(IgniteSystemProperties.IGNITE_MARSHALLER_WHITELIST); + + if (fileName != null) { + clsSet = new ClassSet(); + + addClassNames(JDK_CLS_NAMES_FILE, clsSet, clsLdr); + addClassNames(CLS_NAMES_FILE, clsSet, clsLdr); + addClassNames(fileName, clsSet, clsLdr); + } + + return clsSet; + } + + /** + * @param clsLdr Class loader. + * @return Black list of classes. + */ + private static ClassSet classBlackList(ClassLoader clsLdr) throws IgniteCheckedException { + ClassSet clsSet = null; + + String blackListFileName = IgniteSystemProperties.getString(IgniteSystemProperties.IGNITE_MARSHALLER_BLACKLIST); + + if (blackListFileName != null) + addClassNames(blackListFileName, clsSet = new ClassSet(), clsLdr); + + return clsSet; + } + + /** + * Reads class names from resource referred by given system property name and returns set of classes. + * + * @param fileName File name containing list of classes. + * @param clsSet Class set for update. + * @param clsLdr Class loader. + */ + private static void addClassNames( + String fileName, + ClassSet clsSet, + ClassLoader clsLdr + ) throws IgniteCheckedException { + InputStream is = clsLdr.getResourceAsStream(fileName); + + if (is == null) { + try { + is = new FileInputStream(new File(fileName)); + } + catch (FileNotFoundException e) { + throw new IgniteCheckedException("File " + fileName + " not found."); + } + } + + try (BufferedReader reader = new BufferedReader(new InputStreamReader(is))) { + String line; + + for (int i = 1; (line = reader.readLine()) != null; i++) { + String s = line.trim(); + + if (!s.isEmpty() && s.charAt(0) != '#' && s.charAt(0) != '[') { + try { + clsSet.add(s); + } + catch (IllegalArgumentException e) { + throw new IgniteCheckedException("Exception occurred while reading list of classes" + + "[path=" + fileName + ", row=" + i + ", line=" + s + ']', e); + } + } + } + } + catch (IOException e) { + throw new IgniteCheckedException("Exception occurred while reading and creating list of classes " + + "[path=" + fileName + ']', e); + } + } + } diff --git a/modules/core/src/test/config/class_list_exploit_included.txt b/modules/core/src/test/config/class_list_exploit_included.txt index 9a07d541cbfd0..b8cb550cd9f54 100644 --- a/modules/core/src/test/config/class_list_exploit_included.txt +++ b/modules/core/src/test/config/class_list_exploit_included.txt @@ -16,4 +16,5 @@ # org.apache.ignite.spi.discovery.tcp.DiscoveryUnmarshalVulnerabilityTest$Exploit -org.apache.ignite.stream.socket.SocketStreamerUnmarshalVulnerabilityTest$Exploit \ No newline at end of file +org.apache.ignite.stream.socket.SocketStreamerUnmarshalVulnerabilityTest$Exploit +org.apache.ignite.internal.processors.rest.TcpRestUnmarshalVulnerabilityTest$Exploit \ No newline at end of file diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/mapreduce/MapReduceClient.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/mapreduce/MapReduceClient.java index 3d52176e1ebb2..62e50c0e2e181 100644 --- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/mapreduce/MapReduceClient.java +++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/mapreduce/MapReduceClient.java @@ -17,6 +17,8 @@ package org.apache.ignite.internal.processors.hadoop.mapreduce; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; import org.apache.ignite.internal.client.GridClient; import org.apache.ignite.internal.client.GridClientConfiguration; import org.apache.ignite.internal.client.GridClientException; @@ -26,6 +28,8 @@ import java.io.IOException; import java.util.Collection; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.marshaller.MarshallerUtils; import static org.apache.ignite.internal.client.GridClientProtocol.TCP; @@ -83,7 +87,17 @@ public GridClient client() throws IOException { cliCfg.setProtocol(TCP); cliCfg.setServers(addrs); - cliCfg.setMarshaller(new GridClientJdkMarshaller()); + + try { + IgnitePredicate clsFilter = + MarshallerUtils.classNameFilter(this.getClass().getClassLoader()); + + cliCfg.setMarshaller(new GridClientJdkMarshaller(clsFilter)); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + cliCfg.setMaxConnectionIdleTime(24 * 60 * 60 * 1000L); // 1 day. cliCfg.setDaemon(true); From e733b1c151bcc4011bff6e49287661a6f3403a1a Mon Sep 17 00:00:00 2001 From: Ilya Lantukh Date: Wed, 23 May 2018 18:02:23 +0300 Subject: [PATCH 0258/1463] IGNITE-7809 fixed stable failures of IgniteWalFlushDefaultSelfTest in Ignite PDS 2 & PDS 2 Direct IO, new tests were added. - Fixes #3569. Signed-off-by: dpavlov (cherry picked from commit fed2c02) --- .../wal/FileWriteAheadLogManager.java | 6 +- .../FsyncModeFileWriteAheadLogManager.java | 11 +++- ...FlushBackgroundWithMmapBufferSelfTest.java | 28 ++++++++++ ....java => IgniteWalFlushFsyncSelfTest.java} | 2 +- ...FlushFsyncWithDedicatedWorkerSelfTest.java | 39 +++++++++++++ ...teWalFlushFsyncWithMmapBufferSelfTest.java | 28 ++++++++++ ...WalFlushLogOnlyWithMmapBufferSelfTest.java | 28 ++++++++++ ...lushMultiNodeFailoverAbstractSelfTest.java | 56 ++++++++++++------- .../testsuites/IgnitePdsTestSuite2.java | 16 +++++- 9 files changed, 189 insertions(+), 25 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushBackgroundWithMmapBufferSelfTest.java rename modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/{IgniteWalFlushDefaultSelfTest.java => IgniteWalFlushFsyncSelfTest.java} (91%) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFsyncWithDedicatedWorkerSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFsyncWithMmapBufferSelfTest.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushLogOnlyWithMmapBufferSelfTest.java 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 15b9109a8f01a..cb80961215301 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 @@ -362,7 +362,11 @@ public FileWriteAheadLogManager(@NotNull final GridKernalContext ctx) { evt = ctx.event(); } - /** For test purposes only. */ + /** + * For test purposes only. + * + * @param ioFactory IO factory. + */ public void setFileIOFactory(FileIOFactory ioFactory) { this.ioFactory = ioFactory; } 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 4a642d0193fe3..867f13f69e5c1 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 @@ -230,7 +230,7 @@ public class FsyncModeFileWriteAheadLogManager extends GridCacheSharedManagerAda private volatile long lastTruncatedArchiveIdx = -1L; /** Factory to provide I/O interfaces for read/write operations with files */ - private final FileIOFactory ioFactory; + private FileIOFactory ioFactory; /** Updater for {@link #currentHnd}, used for verify there are no concurrent update for current log segment handle */ private static final AtomicReferenceFieldUpdater currentHndUpd = @@ -320,6 +320,15 @@ public FsyncModeFileWriteAheadLogManager(@NotNull final GridKernalContext ctx) { assert mode == WALMode.FSYNC : dsCfg; } + /** + * For test purposes only. + * + * @param ioFactory IO factory. + */ + public void setFileIOFactory(FileIOFactory ioFactory) { + this.ioFactory = ioFactory; + } + /** {@inheritDoc} */ @Override public void start0() throws IgniteCheckedException { if (!cctx.kernalContext().clientNode()) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushBackgroundWithMmapBufferSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushBackgroundWithMmapBufferSelfTest.java new file mode 100644 index 0000000000000..4f021cb9bd665 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushBackgroundWithMmapBufferSelfTest.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.persistence.db.wal; + +/** + * + */ +public class IgniteWalFlushBackgroundWithMmapBufferSelfTest extends IgniteWalFlushBackgroundSelfTest { + /** {@inheritDoc} */ + @Override protected boolean mmap() { + return true; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushDefaultSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFsyncSelfTest.java similarity index 91% rename from modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushDefaultSelfTest.java rename to modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFsyncSelfTest.java index 94e7e258b298b..453064a3e407d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushDefaultSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFsyncSelfTest.java @@ -22,7 +22,7 @@ /** * */ -public class IgniteWalFlushDefaultSelfTest extends IgniteWalFlushMultiNodeFailoverAbstractSelfTest { +public class IgniteWalFlushFsyncSelfTest extends IgniteWalFlushMultiNodeFailoverAbstractSelfTest { /** {@inheritDoc} */ @Override protected int gridCount() { return 1; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFsyncWithDedicatedWorkerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFsyncWithDedicatedWorkerSelfTest.java new file mode 100644 index 0000000000000..4360fe5221599 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFsyncWithDedicatedWorkerSelfTest.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.internal.processors.cache.persistence.db.wal; + +import org.apache.ignite.IgniteSystemProperties; + +/** + * + */ +public class IgniteWalFlushFsyncWithDedicatedWorkerSelfTest extends IgniteWalFlushFsyncSelfTest { + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + System.setProperty(IgniteSystemProperties.IGNITE_WAL_FSYNC_WITH_DEDICATED_WORKER, "true"); + + super.beforeTest(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + System.clearProperty(IgniteSystemProperties.IGNITE_WAL_FSYNC_WITH_DEDICATED_WORKER); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFsyncWithMmapBufferSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFsyncWithMmapBufferSelfTest.java new file mode 100644 index 0000000000000..63872bace6ce8 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFsyncWithMmapBufferSelfTest.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.persistence.db.wal; + +/** + * + */ +public class IgniteWalFlushFsyncWithMmapBufferSelfTest extends IgniteWalFlushFsyncSelfTest { + /** {@inheritDoc} */ + @Override protected boolean mmap() { + return true; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushLogOnlyWithMmapBufferSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushLogOnlyWithMmapBufferSelfTest.java new file mode 100644 index 0000000000000..d1f5075b56b4a --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushLogOnlyWithMmapBufferSelfTest.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.persistence.db.wal; + +/** + * + */ +public class IgniteWalFlushLogOnlyWithMmapBufferSelfTest extends IgniteWalFlushLogOnlySelfTest { + /** {@inheritDoc} */ + @Override protected boolean mmap() { + return true; + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java index fe1632817e44c..9b5b642272ff0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java @@ -24,20 +24,20 @@ import java.nio.MappedByteBuffer; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.CacheAtomicityMode; 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.IgniteKernal; import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; 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.processors.cache.persistence.wal.FsyncModeFileWriteAheadLogManager; import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -72,9 +72,13 @@ public abstract class IgniteWalFlushMultiNodeFailoverAbstractSelfTest extends Gr /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { + super.beforeTest(); + stopAllGrids(); cleanPersistenceDir(); + + System.setProperty(IgniteSystemProperties.IGNITE_WAL_MMAP, Boolean.toString(mmap())); } /** {@inheritDoc} */ @@ -82,6 +86,10 @@ public abstract class IgniteWalFlushMultiNodeFailoverAbstractSelfTest extends Gr stopAllGrids(); cleanPersistenceDir(); + + System.clearProperty(IgniteSystemProperties.IGNITE_WAL_MMAP); + + super.afterTest(); } /** {@inheritDoc} */ @@ -89,9 +97,18 @@ public abstract class IgniteWalFlushMultiNodeFailoverAbstractSelfTest extends Gr return 60_000; } - /** {@inheritDoc} */ + /** + * @return WAL mode used in test. + */ protected abstract WALMode walMode(); + /** + * @return {@code True} if test should use MMAP buffer mode. + */ + protected boolean mmap() { + return false; + } + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); @@ -138,14 +155,7 @@ public void testFailAfterStart() throws Exception { public void failWhilePut(boolean failWhileStart) throws Exception { final Ignite grid = startGridsMultiThreaded(gridCount()); - IgniteWriteAheadLogManager wal = ((IgniteKernal)grid).context().cache().context().wal(); - - boolean mmap = GridTestUtils.getFieldValue(wal, "mmap"); - - if (mmap) - return; - - grid.active(true); + grid.cluster().active(true); IgniteCache cache = grid.cache(TEST_CACHE); @@ -170,9 +180,7 @@ public void failWhilePut(boolean failWhileStart) throws Exception { startGrid(gridCount()); - FileWriteAheadLogManager wal0 = (FileWriteAheadLogManager)grid(gridCount()).context().cache().context().wal(); - - wal0.setFileIOFactory(new FailingFileIOFactory(canFail)); + setFileIOFactory(grid(gridCount()).context().cache().context().wal()); grid.cluster().setBaselineTopology(grid.cluster().topologyVersion()); @@ -186,7 +194,6 @@ public void failWhilePut(boolean failWhileStart) throws Exception { canFail.set(true); } - // We should await successful stop of node. GridTestUtils.waitForCondition(new GridAbsPredicate() { @Override public boolean apply() { @@ -200,11 +207,9 @@ public void failWhilePut(boolean failWhileStart) throws Exception { Ignite grid0 = startGrids(gridCount() + 1); - FileWriteAheadLogManager wal0 = (FileWriteAheadLogManager)grid(gridCount()).context().cache().context().wal(); + setFileIOFactory(grid(gridCount()).context().cache().context().wal()); - wal0.setFileIOFactory(new FailingFileIOFactory(canFail)); - - grid0.active(true); + grid0.cluster().active(true); cache = grid0.cache(TEST_CACHE); @@ -212,6 +217,16 @@ public void failWhilePut(boolean failWhileStart) throws Exception { assertEquals(cache.get(i), "testValue" + i); } + /** */ + private void setFileIOFactory(IgniteWriteAheadLogManager wal) { + if (wal instanceof FileWriteAheadLogManager) + ((FileWriteAheadLogManager)wal).setFileIOFactory(new FailingFileIOFactory(canFail)); + else if (wal instanceof FsyncModeFileWriteAheadLogManager) + ((FsyncModeFileWriteAheadLogManager)wal).setFileIOFactory(new FailingFileIOFactory(canFail)); + else + fail(wal.getClass().toString()); + } + /** * Create File I/O which fails after second attempt to write to File */ @@ -220,7 +235,7 @@ private static class FailingFileIOFactory implements FileIOFactory { private static final long serialVersionUID = 0L; /** */ - private AtomicBoolean fail; + private final AtomicBoolean fail; /** */ private final FileIOFactory delegateFactory = new RandomAccessFileIOFactory(); @@ -242,6 +257,7 @@ private static class FailingFileIOFactory implements FileIOFactory { return new FileIODecorator(delegate) { int writeAttempts = 2; + /** {@inheritDoc} */ @Override public int write(ByteBuffer srcBuf) throws IOException { if (--writeAttempts <= 0 && fail != null && fail.get()) throw new IOException("No space left on device"); 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 4287a292ddcf5..5bc757200f2b5 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 @@ -42,9 +42,13 @@ 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.IgniteWalFlushBackgroundSelfTest; -import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushDefaultSelfTest; +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.IgniteWalFlushFsyncWithDedicatedWorkerSelfTest; +import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushFsyncWithMmapBufferSelfTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushLogOnlySelfTest; +import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushLogOnlyWithMmapBufferSelfTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalHistoryReservationsTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalIteratorSwitchSegmentTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalSerializerVersionTest; @@ -126,12 +130,20 @@ public static void addRealPageStoreTests(TestSuite suite) { // Failover test suite.addTestSuite(IgniteWalFlushFailoverTest.class); - suite.addTestSuite(IgniteWalFlushDefaultSelfTest.class); + suite.addTestSuite(IgniteWalFlushFsyncSelfTest.class); + + suite.addTestSuite(IgniteWalFlushFsyncWithDedicatedWorkerSelfTest.class); + + suite.addTestSuite(IgniteWalFlushFsyncWithMmapBufferSelfTest.class); suite.addTestSuite(IgniteWalFlushBackgroundSelfTest.class); + suite.addTestSuite(IgniteWalFlushBackgroundWithMmapBufferSelfTest.class); + suite.addTestSuite(IgniteWalFlushLogOnlySelfTest.class); + suite.addTestSuite(IgniteWalFlushLogOnlyWithMmapBufferSelfTest.class); + // Test suite uses Standalone WAL iterator to verify PDS content. suite.addTestSuite(IgniteWalReaderTest.class); From b4723ffea45d711e5a183543966ea193281b70d6 Mon Sep 17 00:00:00 2001 From: dpavlov Date: Fri, 25 May 2018 14:48:04 +0300 Subject: [PATCH 0259/1463] IGNITE-7809 Corrected failure handler to avoid suite timeout (cherry picked from commit 0c3a7a6) --- .../IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java index 9b5b642272ff0..c54ad884f40a1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java @@ -20,8 +20,9 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; - import java.nio.MappedByteBuffer; +import java.nio.file.OpenOption; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteSystemProperties; @@ -31,6 +32,7 @@ import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.configuration.WALMode; +import org.apache.ignite.failure.StopNodeFailureHandler; import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; import org.apache.ignite.internal.processors.cache.persistence.file.FileIO; import org.apache.ignite.internal.processors.cache.persistence.file.FileIODecorator; @@ -45,9 +47,6 @@ import org.apache.ignite.transactions.TransactionConcurrency; import org.apache.ignite.transactions.TransactionIsolation; -import java.nio.file.OpenOption; -import java.util.concurrent.atomic.AtomicBoolean; - import static java.nio.file.StandardOpenOption.CREATE; import static java.nio.file.StandardOpenOption.READ; import static java.nio.file.StandardOpenOption.WRITE; @@ -128,6 +127,8 @@ protected boolean mmap() { cfg.setDataStorageConfiguration(memCfg); + cfg.setFailureHandler(new StopNodeFailureHandler()); + return cfg; } From a579628558067f133103b323bdab4a0b12f3e17c Mon Sep 17 00:00:00 2001 From: dpavlov Date: Wed, 6 Jun 2018 21:09:25 +0300 Subject: [PATCH 0260/1463] IGNITE-8727: Disabling failed tests (cherry picked from commit 095f564) --- ...teWalFlushBackgroundWithMmapBufferSelfTest.java | 14 ++++++++++++++ ...gniteWalFlushLogOnlyWithMmapBufferSelfTest.java | 14 ++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushBackgroundWithMmapBufferSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushBackgroundWithMmapBufferSelfTest.java index 4f021cb9bd665..15d2b7a7258a5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushBackgroundWithMmapBufferSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushBackgroundWithMmapBufferSelfTest.java @@ -25,4 +25,18 @@ public class IgniteWalFlushBackgroundWithMmapBufferSelfTest extends IgniteWalFlu @Override protected boolean mmap() { return true; } + + /** {@inheritDoc} */ + @Override public void testFailWhileStart() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8727"); + + super.testFailWhileStart(); + } + + /** {@inheritDoc} */ + @Override public void testFailAfterStart() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8727"); + + super.testFailAfterStart(); + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushLogOnlyWithMmapBufferSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushLogOnlyWithMmapBufferSelfTest.java index d1f5075b56b4a..ce06b7dc35061 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushLogOnlyWithMmapBufferSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushLogOnlyWithMmapBufferSelfTest.java @@ -25,4 +25,18 @@ public class IgniteWalFlushLogOnlyWithMmapBufferSelfTest extends IgniteWalFlushL @Override protected boolean mmap() { return true; } + + /** {@inheritDoc} */ + @Override public void testFailWhileStart() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8727"); + + super.testFailWhileStart(); + } + + /** {@inheritDoc} */ + @Override public void testFailAfterStart() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8727"); + + super.testFailAfterStart(); + } } From 6ef74cc3706975125d10ad73049f25929d111804 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 15 Jun 2018 17:49:12 +0300 Subject: [PATCH 0261/1463] IGNITE-8727 Fixed WalFlush with MMap tests. Signed-off-by: Andrey Gura (cherry picked from commit 41954cc) --- .../file/FilePageStoreManager.java | 1 - .../file/FileVersionCheckingFactory.java | 7 ++++-- .../wal/FileWriteAheadLogManager.java | 8 +++---- .../FsyncModeFileWriteAheadLogManager.java | 17 ++++++-------- ...FlushBackgroundWithMmapBufferSelfTest.java | 14 ----------- ...WalFlushLogOnlyWithMmapBufferSelfTest.java | 14 ----------- ...lushMultiNodeFailoverAbstractSelfTest.java | 23 +++++++++++++------ .../testsuites/IgnitePdsTestSuite2.java | 14 ++++++----- 8 files changed, 40 insertions(+), 58 deletions(-) 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 d065ff1e1e54b..453e823c7a541 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 @@ -939,5 +939,4 @@ public CacheStoreHolder(FilePageStore idxStore, FilePageStore[] partStores) { this.partStores = partStores; } } - } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileVersionCheckingFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileVersionCheckingFactory.java index c7aaf1bb401fd..ab36d7cbd5201 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileVersionCheckingFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileVersionCheckingFactory.java @@ -52,8 +52,11 @@ public class FileVersionCheckingFactory implements FilePageStoreFactory { * @param fileIOFactoryStoreV1 File IO factory for V1 page store and for version checking. * @param memCfg Memory configuration. */ - public FileVersionCheckingFactory(FileIOFactory fileIOFactory, FileIOFactory fileIOFactoryStoreV1, - DataStorageConfiguration memCfg) { + public FileVersionCheckingFactory( + FileIOFactory fileIOFactory, + FileIOFactory fileIOFactoryStoreV1, + DataStorageConfiguration memCfg + ) { this.fileIOFactory = fileIOFactory; this.fileIOFactoryStoreV1 = fileIOFactoryStoreV1; this.memCfg = memCfg; 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 cb80961215301..59fcf4e05a234 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 @@ -277,7 +277,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl private volatile long lastTruncatedArchiveIdx = -1L; /** Factory to provide I/O interfaces for read/write operations with files */ - private FileIOFactory ioFactory; + private volatile FileIOFactory ioFactory; /** Next WAL segment archived monitor. Manages last archived index, emulates archivation in no-archiver mode. */ private final SegmentArchivedMonitor archivedMonitor = new SegmentArchivedMonitor(); @@ -3291,13 +3291,13 @@ else if (pos == FILE_FORCE) } } } - - unparkWaiters(Long.MAX_VALUE); } catch (Throwable t) { err = t; } finally { + unparkWaiters(Long.MAX_VALUE); + if (err == null && !shutdown) err = new IllegalStateException("Thread " + getName() + " is terminated unexpectedly"); @@ -3458,7 +3458,7 @@ private void writeBuffer(long pos, ByteBuffer buf) throws StorageException, Igni assert hdl.written == hdl.fileIO.position(); } catch (IOException e) { - StorageException se = new StorageException("Unable to write", e); + StorageException se = new StorageException("Failed to write buffer.", e); cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, se)); 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 867f13f69e5c1..c58ab3d22c7c8 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 @@ -230,7 +230,7 @@ public class FsyncModeFileWriteAheadLogManager extends GridCacheSharedManagerAda private volatile long lastTruncatedArchiveIdx = -1L; /** Factory to provide I/O interfaces for read/write operations with files */ - private FileIOFactory ioFactory; + private volatile FileIOFactory ioFactory; /** Updater for {@link #currentHnd}, used for verify there are no concurrent update for current log segment handle */ private static final AtomicReferenceFieldUpdater currentHndUpd = @@ -266,9 +266,6 @@ public class FsyncModeFileWriteAheadLogManager extends GridCacheSharedManagerAda /** Current log segment handle */ private volatile FileWriteHandle currentHnd; - /** Environment failure. */ - private volatile Throwable envFailed; - /** * Positive (non-0) value indicates WAL can be archived even if not complete
    * See {@link DataStorageConfiguration#setWalAutoArchiveAfterInactivity(long)}
    @@ -2412,9 +2409,9 @@ private FileWriteHandle( * Write serializer version to current handle. * NOTE: Method mutates {@code fileIO} position, written and lastFsyncPos fields. * - * @throws IgniteCheckedException If fail to write serializer version. + * @throws IOException If fail to write serializer version. */ - public void writeSerializerVersion() throws IgniteCheckedException { + public void writeSerializerVersion() throws IOException { try { assert fileIO.position() == 0 : "Serializer version can be written only at the begin of file " + fileIO.position(); @@ -2427,7 +2424,7 @@ public void writeSerializerVersion() throws IgniteCheckedException { head.set(new FakeRecord(new FileWALPointer(idx, (int)updatedPosition, 0), false)); } catch (IOException e) { - throw new IgniteCheckedException("Unable to write serializer version for segment " + idx, e); + throw new IOException("Unable to write serializer version for segment " + idx, e); } } @@ -2542,7 +2539,7 @@ else if (stop) { lock.lock(); try { - while (written < expWritten && envFailed == null) + while (written < expWritten && !cctx.kernalContext().invalid()) U.awaitQuiet(writeComplete); } finally { @@ -2858,7 +2855,7 @@ private void signalNextAvailable() { try { WALRecord rec = head.get(); - if (envFailed == null) { + if (!cctx.kernalContext().invalid()) { assert rec instanceof FakeRecord : "Expected head FakeRecord, actual head " + (rec != null ? rec.getClass().getSimpleName() : "null"); @@ -2882,7 +2879,7 @@ private void awaitNext() throws IgniteCheckedException { lock.lock(); try { - while (fileIO != null) + while (fileIO != null && !cctx.kernalContext().invalid()) U.awaitQuiet(nextSegment); } finally { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushBackgroundWithMmapBufferSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushBackgroundWithMmapBufferSelfTest.java index 15d2b7a7258a5..4f021cb9bd665 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushBackgroundWithMmapBufferSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushBackgroundWithMmapBufferSelfTest.java @@ -25,18 +25,4 @@ public class IgniteWalFlushBackgroundWithMmapBufferSelfTest extends IgniteWalFlu @Override protected boolean mmap() { return true; } - - /** {@inheritDoc} */ - @Override public void testFailWhileStart() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-8727"); - - super.testFailWhileStart(); - } - - /** {@inheritDoc} */ - @Override public void testFailAfterStart() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-8727"); - - super.testFailAfterStart(); - } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushLogOnlyWithMmapBufferSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushLogOnlyWithMmapBufferSelfTest.java index ce06b7dc35061..d1f5075b56b4a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushLogOnlyWithMmapBufferSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushLogOnlyWithMmapBufferSelfTest.java @@ -25,18 +25,4 @@ public class IgniteWalFlushLogOnlyWithMmapBufferSelfTest extends IgniteWalFlushL @Override protected boolean mmap() { return true; } - - /** {@inheritDoc} */ - @Override public void testFailWhileStart() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-8727"); - - super.testFailWhileStart(); - } - - /** {@inheritDoc} */ - @Override public void testFailAfterStart() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-8727"); - - super.testFailAfterStart(); - } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java index c54ad884f40a1..d585a82e66a2f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java @@ -27,6 +27,8 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheRebalanceMode; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.DataRegionConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; @@ -59,7 +61,7 @@ public abstract class IgniteWalFlushMultiNodeFailoverAbstractSelfTest extends Gr private static final String TEST_CACHE = "testCache"; /** */ - private static final int ITRS = 1000; + private static final int ITRS = 2000; /** */ private AtomicBoolean canFail = new AtomicBoolean(); @@ -112,9 +114,13 @@ protected boolean mmap() { @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); + cfg.setConsistentId(gridName); + CacheConfiguration cacheCfg = new CacheConfiguration(TEST_CACHE) .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL) - .setBackups(1); + .setBackups(1) + .setRebalanceMode(CacheRebalanceMode.SYNC) + .setAffinity(new RendezvousAffinityFunction(false, 32)); cfg.setCacheConfiguration(cacheCfg); @@ -169,7 +175,8 @@ public void failWhilePut(boolean failWhileStart) throws Exception { tx.commit(); break; - } catch (Exception expected) { + } + catch (Exception expected) { // Expected exception. } } @@ -186,7 +193,8 @@ public void failWhilePut(boolean failWhileStart) throws Exception { grid.cluster().setBaselineTopology(grid.cluster().topologyVersion()); waitForRebalancing(); - } catch (Exception expected) { + } + catch (Throwable expected) { // There can be any exception. Do nothing. } } @@ -256,11 +264,9 @@ private static class FailingFileIOFactory implements FileIOFactory { final FileIO delegate = delegateFactory.create(file, modes); return new FileIODecorator(delegate) { - int writeAttempts = 2; - /** {@inheritDoc} */ @Override public int write(ByteBuffer srcBuf) throws IOException { - if (--writeAttempts <= 0 && fail != null && fail.get()) + if (fail != null && fail.get()) throw new IOException("No space left on device"); return super.write(srcBuf); @@ -268,6 +274,9 @@ private static class FailingFileIOFactory implements FileIOFactory { /** {@inheritDoc} */ @Override public MappedByteBuffer map(int sizeBytes) throws IOException { + if (fail != null && fail.get()) + throw new IOException("No space left on deive"); + return delegate.map(sizeBytes); } }; 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 5bc757200f2b5..27757435a7d9c 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 @@ -103,6 +103,14 @@ private static void addRealPageStoreTestsLongRunning(TestSuite suite) { suite.addTestSuite(IgnitePdsRecoveryAfterFileCorruptionTest.class); suite.addTestSuite(IgnitePdsPartitionFilesDestroyTest.class); + + suite.addTestSuite(LocalWalModeChangeDuringRebalancingSelfTest.class); + + suite.addTestSuite(IgniteWalFlushFsyncSelfTest.class); + + suite.addTestSuite(IgniteWalFlushFsyncWithDedicatedWorkerSelfTest.class); + + suite.addTestSuite(IgniteWalFlushFsyncWithMmapBufferSelfTest.class); } /** @@ -130,12 +138,6 @@ public static void addRealPageStoreTests(TestSuite suite) { // Failover test suite.addTestSuite(IgniteWalFlushFailoverTest.class); - suite.addTestSuite(IgniteWalFlushFsyncSelfTest.class); - - suite.addTestSuite(IgniteWalFlushFsyncWithDedicatedWorkerSelfTest.class); - - suite.addTestSuite(IgniteWalFlushFsyncWithMmapBufferSelfTest.class); - suite.addTestSuite(IgniteWalFlushBackgroundSelfTest.class); suite.addTestSuite(IgniteWalFlushBackgroundWithMmapBufferSelfTest.class); From 82cbf44f745dffbb0bc1cd5348a1bb61af742a92 Mon Sep 17 00:00:00 2001 From: Sergey Chugunov Date: Wed, 6 Jun 2018 17:11:12 +0300 Subject: [PATCH 0262/1463] GG-13865 filling wal segments with non-zero values --- .../wal/FileWriteAheadLogManager.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) 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 59fcf4e05a234..141f4f7d2e065 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 @@ -165,6 +165,24 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl /** */ private static final byte[] FILL_BUF = new byte[1024 * 1024]; + static { + for (int i = 0; i < 32; i++) + FILL_BUF[i] = 0; + + for (int i = 32; i < FILL_BUF.length; i++) { + if (i % 8 == 0 || i % 8 == 3) + FILL_BUF[i] = 0xD; + else if (i % 8 == 1 || i % 8 == 5 || i % 8 == 6) + FILL_BUF[i] = 0xE; + else if (i % 8 == 2) + FILL_BUF[i] = 0xA; + else if (i % 8 == 4) + FILL_BUF[i] = 0xB; + else if (i % 8 == 7) + FILL_BUF[i] = 0xF; + } + } + /** Pattern for segment file names */ private static final Pattern WAL_NAME_PATTERN = Pattern.compile("\\d{16}\\.wal"); From 918f748879182dbfe2a96681775ebddc7504e46d Mon Sep 17 00:00:00 2001 From: Alexey Stelmak Date: Thu, 7 Jun 2018 12:51:57 +0300 Subject: [PATCH 0263/1463] gg-13865 JVM crash if there is no space for wal storage --- .../processors/cache/persistence/file/AsyncFileIO.java | 4 +++- .../internal/processors/cache/persistence/file/FileIO.java | 2 +- .../processors/cache/persistence/file/FileIODecorator.java | 4 ++-- .../cache/persistence/file/RandomAccessFileIO.java | 4 ++-- .../processors/cache/persistence/file/UnzipFileIO.java | 2 +- .../cache/persistence/wal/FileWriteAheadLogManager.java | 3 ++- .../LocalWalModeChangeDuringRebalancingSelfTest.java | 4 ++-- .../db/file/IgnitePdsDiskErrorsRecoveringTest.java | 5 +++-- .../persistence/pagemem/PagesWriteThrottleSmokeTest.java | 4 ++-- .../cache/persistence/file/AlignedBuffersDirectFileIO.java | 4 ++-- 10 files changed, 20 insertions(+), 16 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java index 799a78cb99e60..3e75228618912 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java @@ -154,7 +154,7 @@ public AsyncFileIO(File file, ThreadLocal holder, OpenOption... } /** {@inheritDoc} */ - @Override public void write(byte[] buf, int off, int len) throws IOException { + @Override public int write(byte[] buf, int off, int len) throws IOException { ChannelOpFuture fut = holder.get(); fut.reset(); @@ -166,6 +166,8 @@ public AsyncFileIO(File file, ThreadLocal holder, OpenOption... catch (IgniteCheckedException e) { throw new IOException(e); } + + return 0; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java index 822bd66413dc3..5eae54355f244 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java @@ -122,7 +122,7 @@ public interface FileIO extends AutoCloseable { * * @throws IOException If some I/O error occurs. */ - public void write(byte[] buf, int off, int len) throws IOException; + public int write(byte[] buf, int off, int len) throws IOException; /** * Allocates memory mapped buffer for this file with given size. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java index 683845bc268ba..9c389851ef248 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java @@ -72,8 +72,8 @@ public FileIODecorator(FileIO delegate) { } /** {@inheritDoc} */ - @Override public void write(byte[] buf, int off, int len) throws IOException { - delegate.write(buf, off, len); + @Override public int write(byte[] buf, int off, int len) throws IOException { + return delegate.write(buf, off, len); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java index 8f7454dcae617..018ed276de7ab 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java @@ -79,8 +79,8 @@ public RandomAccessFileIO(File file, OpenOption... modes) throws IOException { } /** {@inheritDoc} */ - @Override public void write(byte[] buf, int off, int len) throws IOException { - ch.write(ByteBuffer.wrap(buf, off, len)); + @Override public int write(byte[] buf, int off, int len) throws IOException { + return ch.write(ByteBuffer.wrap(buf, off, len)); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/UnzipFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/UnzipFileIO.java index 469cf3eb7ebbc..8194ba36eed14 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/UnzipFileIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/UnzipFileIO.java @@ -110,7 +110,7 @@ public UnzipFileIO(File zip) throws IOException { } /** {@inheritDoc} */ - @Override public void write(byte[] buf, int off, int len) throws IOException { + @Override public int write(byte[] buf, int off, int len) throws IOException { throw new UnsupportedOperationException(); } 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 141f4f7d2e065..bab1057eda2c1 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 @@ -1404,7 +1404,8 @@ private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedExc while (left > 0) { int toWrite = Math.min(FILL_BUF.length, left); - fileIO.write(FILL_BUF, 0, toWrite); + if (fileIO.write(FILL_BUF, 0, toWrite) < toWrite) + throw new IgniteCheckedException("Can't extend file: " + file.getName()); left -= toWrite; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java index f365448d40f75..41a18ec2e8c59 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java @@ -590,7 +590,7 @@ private static class TestFileIO implements FileIO { } /** {@inheritDoc} */ - @Override public void write(byte[] buf, int off, int len) throws IOException { + @Override public int write(byte[] buf, int off, int len) throws IOException { CountDownLatch latch = fileIOLatch.get(); if (latch != null && Thread.currentThread().getName().contains("checkpoint")) @@ -601,7 +601,7 @@ private static class TestFileIO implements FileIO { throw new IgniteException(ex); } - delegate.write(buf, off, len); + return delegate.write(buf, off, len); } /** {@inheritDoc} */ 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 c902879cbaae2..6e9ed1fe9318b 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 @@ -429,11 +429,12 @@ public LimitedSizeFileIO(FileIO delegate, AtomicLong availableSpaceBytes) { } /** {@inheritDoc} */ - @Override public void write(byte[] buf, int off, int len) throws IOException { - super.write(buf, off, len); + @Override public int write(byte[] buf, int off, int len) throws IOException { + final int num = super.write(buf, off, len); availableSpaceBytes.addAndGet(-len); if (availableSpaceBytes.get() < 0) throw new IOException("Not enough space!"); + return num; } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java index 249718b02ec06..48ff980d37e46 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java @@ -311,11 +311,11 @@ private static class SlowCheckpointFileIOFactory implements FileIOFactory { return delegate.write(srcBuf, position); } - @Override public void write(byte[] buf, int off, int len) throws IOException { + @Override public int write(byte[] buf, int off, int len) throws IOException { if (slowCheckpointEnabled.get() && Thread.currentThread().getName().contains("checkpoint")) LockSupport.parkNanos(5_000_000); - delegate.write(buf, off, len); + return delegate.write(buf, off, len); } /** {@inheritDoc} */ diff --git a/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIO.java b/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIO.java index 681426cd4c18e..88d77e017f4c0 100644 --- a/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIO.java +++ b/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIO.java @@ -455,8 +455,8 @@ private static String getLastError() { } /** {@inheritDoc} */ - @Override public void write(byte[] buf, int off, int len) throws IOException { - write(ByteBuffer.wrap(buf, off, len)); + @Override public int write(byte[] buf, int off, int len) throws IOException { + return write(ByteBuffer.wrap(buf, off, len)); } /** {@inheritDoc} */ From 5d47fed8c0d6237be13172552b48b113632deb73 Mon Sep 17 00:00:00 2001 From: Alexey Stelmak Date: Thu, 7 Jun 2018 13:13:03 +0300 Subject: [PATCH 0264/1463] gg-13865 JVM crash if there is no space for wal storage --- .../internal/processors/cache/persistence/file/FileIO.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java index 5eae54355f244..50568aff0f14e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java @@ -120,6 +120,8 @@ public interface FileIO extends AutoCloseable { * @param off Start offset in the {@code buffer}. * @param len Number of bytes to write. * + * @return Number of written bytes. + * * @throws IOException If some I/O error occurs. */ public int write(byte[] buf, int off, int len) throws IOException; From 1da633da7e83ddab92f13372cdf22695b53141c8 Mon Sep 17 00:00:00 2001 From: Alexey Stelmak Date: Sat, 9 Jun 2018 15:37:51 +0300 Subject: [PATCH 0265/1463] Add failure handler --- .../wal/FileWriteAheadLogManager.java | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 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 bab1057eda2c1..20889824e220d 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 @@ -71,6 +71,8 @@ import org.apache.ignite.events.EventType; import org.apache.ignite.events.WalSegmentArchivedEvent; import org.apache.ignite.failure.FailureContext; +import org.apache.ignite.failure.FailureType; +import org.apache.ignite.failure.StopNodeFailureHandler; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; @@ -98,6 +100,7 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactory; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactoryImpl; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer; +import org.apache.ignite.internal.processors.failure.FailureProcessor; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.GridUnsafe; @@ -272,6 +275,9 @@ else if (i % 8 == 7) /** Events service */ private final GridEventStorageManager evt; + /** Failure processor */ + private final FailureProcessor failureProcessor; + /** */ private IgniteConfiguration igCfg; @@ -378,6 +384,7 @@ public FileWriteAheadLogManager(@NotNull final GridKernalContext ctx) { ioFactory = new RandomAccessFileIOFactory(); walAutoArchiveAfterInactivity = dsCfg.getWalAutoArchiveAfterInactivity(); evt = ctx.event(); + failureProcessor = ctx.failure(); } /** @@ -1404,8 +1411,11 @@ private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedExc while (left > 0) { int toWrite = Math.min(FILL_BUF.length, left); - if (fileIO.write(FILL_BUF, 0, toWrite) < toWrite) + if (fileIO.write(FILL_BUF, 0, toWrite) < toWrite) { + if (failureProcessor != null) + failureProcessor.process(new FailureContext(FailureType.CRITICAL_ERROR, null), new StopNodeFailureHandler()); throw new IgniteCheckedException("Can't extend file: " + file.getName()); + } left -= toWrite; } @@ -2152,7 +2162,10 @@ private class FileDecompressor extends Thread { int bytesRead; while ((bytesRead = zis.read(arr)) > 0) - io.write(arr, 0, bytesRead); + if (io.write(arr, 0, bytesRead) < bytesRead) { + failureProcessor.process(new FailureContext(FailureType.CRITICAL_ERROR, null), new StopNodeFailureHandler()); + throw new IgniteCheckedException("Can't extend file: " + unzipTmp.getName()); + } } try { From 2e0af995b1a377349b696673a5bab640324881de Mon Sep 17 00:00:00 2001 From: Alexey Stelmak Date: Thu, 14 Jun 2018 15:56:38 +0300 Subject: [PATCH 0266/1463] ignite-8748 Review fix --- .../wal/FileWriteAheadLogManager.java | 24 ++++--------------- 1 file changed, 4 insertions(+), 20 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 20889824e220d..a161b7177f492 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 @@ -168,24 +168,6 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl /** */ private static final byte[] FILL_BUF = new byte[1024 * 1024]; - static { - for (int i = 0; i < 32; i++) - FILL_BUF[i] = 0; - - for (int i = 32; i < FILL_BUF.length; i++) { - if (i % 8 == 0 || i % 8 == 3) - FILL_BUF[i] = 0xD; - else if (i % 8 == 1 || i % 8 == 5 || i % 8 == 6) - FILL_BUF[i] = 0xE; - else if (i % 8 == 2) - FILL_BUF[i] = 0xA; - else if (i % 8 == 4) - FILL_BUF[i] = 0xB; - else if (i % 8 == 7) - FILL_BUF[i] = 0xF; - } - } - /** Pattern for segment file names */ private static final Pattern WAL_NAME_PATTERN = Pattern.compile("\\d{16}\\.wal"); @@ -1412,9 +1394,11 @@ private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedExc int toWrite = Math.min(FILL_BUF.length, left); if (fileIO.write(FILL_BUF, 0, toWrite) < toWrite) { + final IgniteCheckedException ex = new IgniteCheckedException("Failed to extend WAL segment file: " + + file.getName() + ". Probably disk is too busy, please check your device."); if (failureProcessor != null) - failureProcessor.process(new FailureContext(FailureType.CRITICAL_ERROR, null), new StopNodeFailureHandler()); - throw new IgniteCheckedException("Can't extend file: " + file.getName()); + failureProcessor.process(new FailureContext(FailureType.CRITICAL_ERROR, ex), new StopNodeFailureHandler()); + throw ex; } left -= toWrite; From e9e274e8999e176a8d6fb9778dadfa11398b0c13 Mon Sep 17 00:00:00 2001 From: Alexey Stelmak Date: Thu, 14 Jun 2018 17:34:37 +0300 Subject: [PATCH 0267/1463] ignite-8748 Review fix --- .../cache/persistence/wal/FileWriteAheadLogManager.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 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 a161b7177f492..c23c9493393a9 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 @@ -2147,8 +2147,11 @@ private class FileDecompressor extends Thread { int bytesRead; while ((bytesRead = zis.read(arr)) > 0) if (io.write(arr, 0, bytesRead) < bytesRead) { - failureProcessor.process(new FailureContext(FailureType.CRITICAL_ERROR, null), new StopNodeFailureHandler()); - throw new IgniteCheckedException("Can't extend file: " + unzipTmp.getName()); + final IgniteCheckedException ex = new IgniteCheckedException("Failed to extend file: " + + unzipTmp.getName() + ". Probably disk is too busy, please check your device."); + if (failureProcessor != null) + failureProcessor.process(new FailureContext(FailureType.CRITICAL_ERROR, ex), new StopNodeFailureHandler()); + throw ex; } } From 550df383d09b56ea570dd63d0361228a325e1b20 Mon Sep 17 00:00:00 2001 From: Alexey Stelmak Date: Thu, 14 Jun 2018 18:03:24 +0300 Subject: [PATCH 0268/1463] ignite-8748 Code style fix --- .../cache/persistence/wal/FileWriteAheadLogManager.java | 4 ++++ 1 file changed, 4 insertions(+) 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 c23c9493393a9..99ef6b961b71b 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 @@ -1396,8 +1396,10 @@ private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedExc if (fileIO.write(FILL_BUF, 0, toWrite) < toWrite) { final IgniteCheckedException ex = new IgniteCheckedException("Failed to extend WAL segment file: " + file.getName() + ". Probably disk is too busy, please check your device."); + if (failureProcessor != null) failureProcessor.process(new FailureContext(FailureType.CRITICAL_ERROR, ex), new StopNodeFailureHandler()); + throw ex; } @@ -2149,8 +2151,10 @@ private class FileDecompressor extends Thread { if (io.write(arr, 0, bytesRead) < bytesRead) { final IgniteCheckedException ex = new IgniteCheckedException("Failed to extend file: " + unzipTmp.getName() + ". Probably disk is too busy, please check your device."); + if (failureProcessor != null) failureProcessor.process(new FailureContext(FailureType.CRITICAL_ERROR, ex), new StopNodeFailureHandler()); + throw ex; } } From 59cf87ea0f021400f33df7cd82cc94b79263e13d Mon Sep 17 00:00:00 2001 From: Alexey Stelmak Date: Thu, 14 Jun 2018 18:14:17 +0300 Subject: [PATCH 0269/1463] ignite-8748 Use configured failure handler --- .../cache/persistence/wal/FileWriteAheadLogManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 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 99ef6b961b71b..2a77c3311e9e2 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 @@ -1398,7 +1398,7 @@ private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedExc file.getName() + ". Probably disk is too busy, please check your device."); if (failureProcessor != null) - failureProcessor.process(new FailureContext(FailureType.CRITICAL_ERROR, ex), new StopNodeFailureHandler()); + failureProcessor.process(new FailureContext(FailureType.CRITICAL_ERROR, ex)); throw ex; } @@ -2153,7 +2153,7 @@ private class FileDecompressor extends Thread { unzipTmp.getName() + ". Probably disk is too busy, please check your device."); if (failureProcessor != null) - failureProcessor.process(new FailureContext(FailureType.CRITICAL_ERROR, ex), new StopNodeFailureHandler()); + failureProcessor.process(new FailureContext(FailureType.CRITICAL_ERROR, ex)); throw ex; } From 1ce344098e8bfd3031d8e3afde74efdb115d66bf Mon Sep 17 00:00:00 2001 From: Alexey Stelmak Date: Fri, 15 Jun 2018 21:03:34 +0300 Subject: [PATCH 0270/1463] ignite-8740 Return --- .../processors/cache/persistence/file/AsyncFileIO.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java index 3e75228618912..76142bb509626 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java @@ -161,13 +161,11 @@ public AsyncFileIO(File file, ThreadLocal holder, OpenOption... ch.write(ByteBuffer.wrap(buf, off, len), position, this, fut); try { - fut.getUninterruptibly(); + return fut.getUninterruptibly(); } catch (IgniteCheckedException e) { throw new IOException(e); } - - return 0; } /** {@inheritDoc} */ From d4c6a8be200db83a62b7b4679511b377fc1879f9 Mon Sep 17 00:00:00 2001 From: Andrey Kuznetsov Date: Mon, 18 Jun 2018 12:47:25 +0300 Subject: [PATCH 0271/1463] IGNITE-8562 Turn system-critical Ignite threads into GridWorkers Signed-off-by: agura --- .../internal/GridKernalContextImpl.java | 7 +- .../apache/ignite/internal/IgniteKernal.java | 7 +- .../apache/ignite/internal/IgnitionEx.java | 21 +- .../GridCacheDatabaseSharedManager.java | 2 +- .../wal/FileWriteAheadLogManager.java | 83 +++-- .../FsyncModeFileWriteAheadLogManager.java | 51 ++- .../ignite/internal/util/IgniteUtils.java | 5 +- .../ignite/internal/util/StripedExecutor.java | 164 +++++---- .../internal/util/nio/GridNioServer.java | 90 +++-- .../tcp/TcpCommunicationSpi.java | 98 +++--- .../ignite/spi/discovery/tcp/ClientImpl.java | 87 +++-- .../ignite/spi/discovery/tcp/ServerImpl.java | 310 ++++++++++++------ .../IgniteDiagnosticMessagesTest.java | 2 + .../internal/util/StripedExecutorTest.java | 8 +- .../discovery/tcp/TcpDiscoverySelfTest.java | 7 +- .../junits/GridTestKernalContext.java | 1 + 16 files changed, 594 insertions(+), 349 deletions(-) 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 ac4970859d2b9..2f4ecbc7ddfab 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 @@ -363,7 +363,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable /** */ @GridToStringExclude - private final WorkersRegistry workersRegistry = new WorkersRegistry(); + private WorkersRegistry workersRegistry; /** */ private IgniteEx grid; @@ -431,6 +431,7 @@ public GridKernalContextImpl() { * @param schemaExecSvc Schema executor service. * @param customExecSvcs Custom named executors. * @param plugins Plugin providers. + * @param workerRegistry Worker registry. */ @SuppressWarnings("TypeMayBeWeakened") protected GridKernalContextImpl( @@ -455,7 +456,8 @@ protected GridKernalContextImpl( ExecutorService schemaExecSvc, @Nullable Map customExecSvcs, List plugins, - IgnitePredicate clsFilter + IgnitePredicate clsFilter, + WorkersRegistry workerRegistry ) { assert grid != null; assert cfg != null; @@ -480,6 +482,7 @@ protected GridKernalContextImpl( this.qryExecSvc = qryExecSvc; this.schemaExecSvc = schemaExecSvc; this.customExecSvcs = customExecSvcs; + this.workersRegistry = workerRegistry; marshCtx = new MarshallerContextImpl(plugins, clsFilter); 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 cfbefec3e31d4..798644346d79f 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 @@ -759,6 +759,7 @@ private void notifyLifecycleBeansEx(LifecycleEventType evt) { * @param schemaExecSvc Schema executor service. * @param customExecSvcs Custom named executors. * @param errHnd Error handler to use for notification about startup problems. + * @param workerRegistry Worker registry. * @throws IgniteCheckedException Thrown in case of any errors. */ @SuppressWarnings({"CatchGenericClass", "unchecked"}) @@ -780,7 +781,8 @@ public void start( ExecutorService qryExecSvc, ExecutorService schemaExecSvc, @Nullable final Map customExecSvcs, - GridAbsClosure errHnd + GridAbsClosure errHnd, + WorkersRegistry workerRegistry ) throws IgniteCheckedException { gw.compareAndSet(null, new GridKernalGatewayImpl(cfg.getIgniteInstanceName())); @@ -896,7 +898,8 @@ public void start( schemaExecSvc, customExecSvcs, plugins, - MarshallerUtils.classNameFilter(this.getClass().getClassLoader()) + MarshallerUtils.classNameFilter(this.getClass().getClassLoader()), + workerRegistry ); cfg.getMarshaller().setContext(ctx.marshallerContext()); 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 b3c3ee8b5a9b1..4d1724d5afa97 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 @@ -93,7 +93,9 @@ import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.worker.WorkersRegistry; import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.logger.LoggerNodeIdAware; import org.apache.ignite.logger.java.JavaLogger; import org.apache.ignite.marshaller.Marshaller; @@ -1815,17 +1817,20 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { validateThreadPoolSize(cfg.getStripedPoolSize(), "stripedPool"); + WorkersRegistry workerRegistry = new WorkersRegistry(); + stripedExecSvc = new StripedExecutor( cfg.getStripedPoolSize(), cfg.getIgniteInstanceName(), "sys", log, - new Thread.UncaughtExceptionHandler() { - @Override public void uncaughtException(Thread thread, Throwable t) { + new IgniteInClosure() { + @Override public void apply(Throwable t) { if (grid != null) grid.context().failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, t)); } - }); + }, + workerRegistry); // Note that since we use 'LinkedBlockingQueue', number of // maximum threads has no effect. @@ -1867,13 +1872,14 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { cfg.getIgniteInstanceName(), "data-streamer", log, - new Thread.UncaughtExceptionHandler() { - @Override public void uncaughtException(Thread thread, Throwable t) { + new IgniteInClosure() { + @Override public void apply(Throwable t) { if (grid != null) grid.context().failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, t)); } }, - true); + true, + workerRegistry); // Note that we do not pre-start threads here as igfs pool may not be needed. validateThreadPoolSize(cfg.getIgfsThreadPoolSize(), "IGFS"); @@ -2033,7 +2039,8 @@ private void start0(GridStartContext startCtx) throws IgniteCheckedException { @Override public void apply() { startLatch.countDown(); } - } + }, + workerRegistry ); state = STARTED; 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 7d30181c9bf46..4a44f23fe693a 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 @@ -2928,7 +2928,7 @@ public class Checkpointer extends GridWorker { * @param log Logger. */ protected Checkpointer(@Nullable String gridName, String name, IgniteLogger log) { - super(gridName, name, log); + super(gridName, name, log, cctx.kernalContext().workersRegistry()); scheduledCp = new CheckpointProgress(U.currentTimeMillis() + checkpointFreq); 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 2a77c3311e9e2..9b39987edf42a 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 @@ -112,10 +112,12 @@ 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; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.thread.IgniteThread; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -429,7 +431,7 @@ public void setFileIOFactory(FileIOFactory ioFactory) { long lastAbsArchivedIdx = tup == null ? -1 : tup.get2(); if (isArchiverEnabled()) - archiver = new FileArchiver(lastAbsArchivedIdx); + archiver = new FileArchiver(lastAbsArchivedIdx, log); else archiver = null; @@ -439,10 +441,10 @@ public void setFileIOFactory(FileIOFactory ioFactory) { if (dsCfg.isWalCompactionEnabled()) { compressor = new FileCompressor(); - decompressor = new FileDecompressor(); + decompressor = new FileDecompressor(log); if (!cctx.kernalContext().clientNode()) - decompressor.start(); + new IgniteThread(decompressor).start(); } if (mode != WALMode.NONE) { @@ -572,7 +574,7 @@ private void checkWalConfiguration() throws IgniteCheckedException { if (isArchiverEnabled()) { assert archiver != null; - archiver.start(); + new IgniteThread(archiver).start(); } if (compressor != null) @@ -609,10 +611,10 @@ private void checkWalConfiguration() throws IgniteCheckedException { FileWALPointer filePtr = (FileWALPointer)lastPtr; - walWriter = new WALWriter(); + walWriter = new WALWriter(log); if (!mmap) - walWriter.start(); + new IgniteThread(walWriter).start(); currHnd = restoreWriteHandle(filePtr); @@ -1523,7 +1525,7 @@ public long maxWalSegmentSize() { * FileArchiver#lastAbsArchivedIdx})
  • some WAL index was removed from {@link FileArchiver#locked} map
  • * */ - private class FileArchiver extends Thread { + private class FileArchiver extends GridWorker { /** Exception which occurred during initial creation of files or during archiving WAL segment */ private IgniteCheckedException cleanErr; @@ -1551,8 +1553,9 @@ private class FileArchiver extends Thread { /** * */ - private FileArchiver(long lastAbsArchivedIdx) { - super("wal-file-archiver%" + cctx.igniteInstanceName()); + private FileArchiver(long lastAbsArchivedIdx, IgniteLogger log) { + super(cctx.igniteInstanceName(), "wal-file-archiver%" + cctx.igniteInstanceName(), log, + cctx.kernalContext().workersRegistry()); this.lastAbsArchivedIdx = lastAbsArchivedIdx; } @@ -1567,7 +1570,7 @@ private void shutdown() throws IgniteInterruptedCheckedException { notifyAll(); } - U.join(this); + U.join(runner()); } /** @@ -1592,7 +1595,7 @@ private synchronized boolean locked(long absIdx) { } /** {@inheritDoc} */ - @Override public void run() { + @Override protected void body() { try { allocateRemainingFiles(); } @@ -1661,7 +1664,7 @@ private synchronized boolean locked(long absIdx) { } finally { if (err == null && !stopped) - err = new IllegalStateException("Thread " + getName() + " is terminated unexpectedly"); + err = new IllegalStateException("Worker " + name() + " is terminated unexpectedly"); if (err instanceof OutOfMemoryError) cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, err)); @@ -2107,10 +2110,7 @@ private void shutdown() throws IgniteInterruptedCheckedException { /** * Responsible for decompressing previously compressed segments of WAL archive if they are needed for replay. */ - private class FileDecompressor extends Thread { - /** Current thread stopping advice. */ - private volatile boolean stopped; - + private class FileDecompressor extends GridWorker { /** Decompression futures. */ private Map> decompressionFutures = new HashMap<>(); @@ -2121,21 +2121,21 @@ private class FileDecompressor extends Thread { private byte[] arr = new byte[BUF_SIZE]; /** - * + * @param log Logger. */ - FileDecompressor() { - super("wal-file-decompressor%" + cctx.igniteInstanceName()); + FileDecompressor(IgniteLogger log) { + super(cctx.igniteInstanceName(), "wal-file-decompressor%" + cctx.igniteInstanceName(), log); } /** {@inheritDoc} */ - @Override public void run() { - while (!Thread.currentThread().isInterrupted() && !stopped) { + @Override protected void body() { + while (!isCancelled()) { long segmentToDecompress = -1L; try { segmentToDecompress = segmentsQueue.take(); - if (stopped) + if (isCancelled()) break; File zip = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress) + ".zip"); @@ -2143,7 +2143,7 @@ private class FileDecompressor extends Thread { File unzip = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress)); try (ZipInputStream zis = new ZipInputStream(new BufferedInputStream(new FileInputStream(zip))); - FileIO io = ioFactory.create(unzipTmp)) { + FileIO io = ioFactory.create(unzipTmp)) { zis.getNextEntry(); int bytesRead; @@ -2178,7 +2178,7 @@ private class FileDecompressor extends Thread { Thread.currentThread().interrupt(); } catch (Throwable t) { - if (!stopped && segmentToDecompress != -1L) { + if (!isCancelled && segmentToDecompress != -1L) { IgniteCheckedException e = new IgniteCheckedException("Error during WAL segment " + "decompression [segmentIdx=" + segmentToDecompress + "]", t); @@ -2218,13 +2218,13 @@ synchronized IgniteInternalFuture decompressFile(long idx) { */ private void shutdown() throws IgniteInterruptedCheckedException { synchronized (this) { - stopped = true; + U.cancel(this); // Put fake -1 to wake thread from queue.take() segmentsQueue.put(-1L); } - U.join(this); + U.join(this, log); } } @@ -3212,7 +3212,7 @@ private void doFlush() { * WAL writer worker. */ @SuppressWarnings("ForLoopReplaceableByForEach") - private class WALWriter extends Thread { + private class WALWriter extends GridWorker { /** Unconditional flush. */ private static final long UNCONDITIONAL_FLUSH = -1L; @@ -3222,9 +3222,6 @@ private class WALWriter extends Thread { /** File force. */ private static final long FILE_FORCE = -3L; - /** Shutdown. */ - private volatile boolean shutdown; - /** Err. */ private volatile Throwable err; @@ -3234,19 +3231,22 @@ private class WALWriter extends Thread { /** * Default constructor. + * + * @param log Logger. */ - WALWriter() { - super("wal-write-worker%" + cctx.igniteInstanceName()); + WALWriter(IgniteLogger log) { + super(cctx.igniteInstanceName(), "wal-write-worker%" + cctx.igniteInstanceName(), log, + cctx.kernalContext().workersRegistry()); } /** {@inheritDoc} */ - @Override public void run() { + @Override protected void body() { Throwable err = null; try { - while (!shutdown && !Thread.currentThread().isInterrupted()) { + while (!isCancelled()) { while (waiters.isEmpty()) { - if (!shutdown) + if (!isCancelled()) LockSupport.park(); else { unparkWaiters(Long.MAX_VALUE); @@ -3321,8 +3321,8 @@ else if (pos == FILE_FORCE) finally { unparkWaiters(Long.MAX_VALUE); - if (err == null && !shutdown) - err = new IllegalStateException("Thread " + getName() + " is terminated unexpectedly"); + if (err == null && !isCancelled) + err = new IllegalStateException("Worker " + name() + " is terminated unexpectedly"); if (err instanceof OutOfMemoryError) cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, err)); @@ -3335,11 +3335,11 @@ else if (err != null) * Shutdowns thread. */ public void shutdown() throws IgniteInterruptedCheckedException { - shutdown = true; + U.cancel(this); - LockSupport.unpark(this); + LockSupport.unpark(runner()); - U.join(this); + U.join(runner()); } /** @@ -3386,7 +3386,6 @@ void flushAll() throws IgniteCheckedException { /** * @param expPos Expected position. */ - @SuppressWarnings("ForLoopReplaceableByForEach") void flushBuffer(long expPos) throws StorageException, IgniteCheckedException { if (mmap) return; @@ -3403,7 +3402,7 @@ void flushBuffer(long expPos) throws StorageException, IgniteCheckedException { waiters.put(t, expPos); - LockSupport.unpark(walWriter); + LockSupport.unpark(walWriter.runner()); while (true) { Long val = waiters.get(t); 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 c58ab3d22c7c8..49fbc73b9a48c 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 @@ -104,10 +104,12 @@ 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; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.thread.IgniteThread; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -373,12 +375,12 @@ public void setFileIOFactory(FileIOFactory ioFactory) { lastTruncatedArchiveIdx = tup == null ? -1 : tup.get1() - 1; - archiver = new FileArchiver(tup == null ? -1 : tup.get2()); + archiver = new FileArchiver(tup == null ? -1 : tup.get2(), log); if (dsCfg.isWalCompactionEnabled()) { compressor = new FileCompressor(); - decompressor = new FileDecompressor(); + decompressor = new FileDecompressor(log); } if (mode != WALMode.NONE) { @@ -452,13 +454,13 @@ private void checkWalConfiguration() throws IgniteCheckedException { if (!cctx.kernalContext().clientNode()) { assert archiver != null; - archiver.start(); + new IgniteThread(archiver).start(); if (compressor != null) compressor.start(); if (decompressor != null) - decompressor.start(); + new IgniteThread(decompressor).start(); } } @@ -1314,7 +1316,7 @@ private void checkNode() throws StorageException { *
  • some WAL index was removed from {@link FileArchiver#locked} map
  • * */ - private class FileArchiver extends Thread { + private class FileArchiver extends GridWorker { /** Exception which occurred during initial creation of files or during archiving WAL segment */ private IgniteCheckedException cleanException; @@ -1345,8 +1347,9 @@ private class FileArchiver extends Thread { /** * */ - private FileArchiver(long lastAbsArchivedIdx) { - super("wal-file-archiver%" + cctx.igniteInstanceName()); + private FileArchiver(long lastAbsArchivedIdx, IgniteLogger log) { + super(cctx.igniteInstanceName(), "wal-file-archiver%" + cctx.igniteInstanceName(), log, + cctx.kernalContext().workersRegistry()); this.lastAbsArchivedIdx = lastAbsArchivedIdx; } @@ -1368,7 +1371,7 @@ private void shutdown() throws IgniteInterruptedCheckedException { notifyAll(); } - U.join(this); + U.join(runner()); } /** @@ -1419,7 +1422,7 @@ private synchronized void release(long absIdx) { } /** {@inheritDoc} */ - @Override public void run() { + @Override protected void body() { try { allocateRemainingFiles(); } @@ -1468,7 +1471,6 @@ private synchronized void release(long absIdx) { while (locked.containsKey(toArchive) && !stopped) wait(); - // Then increase counter to allow rollover on clean working file changeLastArchivedIndexAndWakeupCompressor(toArchive); notifyAll(); @@ -1488,7 +1490,7 @@ private synchronized void release(long absIdx) { } finally { if (err == null && !stopped) - err = new IllegalStateException("Thread " + getName() + " is terminated unexpectedly"); + err = new IllegalStateException("Worker " + name() + " is terminated unexpectedly"); if (err instanceof OutOfMemoryError) cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, err)); @@ -1910,10 +1912,7 @@ private void shutdown() throws IgniteInterruptedCheckedException { /** * Responsible for decompressing previously compressed segments of WAL archive if they are needed for replay. */ - private class FileDecompressor extends Thread { - /** Current thread stopping advice. */ - private volatile boolean stopped; - + private class FileDecompressor extends GridWorker { /** Decompression futures. */ private Map> decompressionFutures = new HashMap<>(); @@ -1924,21 +1923,21 @@ private class FileDecompressor extends Thread { private byte[] arr = new byte[tlbSize]; /** - * + * @param log Logger. */ - FileDecompressor() { - super("wal-file-decompressor%" + cctx.igniteInstanceName()); + FileDecompressor(IgniteLogger log) { + super(cctx.igniteInstanceName(), "wal-file-decompressor%" + cctx.igniteInstanceName(), log); } /** {@inheritDoc} */ - @Override public void run() { - while (!Thread.currentThread().isInterrupted() && !stopped) { + @Override protected void body() { + while (!isCancelled()) { long segmentToDecompress = -1L; try { segmentToDecompress = segmentsQueue.take(); - if (stopped) + if (isCancelled()) break; File zip = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress) + ".zip"); @@ -1946,7 +1945,7 @@ private class FileDecompressor extends Thread { File unzip = new File(walArchiveDir, FileDescriptor.fileName(segmentToDecompress)); try (ZipInputStream zis = new ZipInputStream(new BufferedInputStream(new FileInputStream(zip))); - FileIO io = ioFactory.create(unzipTmp)) { + FileIO io = ioFactory.create(unzipTmp)) { zis.getNextEntry(); int bytesRead; @@ -1973,7 +1972,7 @@ private class FileDecompressor extends Thread { Thread.currentThread().interrupt(); } catch (Throwable t) { - if (!stopped && segmentToDecompress != -1L) { + if (!isCancelled && segmentToDecompress != -1L) { IgniteCheckedException e = new IgniteCheckedException("Error during WAL segment " + "decompression [segmentIdx=" + segmentToDecompress + ']', t); @@ -2011,15 +2010,15 @@ synchronized IgniteInternalFuture decompressFile(long idx) { /** * @throws IgniteInterruptedCheckedException If failed to wait for thread shutdown. */ - private void shutdown() throws IgniteInterruptedCheckedException { + private void shutdown() { synchronized (this) { - stopped = true; + U.cancel(this); // Put fake -1 to wake thread from queue.take() segmentsQueue.put(-1L); } - U.join(this); + U.join(this, log); } } 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 1e34c2d66a93e..e02fe4375ab2c 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 @@ -7698,7 +7698,10 @@ public static T get(Future fut) throws IgniteCheckedException { * @param t Thread. * @throws org.apache.ignite.internal.IgniteInterruptedCheckedException Wrapped {@link InterruptedException}. */ - public static void join(Thread t) throws IgniteInterruptedCheckedException { + public static void join(@Nullable Thread t) throws IgniteInterruptedCheckedException { + if (t == null) + return; + try { t.join(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java index c6383ee41319e..904b8d1a5f48b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java @@ -35,14 +35,16 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.locks.LockSupport; -import org.apache.ignite.IgniteInterruptedException; import org.apache.ignite.IgniteLogger; -import org.apache.ignite.internal.managers.communication.GridIoPolicy; import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.internal.managers.communication.GridIoPolicy; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.util.worker.GridWorker; +import org.apache.ignite.internal.util.worker.GridWorkerListener; +import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.thread.IgniteThread; import org.jetbrains.annotations.NotNull; @@ -64,11 +66,18 @@ public class StripedExecutor implements ExecutorService { * @param igniteInstanceName Node name. * @param poolName Pool name. * @param log Logger. - * @param errHnd Exception handler. + * @param errHnd Critical failure handler. + * @param gridWorkerLsnr listener to link with every stripe worker. */ - public StripedExecutor(int cnt, String igniteInstanceName, String poolName, final IgniteLogger log, - Thread.UncaughtExceptionHandler errHnd) { - this(cnt, igniteInstanceName, poolName, log, errHnd, false); + public StripedExecutor( + int cnt, + String igniteInstanceName, + String poolName, + final IgniteLogger log, + IgniteInClosure errHnd, + GridWorkerListener gridWorkerLsnr + ) { + this(cnt, igniteInstanceName, poolName, log, errHnd, false, gridWorkerLsnr); } /** @@ -76,11 +85,19 @@ public StripedExecutor(int cnt, String igniteInstanceName, String poolName, fina * @param igniteInstanceName Node name. * @param poolName Pool name. * @param log Logger. - * @param errHnd Exception handler. + * @param errHnd Critical failure handler. * @param stealTasks {@code True} to steal tasks. + * @param gridWorkerLsnr listener to link with every stripe worker. */ - public StripedExecutor(int cnt, String igniteInstanceName, String poolName, final IgniteLogger log, - Thread.UncaughtExceptionHandler errHnd, boolean stealTasks) { + public StripedExecutor( + int cnt, + String igniteInstanceName, + String poolName, + final IgniteLogger log, + IgniteInClosure errHnd, + boolean stealTasks, + GridWorkerListener gridWorkerLsnr + ) { A.ensure(cnt > 0, "cnt > 0"); boolean success = false; @@ -96,8 +113,8 @@ public StripedExecutor(int cnt, String igniteInstanceName, String poolName, fina try { for (int i = 0; i < cnt; i++) { stripes[i] = stealTasks - ? new StripeConcurrentQueue(igniteInstanceName, poolName, i, log, stripes, errHnd) - : new StripeConcurrentQueue(igniteInstanceName, poolName, i, log, errHnd); + ? new StripeConcurrentQueue(igniteInstanceName, poolName, i, log, stripes, errHnd, gridWorkerLsnr) + : new StripeConcurrentQueue(igniteInstanceName, poolName, i, log, errHnd, gridWorkerLsnr); } for (int i = 0; i < cnt; i++) @@ -112,15 +129,11 @@ public StripedExecutor(int cnt, String igniteInstanceName, String poolName, fina } finally { if (!success) { - for (Stripe stripe : stripes) { - if (stripe != null) - stripe.signalStop(); - } + for (Stripe stripe : stripes) + U.cancel(stripe); - for (Stripe stripe : stripes) { - if (stripe != null) - stripe.awaitStop(); - } + for (Stripe stripe : stripes) + U.join(stripe, log); } } } @@ -221,7 +234,7 @@ public void execute(int idx, Runnable cmd) { /** {@inheritDoc} */ @Override public boolean isShutdown() { for (Stripe stripe : stripes) { - if (stripe != null && stripe.stopping) + if (stripe != null && stripe.isCancelled()) return true; } @@ -252,15 +265,15 @@ public void stop() { */ private void signalStop() { for (Stripe stripe : stripes) - stripe.signalStop(); + U.cancel(stripe); } /** - * @throws IgniteInterruptedException If interrupted. + * Waits for all stripes to stop. */ - private void awaitStop() throws IgniteInterruptedException { + private void awaitStop() { for (Stripe stripe : stripes) - stripe.awaitStop(); + U.join(stripe, log); } /** @@ -407,22 +420,16 @@ public int[] stripesQueueSizes() { /** * Stripe. */ - private static abstract class Stripe implements Runnable { + private static abstract class Stripe extends GridWorker { /** */ private final String igniteInstanceName; - /** */ - private final String poolName; - /** */ protected final int idx; /** */ private final IgniteLogger log; - /** Stopping flag. */ - private volatile boolean stopping; - /** */ private volatile long completedCnt; @@ -432,8 +439,8 @@ private static abstract class Stripe implements Runnable { /** Thread executing the loop. */ protected Thread thread; - /** Exception handler. */ - private Thread.UncaughtExceptionHandler errHnd; + /** Critical failure handler. */ + private IgniteInClosure errHnd; /** * @param igniteInstanceName Ignite instance name. @@ -441,16 +448,19 @@ private static abstract class Stripe implements Runnable { * @param idx Stripe index. * @param log Logger. * @param errHnd Exception handler. + * @param gridWorkerLsnr listener to link with stripe worker. */ public Stripe( String igniteInstanceName, String poolName, int idx, IgniteLogger log, - Thread.UncaughtExceptionHandler errHnd + IgniteInClosure errHnd, + GridWorkerListener gridWorkerLsnr ) { + super(igniteInstanceName, poolName + "-stripe-" + idx, log, gridWorkerLsnr); + this.igniteInstanceName = igniteInstanceName; - this.poolName = poolName; this.idx = idx; this.log = log; this.errHnd = errHnd; @@ -461,44 +471,19 @@ public Stripe( */ void start() { thread = new IgniteThread(igniteInstanceName, - poolName + "-stripe-" + idx, + name(), this, IgniteThread.GRP_IDX_UNASSIGNED, idx, GridIoPolicy.UNDEFINED); - thread.setUncaughtExceptionHandler(errHnd); - thread.start(); } - /** - * Stop the stripe. - */ - void signalStop() { - stopping = true; - - U.interrupt(thread); - } - - /** - * Await thread stop. - */ - void awaitStop() { - try { - if (thread != null) - thread.join(); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - - throw new IgniteInterruptedException(e); - } - } - /** {@inheritDoc} */ - @Override public void run() { - while (!stopping) { + @SuppressWarnings("NonAtomicOperationOnVolatileField") + @Override public void body() { + while (!isCancelled()) { Runnable cmd; try { @@ -517,25 +502,21 @@ void awaitStop() { } } catch (InterruptedException ignored) { - stopping = true; - Thread.currentThread().interrupt(); - return; + break; } catch (Throwable e) { - if (e instanceof OutOfMemoryError) { - // Re-throwing to exploit uncaught exception handler. - throw e; - } + if (e instanceof OutOfMemoryError) + errHnd.apply(e); U.error(log, "Failed to execute runnable.", e); } } - if (!stopping) { - throw new IllegalStateException("Thread " + Thread.currentThread().getName() + - " is terminated unexpectedly"); + if (!isCancelled) { + errHnd.apply(new IllegalStateException("Thread " + Thread.currentThread().getName() + + " is terminated unexpectedly")); } } @@ -592,16 +573,18 @@ private static class StripeConcurrentQueue extends Stripe { * @param poolName Pool name. * @param idx Stripe index. * @param log Logger. - * @param errHnd Exception handler. + * @param errHnd Critical failure handler. + * @param gridWorkerLsnr listener to link with stripe worker. */ StripeConcurrentQueue( String igniteInstanceName, String poolName, int idx, IgniteLogger log, - Thread.UncaughtExceptionHandler errHnd + IgniteInClosure errHnd, + GridWorkerListener gridWorkerLsnr ) { - this(igniteInstanceName, poolName, idx, log, null, errHnd); + this(igniteInstanceName, poolName, idx, log, null, errHnd, gridWorkerLsnr); } /** @@ -609,7 +592,8 @@ private static class StripeConcurrentQueue extends Stripe { * @param poolName Pool name. * @param idx Stripe index. * @param log Logger. - * @param errHnd Exception handler. + * @param errHnd Critical failure handler. + * @param gridWorkerLsnr listener to link with stripe worker. */ StripeConcurrentQueue( String igniteInstanceName, @@ -617,14 +601,16 @@ private static class StripeConcurrentQueue extends Stripe { int idx, IgniteLogger log, Stripe[] others, - Thread.UncaughtExceptionHandler errHnd + IgniteInClosure errHnd, + GridWorkerListener gridWorkerLsnr ) { super( igniteInstanceName, poolName, idx, log, - errHnd); + errHnd, + gridWorkerLsnr); this.others = others; @@ -723,20 +709,23 @@ private static class StripeConcurrentQueueNoPark extends Stripe { * @param poolName Pool name. * @param idx Stripe index. * @param log Logger. - * @param errHnd Exception handler. + * @param errHnd Critical failure handler. + * @param gridWorkerLsnr listener to link with stripe worker. */ public StripeConcurrentQueueNoPark( String igniteInstanceName, String poolName, int idx, IgniteLogger log, - Thread.UncaughtExceptionHandler errHnd + IgniteInClosure errHnd, + GridWorkerListener gridWorkerLsnr ) { super(igniteInstanceName, poolName, idx, log, - errHnd); + errHnd, + gridWorkerLsnr); } /** {@inheritDoc} */ @@ -782,20 +771,23 @@ private static class StripeConcurrentBlockingQueue extends Stripe { * @param poolName Pool name. * @param idx Stripe index. * @param log Logger. - * @param errHnd Exception handler. + * @param errHnd Critical failure handler. + * @param gridWorkerLsnr listener to link with stripe worker. */ public StripeConcurrentBlockingQueue( String igniteInstanceName, String poolName, int idx, IgniteLogger log, - Thread.UncaughtExceptionHandler errHnd + IgniteInClosure errHnd, + GridWorkerListener gridWorkerLsnr ) { super(igniteInstanceName, poolName, idx, log, - errHnd); + errHnd, + gridWorkerLsnr); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java index 03870dbb3e0d9..85332f4d85235 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java @@ -66,6 +66,7 @@ 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.internal.util.worker.GridWorkerListener; import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgnitePredicate; @@ -142,9 +143,9 @@ public class GridNioServer { /** Defines how many times selector should do {@code selectNow()} before doing {@code select(long)}. */ private long selectorSpins; - /** Accept worker thread. */ + /** Accept worker. */ @GridToStringExclude - private final IgniteThread acceptThread; + private final GridNioAcceptWorker acceptWorker; /** Read worker threads. */ private final IgniteThread[] clientThreads; @@ -267,6 +268,7 @@ public class GridNioServer { * @param skipRecoveryPred Skip recovery predicate. * @param msgQueueLsnr Message queue size listener. * @param readWriteSelectorsAssign If {@code true} then in/out connections are assigned to even/odd workers. + * @param workerLsnr Worker lifecycle listener. * @param filters Filters for this server. * @throws IgniteCheckedException If failed. */ @@ -292,6 +294,7 @@ private GridNioServer( IgnitePredicate skipRecoveryPred, IgniteBiInClosure msgQueueLsnr, boolean readWriteSelectorsAssign, + @Nullable GridWorkerListener workerLsnr, GridNioFilter... filters ) throws IgniteCheckedException { if (port != -1) @@ -345,13 +348,11 @@ private GridNioServer( else threadName = "nio-acceptor-" + srvName; - GridNioAcceptWorker w = new GridNioAcceptWorker(igniteInstanceName, threadName, log, acceptSelector); - - acceptThread = new IgniteThread(w); + acceptWorker = new GridNioAcceptWorker(igniteInstanceName, threadName, log, acceptSelector, workerLsnr); } else { locAddr = null; - acceptThread = null; + acceptWorker = null; } clientWorkers = new ArrayList<>(selectorCnt); @@ -366,8 +367,8 @@ private GridNioServer( threadName = "grid-nio-worker-" + srvName + "-" + i; AbstractNioClientWorker worker = directMode ? - new DirectNioClientWorker(i, igniteInstanceName, threadName, log) : - new ByteBufferNioClientWorker(i, igniteInstanceName, threadName, log); + new DirectNioClientWorker(i, igniteInstanceName, threadName, log, workerLsnr) : + new ByteBufferNioClientWorker(i, igniteInstanceName, threadName, log, workerLsnr); clientWorkers.add(worker); @@ -437,8 +438,8 @@ public static Builder builder() { public void start() { filterChain.start(); - if (acceptThread != null) - acceptThread.start(); + if (acceptWorker != null) + new IgniteThread(acceptWorker).start(); for (IgniteThread thread : clientThreads) thread.start(); @@ -452,8 +453,8 @@ public void stop() { closed = true; // Make sure to entirely stop acceptor if any. - U.interrupt(acceptThread); - U.join(acceptThread, log); + U.cancel(acceptWorker); + U.join(acceptWorker, log); U.cancel(clientWorkers); U.join(clientWorkers, log); @@ -1054,11 +1055,17 @@ private class ByteBufferNioClientWorker extends AbstractNioClientWorker { * @param igniteInstanceName Ignite instance name. * @param name Worker name. * @param log Logger. + * @param workerLsnr Worker lifecycle listener. * @throws IgniteCheckedException If selector could not be created. */ - protected ByteBufferNioClientWorker(int idx, @Nullable String igniteInstanceName, String name, IgniteLogger log) - throws IgniteCheckedException { - super(idx, igniteInstanceName, name, log); + protected ByteBufferNioClientWorker( + int idx, + @Nullable String igniteInstanceName, + String name, + IgniteLogger log, + @Nullable GridWorkerListener workerLsnr + ) throws IgniteCheckedException { + super(idx, igniteInstanceName, name, log, workerLsnr); readBuf = directBuf ? ByteBuffer.allocateDirect(8 << 10) : ByteBuffer.allocate(8 << 10); @@ -1224,11 +1231,17 @@ private class DirectNioClientWorker extends AbstractNioClientWorker { * @param igniteInstanceName Ignite instance name. * @param name Worker name. * @param log Logger. + * @param workerLsnr Worker lifecycle listener. * @throws IgniteCheckedException If selector could not be created. */ - protected DirectNioClientWorker(int idx, @Nullable String igniteInstanceName, String name, IgniteLogger log) - throws IgniteCheckedException { - super(idx, igniteInstanceName, name, log); + protected DirectNioClientWorker( + int idx, + @Nullable String igniteInstanceName, + String name, + IgniteLogger log, + @Nullable GridWorkerListener workerLsnr + ) throws IgniteCheckedException { + super(idx, igniteInstanceName, name, log, workerLsnr); } /** @@ -1747,11 +1760,17 @@ private abstract class AbstractNioClientWorker extends GridWorker implements Gri * @param igniteInstanceName Ignite instance name. * @param name Worker name. * @param log Logger. + * @param workerLsnr Worker lifecycle listener. * @throws IgniteCheckedException If selector could not be created. */ - protected AbstractNioClientWorker(int idx, @Nullable String igniteInstanceName, String name, IgniteLogger log) - throws IgniteCheckedException { - super(igniteInstanceName, name, log); + AbstractNioClientWorker( + int idx, + @Nullable String igniteInstanceName, + String name, + IgniteLogger log, + @Nullable GridWorkerListener workerLsnr + ) throws IgniteCheckedException { + super(igniteInstanceName, name, log, workerLsnr); createSelector(); @@ -2118,6 +2137,10 @@ private void bodyInternal() throws IgniteCheckedException, InterruptedException else processSelectedKeysOptimized(selectedKeys.flip()); } + + // select() call above doesn't throw on interruption; checking it here to propagate timely. + if (!closed && !isCancelled && Thread.interrupted()) + throw new InterruptedException(); } finally { select = false; @@ -2807,11 +2830,16 @@ private class GridNioAcceptWorker extends GridWorker { * @param name Thread name. * @param log Log. * @param selector Which will accept incoming connections. + * @param workerLsnr Worker lifecycle listener. */ protected GridNioAcceptWorker( - @Nullable String igniteInstanceName, String name, IgniteLogger log, Selector selector + @Nullable String igniteInstanceName, + String name, + IgniteLogger log, + Selector selector, + @Nullable GridWorkerListener workerLsnr ) { - super(igniteInstanceName, name, log); + super(igniteInstanceName, name, log, workerLsnr); this.selector = selector; } @@ -2823,7 +2851,7 @@ protected GridNioAcceptWorker( try { boolean reset = false; - while (!closed && !Thread.currentThread().isInterrupted()) { + while (!closed && !isCancelled()) { try { if (reset) selector = createSelector(locAddr); @@ -3605,6 +3633,9 @@ public static class Builder { /** */ private boolean readWriteSelectorsAssign; + /** Worker lifecycle listener to be used by server's worker threads. */ + private GridWorkerListener workerLsnr; + /** * Finishes building the instance. * @@ -3634,6 +3665,7 @@ public GridNioServer build() throws IgniteCheckedException { skipRecoveryPred, msgQueueLsnr, readWriteSelectorsAssign, + workerLsnr, filters != null ? Arrays.copyOf(filters, filters.length) : EMPTY_FILTERS ); @@ -3888,6 +3920,16 @@ public Builder messageQueueSizeListener(IgniteBiInClosure workerListener(GridWorkerListener workerLsnr) { + this.workerLsnr = workerLsnr; + + return this; + } } /** 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 7583d96dd4af2..3eab09bb24d5b 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 @@ -805,7 +805,8 @@ else if (connKey.dummy()) { /** {@inheritDoc} */ @Override public void onFailure(FailureType failureType, Throwable failure) { - ((IgniteEx)ignite).context().failure().process(new FailureContext(failureType, failure)); + if (ignite instanceof IgniteEx) + ((IgniteEx)ignite).context().failure().process(new FailureContext(failureType, failure)); } /** @@ -2219,9 +2220,13 @@ public int boundPort() { nioSrvr.start(); - commWorker = new CommunicationWorker(igniteInstanceName); + commWorker = new CommunicationWorker(igniteInstanceName, log); - commWorker.start(); + new IgniteSpiThread(igniteInstanceName, commWorker.name(), log) { + @Override protected void body() { + commWorker.run(); + } + }.start(); // Ack start. if (log.isDebugEnabled()) @@ -2367,31 +2372,37 @@ private GridNioServer resetNioServer() throws IgniteCheckedException { new GridConnectionBytesVerifyFilter(log) }; - GridNioServer srvr = - GridNioServer.builder() - .address(locHost) - .port(port) - .listener(srvLsnr) - .logger(log) - .selectorCount(selectorsCnt) - .igniteInstanceName(igniteInstanceName) - .serverName("tcp-comm") - .tcpNoDelay(tcpNoDelay) - .directBuffer(directBuf) - .byteOrder(ByteOrder.nativeOrder()) - .socketSendBufferSize(sockSndBuf) - .socketReceiveBufferSize(sockRcvBuf) - .sendQueueLimit(msgQueueLimit) - .directMode(true) - .metricsListener(metricsLsnr) - .writeTimeout(sockWriteTimeout) - .selectorSpins(selectorSpins) - .filters(filters) - .writerFactory(writerFactory) - .skipRecoveryPredicate(skipRecoveryPred) - .messageQueueSizeListener(queueSizeMonitor) - .readWriteSelectorsAssign(usePairedConnections) - .build(); + GridNioServer.Builder builder = GridNioServer.builder() + .address(locHost) + .port(port) + .listener(srvLsnr) + .logger(log) + .selectorCount(selectorsCnt) + .igniteInstanceName(igniteInstanceName) + .serverName("tcp-comm") + .tcpNoDelay(tcpNoDelay) + .directBuffer(directBuf) + .byteOrder(ByteOrder.nativeOrder()) + .socketSendBufferSize(sockSndBuf) + .socketReceiveBufferSize(sockRcvBuf) + .sendQueueLimit(msgQueueLimit) + .directMode(true) + .metricsListener(metricsLsnr) + .writeTimeout(sockWriteTimeout) + .selectorSpins(selectorSpins) + .filters(filters) + .writerFactory(writerFactory) + .skipRecoveryPredicate(skipRecoveryPred) + .messageQueueSizeListener(queueSizeMonitor) + .readWriteSelectorsAssign(usePairedConnections); + + if (ignite instanceof IgniteEx) { + IgniteEx igniteEx = (IgniteEx)ignite; + + builder.workerListener(igniteEx.context().workersRegistry()); + } + + GridNioServer srvr = builder.build(); boundTcpPort = port; @@ -2492,7 +2503,7 @@ private GridNioServer resetNioServer() throws IgniteCheckedException { if (nioSrvr != null) nioSrvr.stop(); - U.interrupt(commWorker); + U.cancel(commWorker); U.join(commWorker, log); U.cancel(shmemAcceptWorker); @@ -3856,7 +3867,8 @@ public void simulateNodeFailure() { if (nioSrvr != null) nioSrvr.stop(); - U.interrupt(commWorker); + if (commWorker != null) + U.interrupt(commWorker.runner()); U.join(commWorker, log); @@ -4216,15 +4228,17 @@ private ShmemWorker(IpcEndpoint endpoint) { /** * */ - private class CommunicationWorker extends IgniteSpiThread { + private class CommunicationWorker extends GridWorker { /** */ private final BlockingQueue q = new LinkedBlockingQueue<>(); /** * @param igniteInstanceName Ignite instance name. + * @param log Logger. */ - private CommunicationWorker(String igniteInstanceName) { - super(igniteInstanceName, "tcp-comm-worker", log); + private CommunicationWorker(String igniteInstanceName, IgniteLogger log) { + super(igniteInstanceName, "tcp-comm-worker", log, + ignite instanceof IgniteEx ? ((IgniteEx)ignite).context().workersRegistry() : null); } /** {@inheritDoc} */ @@ -4235,7 +4249,7 @@ private CommunicationWorker(String igniteInstanceName) { Throwable err = null; try { - while (!isInterrupted()) { + while (!isCancelled()) { DisconnectedSessionInfo disconnectData = q.poll(idleConnTimeout, TimeUnit.MILLISECONDS); if (disconnectData != null) @@ -4251,13 +4265,15 @@ private CommunicationWorker(String igniteInstanceName) { throw t; } finally { - if (err == null && !stopping) - err = new IllegalStateException("Thread " + getName() + " is terminated unexpectedly."); - - if (err instanceof OutOfMemoryError) - ((IgniteEx)ignite).context().failure().process(new FailureContext(CRITICAL_ERROR, err)); - else if (err != null) - ((IgniteEx)ignite).context().failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, err)); + if (ignite instanceof IgniteEx) { + if (err == null && !stopping) + err = new IllegalStateException("Thread " + getName() + " is terminated unexpectedly."); + + if (err instanceof OutOfMemoryError) + ((IgniteEx)ignite).context().failure().process(new FailureContext(CRITICAL_ERROR, err)); + else if (err != null) + ((IgniteEx)ignite).context().failure().process(new FailureContext(SYSTEM_WORKER_TERMINATION, err)); + } } } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java index b22a3970f2710..dc62bf3331fb2 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java @@ -26,7 +26,6 @@ import java.net.SocketTimeoutException; import java.util.ArrayDeque; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -42,11 +41,13 @@ import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.BlockingDeque; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.atomic.AtomicReference; import javax.net.ssl.SSLException; +import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteClientDisconnectedException; import org.apache.ignite.IgniteException; @@ -55,7 +56,9 @@ import org.apache.ignite.cache.CacheMetrics; import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.failure.FailureContext; import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; +import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.internal.managers.discovery.CustomMessageWrapper; @@ -68,6 +71,8 @@ import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.util.worker.GridWorker; +import org.apache.ignite.internal.worker.WorkersRegistry; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.spi.IgniteSpiContext; @@ -104,7 +109,6 @@ import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryServerOnlyCustomEventMessage; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.concurrent.ConcurrentHashMap; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISCO_FAILED_CLIENT_RECONNECT_DELAY; @@ -115,6 +119,7 @@ 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.failure.FailureType.CRITICAL_ERROR; import static org.apache.ignite.internal.events.DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT; import static org.apache.ignite.spi.discovery.tcp.ClientImpl.State.CONNECTED; import static org.apache.ignite.spi.discovery.tcp.ClientImpl.State.DISCONNECTED; @@ -207,7 +212,7 @@ class ClientImpl extends TcpDiscoveryImpl { b.append("Internal threads: ").append(U.nl()); - b.append(" Message worker: ").append(threadStatus(msgWorker)).append(U.nl()); + b.append(" Message worker: ").append(threadStatus(msgWorker.runner())).append(U.nl()); b.append(" Socket reader: ").append(threadStatus(sockReader)).append(U.nl()); b.append(" Socket writer: ").append(threadStatus(sockWriter)).append(U.nl()); @@ -264,8 +269,13 @@ class ClientImpl extends TcpDiscoveryImpl { if (spi.ipFinder.isShared()) registerLocalNodeAddress(); - msgWorker = new MessageWorker(); - msgWorker.start(); + msgWorker = new MessageWorker(log); + + new IgniteSpiThread(msgWorker.igniteInstanceName(), msgWorker.name(), log) { + @Override protected void body() { + msgWorker.run(); + } + }.start(); try { joinLatch.await(); @@ -289,7 +299,7 @@ class ClientImpl extends TcpDiscoveryImpl { /** {@inheritDoc} */ @Override public void spiStop() throws IgniteSpiException { - if (msgWorker != null && msgWorker.isAlive()) { // Should always be alive + if (msgWorker != null && !msgWorker.isDone()) { // Should always be alive msgWorker.addMessage(SPI_STOP); try { @@ -306,11 +316,15 @@ class ClientImpl extends TcpDiscoveryImpl { rmtNodes.clear(); - U.interrupt(msgWorker); + if (msgWorker != null) + U.interrupt(msgWorker.runner()); + U.interrupt(sockWriter); U.interrupt(sockReader); - U.join(msgWorker, log); + if (msgWorker != null) + U.join(msgWorker.runner(), log); + U.join(sockWriter, log); U.join(sockReader, log); @@ -405,11 +419,15 @@ else if (state == DISCONNECTED) { /** {@inheritDoc} */ @Override public void disconnect() throws IgniteSpiException { - U.interrupt(msgWorker); + if (msgWorker != null) + U.interrupt(msgWorker.runner()); + U.interrupt(sockWriter); U.interrupt(sockReader); - U.join(msgWorker, log); + if (msgWorker != null) + U.join(msgWorker.runner(), log); + U.join(sockWriter, log); U.join(sockReader, log); @@ -846,12 +864,14 @@ private NavigableSet allVisibleNodes() { U.warn(log, "Simulating client node failure: " + getLocalNodeId()); U.interrupt(sockWriter); - U.interrupt(msgWorker); + + if (msgWorker != null) + U.interrupt(msgWorker.runner()); U.join(sockWriter, log); - U.join( - msgWorker, - log); + + if (msgWorker != null) + U.join(msgWorker.runner(), log); } /** {@inheritDoc} */ @@ -879,7 +899,20 @@ private NavigableSet allVisibleNodes() { /** {@inheritDoc} */ @Override protected Collection threads() { - return Arrays.asList(sockWriter, msgWorker); + ArrayList res = new ArrayList<>(); + + res.add(sockWriter); + + MessageWorker msgWorker0 = msgWorker; + + if (msgWorker0 != null) { + Thread runner = msgWorker0.runner(); + + if (runner instanceof IgniteSpiThread) + res.add((IgniteSpiThread)runner); + } + + return res; } /** @@ -889,7 +922,7 @@ private NavigableSet allVisibleNodes() { public void waitForClientMessagePrecessed() { Object last = msgWorker.queue.peekLast(); - while (last != null && msgWorker.isAlive() && msgWorker.queue.contains(last)) { + while (last != null && !msgWorker.isDone() && msgWorker.queue.contains(last)) { try { Thread.sleep(10); } @@ -910,6 +943,13 @@ private void joinError(IgniteSpiException err) { joinLatch.countDown(); } + /** */ + private WorkersRegistry getWorkersRegistry() { + Ignite ignite = spi.ignite(); + + return ignite instanceof IgniteEx ? ((IgniteEx)ignite).context().workersRegistry() : null; + } + /** * Metrics sender. */ @@ -1530,7 +1570,7 @@ else if (spi.ensured(msg)) { /** * Message worker. */ - protected class MessageWorker extends IgniteSpiThread { + protected class MessageWorker extends GridWorker { /** Message queue. */ private final BlockingDeque queue = new LinkedBlockingDeque<>(); @@ -1544,10 +1584,10 @@ protected class MessageWorker extends IgniteSpiThread { private boolean nodeAdded; /** - * + * @param log Logger. */ - private MessageWorker() { - super(spi.ignite().name(), "tcp-client-disco-msg-worker", log); + private MessageWorker(IgniteLogger log) { + super(spi.ignite().name(), "tcp-client-disco-msg-worker", log, getWorkersRegistry()); } /** {@inheritDoc} */ @@ -1789,6 +1829,13 @@ else if (discoMsg instanceof TcpDiscoveryCheckFailedMessage) } } } + catch (InterruptedException ignored) { + Thread.currentThread().interrupt(); + } + catch (Throwable t) { + if (spi.ignite() instanceof IgniteEx) + ((IgniteEx)spi.ignite()).context().failure().process(new FailureContext(CRITICAL_ERROR, t)); + } finally { SocketStream currSock = this.currSock; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 079058b799287..170c1badf3d54 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -94,6 +94,8 @@ import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.util.worker.GridWorker; +import org.apache.ignite.internal.util.worker.GridWorkerListener; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgniteProductVersion; @@ -108,7 +110,6 @@ import org.apache.ignite.spi.IgniteSpiThread; import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; import org.apache.ignite.spi.discovery.DiscoverySpiListener; -import org.apache.ignite.spi.discovery.IgniteDiscoveryThread; import org.apache.ignite.spi.discovery.tcp.internal.DiscoveryDataPacket; import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNodesRing; @@ -305,7 +306,7 @@ class ServerImpl extends TcpDiscoveryImpl { /** {@inheritDoc} */ @Override public int boundPort() throws IgniteSpiException { if (tcpSrvr == null) - tcpSrvr = new TcpServer(); + tcpSrvr = new TcpServer(log); return tcpSrvr.port; } @@ -337,18 +338,19 @@ class ServerImpl extends TcpDiscoveryImpl { fromAddrs.clear(); noResAddrs.clear(); - msgWorker = new RingMessageWorker(); - msgWorker.start(); + msgWorker = new RingMessageWorker(log); + + new MessageWorkerThread(msgWorker, log).start(); if (tcpSrvr == null) - tcpSrvr = new TcpServer(); + tcpSrvr = new TcpServer(log); spi.initLocalNode(tcpSrvr.port, true); locNode = spi.locNode; // Start TCP server thread after local node is initialized. - tcpSrvr.start(); + new TcpServerThread(tcpSrvr, log).start(); ring.localNode(locNode); @@ -412,7 +414,7 @@ private void spiStop0(boolean disconnect) throws IgniteSpiException { } } - if (msgWorker != null && msgWorker.isAlive() && !disconnect) { + if (msgWorker != null && msgWorker.runner() != null && msgWorker.runner().isAlive() && !disconnect) { // Send node left message only if it is final stop. msgWorker.addMessage(new TcpDiscoveryNodeLeftMessage(locNode.id())); @@ -446,7 +448,7 @@ else if (log.isInfoEnabled()) { } } - U.interrupt(tcpSrvr); + U.cancel(tcpSrvr); U.join(tcpSrvr, log); tcpSrvr = null; @@ -463,12 +465,14 @@ else if (log.isInfoEnabled()) { U.interrupt(ipFinderCleaner); U.join(ipFinderCleaner, log); - U.interrupt(msgWorker); + U.cancel(msgWorker); U.join(msgWorker, log); for (ClientMessageWorker clientWorker : clientMsgWorkers.values()) { - U.interrupt(clientWorker); - U.join(clientWorker, log); + if (clientWorker != null) { + U.interrupt(clientWorker.runner()); + U.join(clientWorker.runner(), log); + } } clientMsgWorkers.clear(); @@ -1657,7 +1661,7 @@ private void clearNodeAddedMessage(TcpDiscoveryAbstractMessage msg) { @Override void simulateNodeFailure() { U.warn(log, "Simulating node failure: " + getLocalNodeId()); - U.interrupt(tcpSrvr); + U.cancel(tcpSrvr); U.join(tcpSrvr, log); U.interrupt(ipFinderCleaner); @@ -1672,13 +1676,14 @@ private void clearNodeAddedMessage(TcpDiscoveryAbstractMessage msg) { U.interrupt(tmp); U.joinThreads(tmp, log); - U.interrupt(msgWorker); + U.cancel(msgWorker); U.join(msgWorker, log); for (ClientMessageWorker msgWorker : clientMsgWorkers.values()) { - U.interrupt(msgWorker); - - U.join(msgWorker, log); + if (msgWorker != null) { + U.interrupt(msgWorker.runner()); + U.join(msgWorker.runner(), log); + } } U.interrupt(statsPrinter); @@ -1707,10 +1712,36 @@ private void clearNodeAddedMessage(TcpDiscoveryAbstractMessage msg) { threads.addAll(readers); } - threads.addAll(clientMsgWorkers.values()); - threads.add(tcpSrvr); + for (ClientMessageWorker wrk : clientMsgWorkers.values()) { + Thread t = wrk.runner(); + + assert t instanceof IgniteSpiThread; + + threads.add((IgniteSpiThread)t); + } + + TcpServer tcpSrvr0 = tcpSrvr; + + if (tcpSrvr0 != null) { + Thread tcpServerThread = tcpSrvr0.runner(); + + if (tcpServerThread != null) { + assert tcpServerThread instanceof IgniteSpiThread; + + threads.add((IgniteSpiThread)tcpServerThread); + } + } + threads.add(ipFinderCleaner); - threads.add(msgWorker); + + Thread msgWorkerThread = msgWorker.runner(); + + if (msgWorkerThread != null) { + assert msgWorkerThread instanceof IgniteSpiThread; + + threads.add((IgniteSpiThread)msgWorkerThread); + } + threads.add(statsPrinter); threads.removeAll(Collections.singleton(null)); @@ -1776,7 +1807,7 @@ TcpDiscoveryNodesRing ring() { b.append("Internal threads: ").append(U.nl()); - b.append(" Message worker: ").append(threadStatus(msgWorker)).append(U.nl()); + b.append(" Message worker: ").append(threadStatus(msgWorker.runner())).append(U.nl()); b.append(" IP finder cleaner: ").append(threadStatus(ipFinderCleaner)).append(U.nl()); b.append(" Stats printer: ").append(threadStatus(statsPrinter)).append(U.nl()); @@ -2534,9 +2565,9 @@ private void advance() { } /** - * Message worker thread for messages processing. + * Message worker for discovery messages processing. */ - private class RingMessageWorker extends MessageWorkerAdapter implements IgniteDiscoveryThread { + private class RingMessageWorker extends MessageWorker { /** Next node. */ @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized"}) private TcpDiscoveryNode next; @@ -2581,9 +2612,11 @@ private class RingMessageWorker extends MessageWorkerAdapter * Tcp server will call provided closure when accepts incoming connection. * From that moment server is no more responsible for the socket. */ - private class TcpServer extends IgniteSpiThread { + private class TcpServer extends GridWorker { /** Socket TCP server listens to. */ private ServerSocket srvrSock; @@ -5586,14 +5651,12 @@ private class TcpServer extends IgniteSpiThread { private int port; /** - * Constructor. - * + * @param log Logger. * @throws IgniteSpiException In case of error. */ - TcpServer() throws IgniteSpiException { - super(spi.ignite().name(), "tcp-disco-srvr", log); - - setPriority(spi.threadPri); + TcpServer(IgniteLogger log) throws IgniteSpiException { + super(spi.ignite().name(), "tcp-disco-srvr", log, + spi.ignite() instanceof IgniteEx ? ((IgniteEx)spi.ignite()).context().workersRegistry() : null); int lastPort = spi.locPortRange == 0 ? spi.locPort : spi.locPort + spi.locPortRange - 1; @@ -5640,7 +5703,7 @@ private class TcpServer extends IgniteSpiThread { Throwable err = null; try { - while (!isInterrupted()) { + while (!isCancelled()) { Socket sock = srvrSock.accept(); long tstamp = U.currentTimeMillis(); @@ -5670,7 +5733,7 @@ private class TcpServer extends IgniteSpiThread { onException("Failed to accept TCP connection.", e); - if (!isInterrupted()) { + if (!runner().isInterrupted()) { err = e; if (U.isMacInvalidArgumentError(e)) @@ -5685,24 +5748,24 @@ private class TcpServer extends IgniteSpiThread { throw t; } finally { - if (err == null && !spi.isNodeStopping0() && spiStateCopy() != DISCONNECTING) - err = new IllegalStateException("Thread " + getName() + " is terminated unexpectedly."); + if (spi.ignite() instanceof IgniteEx) { + if (err == null && !spi.isNodeStopping0() && spiStateCopy() != DISCONNECTING) + err = new IllegalStateException("Worker " + name() + " is terminated unexpectedly."); - FailureProcessor failure = ((IgniteEx)spi.ignite()).context().failure(); + FailureProcessor failure = ((IgniteEx)spi.ignite()).context().failure(); - if (err instanceof OutOfMemoryError) - failure.process(new FailureContext(CRITICAL_ERROR, err)); - else if (err != null) - failure.process(new FailureContext(SYSTEM_WORKER_TERMINATION, err)); + if (err instanceof OutOfMemoryError) + failure.process(new FailureContext(CRITICAL_ERROR, err)); + else if (err != null) + failure.process(new FailureContext(SYSTEM_WORKER_TERMINATION, err)); + } U.closeQuiet(srvrSock); } } - /** {@inheritDoc} */ - @Override public void interrupt() { - super.interrupt(); - + /** */ + public void onInterruption() { U.close(srvrSock, log); } } @@ -5874,7 +5937,7 @@ else if (log.isDebugEnabled()) } if (req.client()) { - ClientMessageWorker clientMsgWrk0 = new ClientMessageWorker(sock, nodeId); + ClientMessageWorker clientMsgWrk0 = new ClientMessageWorker(sock, nodeId, log); while (true) { ClientMessageWorker old = clientMsgWorkers.putIfAbsent(nodeId, clientMsgWrk0); @@ -5882,13 +5945,14 @@ else if (log.isDebugEnabled()) if (old == null) break; - if (old.isInterrupted()) { + if (old.isDone() || (old.runner() != null && old.runner().isInterrupted())) { clientMsgWorkers.remove(nodeId, old); continue; } - old.join(500); + if (old.runner() != null) + old.runner().join(500); old = clientMsgWorkers.putIfAbsent(nodeId, clientMsgWrk0); @@ -6020,8 +6084,8 @@ else if (msg instanceof TcpDiscoveryClientReconnectMessage) { if (state == CONNECTED) { spi.writeToSocket(msg, sock, RES_OK, sockTimeout); - if (clientMsgWrk != null && clientMsgWrk.getState() == State.NEW) - clientMsgWrk.start(); + if (clientMsgWrk != null && clientMsgWrk.runner() == null && !clientMsgWrk.isDone()) + new MessageWorkerThreadWithCleanup<>(clientMsgWrk, log).start(); processClientReconnectMessage((TcpDiscoveryClientReconnectMessage)msg); @@ -6262,7 +6326,7 @@ else if (msg instanceof TcpDiscoveryRingLatencyCheckMessage) { clientMsgWorkers.remove(nodeId, clientMsgWrk); - U.interrupt(clientMsgWrk); + U.interrupt(clientMsgWrk.runner()); } U.closeQuiet(sock); @@ -6408,10 +6472,10 @@ private boolean processJoinRequestMessage(TcpDiscoveryJoinRequestMessage msg, msg.responded(true); - if (clientMsgWrk != null && clientMsgWrk.getState() == State.NEW) { + if (clientMsgWrk != null && clientMsgWrk.runner() == null && !clientMsgWrk.isDone()) { clientMsgWrk.clientVersion(U.productVersion(msg.node())); - clientMsgWrk.start(); + new MessageWorkerThreadWithCleanup<>(clientMsgWrk, log).start(); } msgWorker.addMessage(msg); @@ -6508,9 +6572,8 @@ private class StatisticsPrinter extends IgniteSpiThread { } } - /** - */ - private class ClientMessageWorker extends MessageWorkerAdapter> { + /** */ + private class ClientMessageWorker extends MessageWorker> { /** Node ID. */ private final UUID clientNodeId; @@ -6529,9 +6592,10 @@ private class ClientMessageWorker extends MessageWorkerAdapter extends MessageWorkerThread { + /** */ + private final MessageWorker worker; + + /** {@inheritDoc} */ + private MessageWorkerThreadWithCleanup(MessageWorker worker, IgniteLogger log) { + super(worker, log); + + this.worker = worker; + } + /** {@inheritDoc} */ @Override protected void cleanup() { super.cleanup(); - pingResult(false); + worker.tearDown(); + } + } - U.closeQuiet(sock); + /** + * Slightly modified {@link IgniteSpiThread} intended to use with message workers. + */ + private class MessageWorkerThread extends IgniteSpiThread { + /** + * Backed interrupted flag, once set, it is not affected by further {@link Thread#interrupted()} calls. + */ + private volatile boolean interrupted; + + /** */ + private final GridWorker worker; + + /** {@inheritDoc} */ + private MessageWorkerThread(GridWorker worker, IgniteLogger log) { + super(worker.igniteInstanceName(), worker.name(), log); + + this.worker = worker; + + setPriority(spi.threadPri); + } + + /** {@inheritDoc} */ + @Override protected void body() throws InterruptedException { + worker.run(); + } + + /** {@inheritDoc} */ + @Override public void interrupt() { + interrupted = true; + + super.interrupt(); + } + + /** {@inheritDoc} */ + @Override public boolean isInterrupted() { + return interrupted || super.isInterrupted(); } } /** - * Base class for message workers. + * Superclass for all message workers. + * + * @param Message type. */ - protected abstract class MessageWorkerAdapter extends IgniteSpiThread { + private abstract class MessageWorker extends GridWorker { /** Message queue. */ protected final BlockingDeque queue = new LinkedBlockingDeque<>(); - /** Backed interrupted flag. */ - private volatile boolean interrupted; - /** Polling timeout. */ private final long pollingTimeout; /** - * @param name Thread name. + * @param name Worker name. + * @param log Logger. * @param pollingTimeout Messages polling timeout. + * @param lsnr Listener for life-cycle events. */ - protected MessageWorkerAdapter(String name, long pollingTimeout) { - super(spi.ignite().name(), name, log); + protected MessageWorker( + String name, + IgniteLogger log, + long pollingTimeout, + @Nullable GridWorkerListener lsnr + ) { + super(spi.ignite().name(), name, log, lsnr); this.pollingTimeout = pollingTimeout; - - setPriority(spi.threadPri); } /** {@inheritDoc} */ @@ -6769,7 +6894,7 @@ protected MessageWorkerAdapter(String name, long pollingTimeout) { if (log.isDebugEnabled()) log.debug("Message worker started [locNodeId=" + getConfiguredNodeId() + ']'); - while (!isInterrupted()) { + while (!isCancelled()) { T msg = queue.poll(pollingTimeout, TimeUnit.MILLISECONDS); if (msg == null) @@ -6779,26 +6904,16 @@ protected MessageWorkerAdapter(String name, long pollingTimeout) { } } - /** {@inheritDoc} */ - @Override public void interrupt() { - interrupted = true; - - super.interrupt(); - } - - /** {@inheritDoc} */ - @Override public boolean isInterrupted() { - return interrupted || super.isInterrupted(); - } - /** - * @return Current queue size. + * @return Current message queue size. */ int queueSize() { return queue.size(); } /** + * Processes succeeding message. + * * @param msg Message. */ protected abstract void processMessage(T msg); @@ -6809,6 +6924,13 @@ int queueSize() { protected void noMessageLoop() { // No-op. } + + /** + * Actions to be done before worker termination. + */ + protected void tearDown() { + // No-op. + } } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/IgniteDiagnosticMessagesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/IgniteDiagnosticMessagesTest.java index 572f3562778d0..11dceef77836a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/IgniteDiagnosticMessagesTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/IgniteDiagnosticMessagesTest.java @@ -305,6 +305,8 @@ public void testLongRunningTx() throws Exception { GridStringLogger strLog = this.strLog = new GridStringLogger(); + strLog.logLength(65536); + startGrid(1); awaitPartitionMapExchange(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/StripedExecutorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/StripedExecutorTest.java index 3fca7afed7ff5..9a4bf0619c4df 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/StripedExecutorTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/StripedExecutorTest.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.util; +import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.logger.java.JavaLogger; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -29,7 +30,10 @@ public class StripedExecutorTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override public void beforeTest() { - stripedExecSvc = new StripedExecutor(3, "foo name", "pool name", new JavaLogger(), (thread, t) -> {}); + stripedExecSvc = new StripedExecutor(3, "foo name", "pool name", new JavaLogger(), + new IgniteInClosure() { + @Override public void apply(Throwable throwable) {} + }, null); } /** {@inheritDoc} */ @@ -165,4 +169,4 @@ public TestRunnable(boolean infinitely) { private void sleepASec() throws InterruptedException { Thread.sleep(1000); } -} \ No newline at end of file +} diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java index d6d484ca5956c..d50a967aae172 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java @@ -220,10 +220,15 @@ else if (igniteInstanceName.contains("testNoRingMessageWorkerAbnormalFailureSegm cfg.setFailureDetectionTimeout(6_000); cfg.setGridLogger(strLog = new GridStringLogger()); + + strLog.logLength(300_000); } - else if (igniteInstanceName.contains("testNodeShutdownOnRingMessageWorkerFailureFailedNode")) + else if (igniteInstanceName.contains("testNodeShutdownOnRingMessageWorkerFailureFailedNode")) { cfg.setGridLogger(strLog = new GridStringLogger()); + strLog.logLength(300_000); + } + cfg.setClientMode(client); return cfg; diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java index 5b853856bcde5..d1de34797cc70 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java @@ -79,6 +79,7 @@ public GridTestKernalContext(IgniteLogger log, IgniteConfiguration cfg) { null, null, U.allPluginProviders(), + null, null ); From f071b67bfebd2a7a1ac11f890cc009560fb66321 Mon Sep 17 00:00:00 2001 From: Dmitriy Sorokin Date: Mon, 18 Jun 2018 13:48:11 +0300 Subject: [PATCH 0272/1463] IGNITE-8749 Exception for no space left situation should be propagated to FailureHandler. Signed-off-by: agura --- .../wal/FileWriteAheadLogManager.java | 167 ++++++------ .../FsyncModeFileWriteAheadLogManager.java | 215 ++++++++------- .../ignite/failure/TestFailureHandler.java | 19 ++ .../wal/IgniteWalFormatFileFailoverTest.java | 258 ++++++++++++++++++ .../testsuites/IgnitePdsTestSuite2.java | 3 + 5 files changed, 483 insertions(+), 179 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFormatFileFailoverTest.java 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 9b39987edf42a..09a08c9d4bce5 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 @@ -605,48 +605,43 @@ private void checkWalConfiguration() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public void resumeLogging(WALPointer lastPtr) throws IgniteCheckedException { - try { - assert currHnd == null; - assert lastPtr == null || lastPtr instanceof FileWALPointer; + assert currHnd == null; + assert lastPtr == null || lastPtr instanceof FileWALPointer; - FileWALPointer filePtr = (FileWALPointer)lastPtr; + FileWALPointer filePtr = (FileWALPointer)lastPtr; walWriter = new WALWriter(log); if (!mmap) new IgniteThread(walWriter).start(); - currHnd = restoreWriteHandle(filePtr); + currHnd = restoreWriteHandle(filePtr); - // For new handle write serializer version to it. - if (filePtr == null) - currHnd.writeHeader(); + // For new handle write serializer version to it. + if (filePtr == null) + currHnd.writeHeader(); - if (currHnd.serializer.version() != serializer.version()) { - if (log.isInfoEnabled()) - log.info("Record serializer version change detected, will start logging with a new WAL record " + - "serializer to a new WAL segment [curFile=" + currHnd + ", newVer=" + serializer.version() + - ", oldVer=" + currHnd.serializer.version() + ']'); + if (currHnd.serializer.version() != serializer.version()) { + if (log.isInfoEnabled()) + log.info("Record serializer version change detected, will start logging with a new WAL record " + + "serializer to a new WAL segment [curFile=" + currHnd + ", newVer=" + serializer.version() + + ", oldVer=" + currHnd.serializer.version() + ']'); - rollOver(currHnd); - } + rollOver(currHnd); + } - currHnd.resume = false; + currHnd.resume = false; - if (mode == WALMode.BACKGROUND) { - backgroundFlushSchedule = cctx.time().schedule(new Runnable() { - @Override public void run() { - doFlush(); - } - }, flushFreq, flushFreq); - } - - if (walAutoArchiveAfterInactivity > 0) - scheduleNextInactivityPeriodElapsedCheck(); - } - catch (StorageException e) { - throw new IgniteCheckedException(e); + if (mode == WALMode.BACKGROUND) { + backgroundFlushSchedule = cctx.time().schedule(new Runnable() { + @Override public void run() { + doFlush(); + } + }, flushFreq, flushFreq); } + + if (walAutoArchiveAfterInactivity > 0) + scheduleNextInactivityPeriodElapsedCheck(); } /** @@ -1131,9 +1126,9 @@ private FileWriteHandle rollOver(FileWriteHandle cur) throws StorageException, I /** * @param lastReadPtr Last read WAL file pointer. * @return Initialized file write handle. - * @throws IgniteCheckedException If failed to initialize WAL write handle. + * @throws StorageException If failed to initialize WAL write handle. */ - private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws IgniteCheckedException { + private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws StorageException { long absIdx = lastReadPtr == null ? 0 : lastReadPtr.index(); @Nullable FileArchiver archiver0 = archiver; @@ -1175,14 +1170,9 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws Ig SegmentedRingByteBuffer rbuf; if (mmap) { - try { - MappedByteBuffer buf = fileIO.map((int)maxWalSegmentSize); + MappedByteBuffer buf = fileIO.map((int)maxWalSegmentSize); - rbuf = new SegmentedRingByteBuffer(buf, metrics); - } - catch (IOException e) { - throw new IgniteCheckedException(e); - } + rbuf = new SegmentedRingByteBuffer(buf, metrics); } else rbuf = new SegmentedRingByteBuffer(dsCfg.getWalBufferSize(), maxWalSegmentSize, DIRECT, metrics); @@ -1206,13 +1196,21 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws Ig return hnd; } catch (IgniteCheckedException | IOException e) { - fileIO.close(); + try { + fileIO.close(); + } + catch (IOException suppressed) { + e.addSuppressed(suppressed); + } - throw e; + if (e instanceof StorageException) + throw (StorageException) e; + + throw e instanceof IOException ? (IOException) e : new IOException(e); } } catch (IOException e) { - throw new IgniteCheckedException("Failed to restore WAL write handle: " + curFile.getAbsolutePath(), e); + throw new StorageException("Failed to restore WAL write handle: " + curFile.getAbsolutePath(), e); } } @@ -1223,9 +1221,8 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws Ig * @param cur Current file write handle released by WAL writer * @return Initialized file handle. * @throws StorageException If IO exception occurred. - * @throws IgniteCheckedException If failed. */ - private FileWriteHandle initNextWriteHandle(FileWriteHandle cur) throws StorageException, IgniteCheckedException { + private FileWriteHandle initNextWriteHandle(FileWriteHandle cur) throws StorageException { try { File nextFile = pollNextFile(cur.idx); @@ -1310,8 +1307,10 @@ private FileWriteHandle initNextWriteHandle(FileWriteHandle cur) throws StorageE /** * Deletes temp files, creates and prepares new; Creates first segment if necessary + * + * @throws StorageException If failed. */ - private void checkOrPrepareFiles() throws IgniteCheckedException { + private void checkOrPrepareFiles() throws StorageException { // Clean temp files. { File[] tmpFiles = walWorkDir.listFiles(WAL_SEGMENT_TEMP_FILE_FILTER); @@ -1321,7 +1320,7 @@ private void checkOrPrepareFiles() throws IgniteCheckedException { boolean deleted = tmp.delete(); if (!deleted) - throw new IgniteCheckedException("Failed to delete previously created temp file " + + throw new StorageException("Failed to delete previously created temp file " + "(make sure Ignite process has enough rights): " + tmp.getAbsolutePath()); } } @@ -1331,7 +1330,7 @@ private void checkOrPrepareFiles() throws IgniteCheckedException { if(isArchiverEnabled()) if (allFiles.length != 0 && allFiles.length > dsCfg.getWalSegments()) - throw new IgniteCheckedException("Failed to initialize wal (work directory contains " + + throw new StorageException("Failed to initialize wal (work directory contains " + "incorrect number of segments) [cur=" + allFiles.length + ", expected=" + dsCfg.getWalSegments() + ']'); // Allocate the first segment synchronously. All other segments will be allocated by archiver in background. @@ -1371,9 +1370,9 @@ public void cleanupWalDirectories() throws IgniteCheckedException { * Clears whole the file, fills with zeros for Default mode. * * @param file File to format. - * @throws IgniteCheckedException if formatting failed + * @throws StorageException if formatting failed */ - private void formatFile(File file) throws IgniteCheckedException { + private void formatFile(File file) throws StorageException { formatFile(file, dsCfg.getWalSegmentSize()); } @@ -1382,9 +1381,9 @@ private void formatFile(File file) throws IgniteCheckedException { * * @param file File to format. * @param bytesCntToFormat Count of first bytes to format. - * @throws IgniteCheckedException if formatting failed + * @throws StorageException if formatting failed */ - private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedException { + private void formatFile(File file, int bytesCntToFormat) throws StorageException { if (log.isDebugEnabled()) log.debug("Formatting file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']'); @@ -1396,7 +1395,7 @@ private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedExc int toWrite = Math.min(FILL_BUF.length, left); if (fileIO.write(FILL_BUF, 0, toWrite) < toWrite) { - final IgniteCheckedException ex = new IgniteCheckedException("Failed to extend WAL segment file: " + + final StorageException ex = new StorageException("Failed to extend WAL segment file: " + file.getName() + ". Probably disk is too busy, please check your device."); if (failureProcessor != null) @@ -1414,7 +1413,7 @@ private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedExc fileIO.clear(); } catch (IOException e) { - throw new IgniteCheckedException("Failed to format WAL segment file: " + file.getAbsolutePath(), e); + throw new StorageException("Failed to format WAL segment file: " + file.getAbsolutePath(), e); } } @@ -1422,9 +1421,9 @@ private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedExc * Creates a file atomically with temp file. * * @param file File to create. - * @throws IgniteCheckedException If failed. + * @throws StorageException If failed. */ - private void createFile(File file) throws IgniteCheckedException { + private void createFile(File file) throws StorageException { if (log.isDebugEnabled()) log.debug("Creating new file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']'); @@ -1436,7 +1435,7 @@ private void createFile(File file) throws IgniteCheckedException { Files.move(tmp.toPath(), file.toPath()); } catch (IOException e) { - throw new IgniteCheckedException("Failed to move temp file to a regular WAL segment file: " + + throw new StorageException("Failed to move temp file to a regular WAL segment file: " + file.getAbsolutePath(), e); } @@ -1449,9 +1448,9 @@ private void createFile(File file) throws IgniteCheckedException { * * @param curIdx Current absolute WAL segment index. * @return File ready for use as new WAL segment. - * @throws IgniteCheckedException If failed. + * @throws StorageException If exception occurred in the archiver thread. */ - private File pollNextFile(long curIdx) throws IgniteCheckedException { + private File pollNextFile(long curIdx) throws StorageException { FileArchiver archiver0 = archiver; if (archiver0 == null) { @@ -1527,7 +1526,7 @@ public long maxWalSegmentSize() { */ private class FileArchiver extends GridWorker { /** Exception which occurred during initial creation of files or during archiving WAL segment */ - private IgniteCheckedException cleanErr; + private StorageException cleanErr; /** * Absolute current segment index WAL Manager writes to. Guarded by this. Incremented during @@ -1599,15 +1598,17 @@ private synchronized boolean locked(long absIdx) { try { allocateRemainingFiles(); } - catch (IgniteCheckedException e) { + catch (StorageException e) { synchronized (this) { // Stop the thread and report to starter. cleanErr = e; notifyAll(); - - return; } + + cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, e)); + + return; } Throwable err = null; @@ -1691,9 +1692,9 @@ private void changeLastArchivedIndexAndNotifyWaiters(long idx) { * * @param curIdx Current absolute index that we want to increment. * @return Next index (curWalSegmIdx+1) when it is ready to be written. - * @throws IgniteCheckedException If failed (if interrupted or if exception occurred in the archiver thread). + * @throws StorageException If exception occurred in the archiver thread. */ - private long nextAbsoluteSegmentIndex(long curIdx) throws IgniteCheckedException { + private long nextAbsoluteSegmentIndex(long curIdx) throws StorageException { synchronized (this) { if (cleanErr != null) throw cleanErr; @@ -1708,6 +1709,9 @@ private long nextAbsoluteSegmentIndex(long curIdx) throws IgniteCheckedException while (curAbsWalIdx - lastAbsArchivedIdx > dsCfg.getWalSegments() && cleanErr == null) { try { wait(); + + if (cleanErr != null) + throw cleanErr; } catch (InterruptedException ignore) { interrupted.set(true); @@ -1715,9 +1719,12 @@ private long nextAbsoluteSegmentIndex(long curIdx) throws IgniteCheckedException } // Wait for formatter so that we do not open an empty file in DEFAULT mode. - while (curAbsWalIdx % dsCfg.getWalSegments() > formatted) + while (curAbsWalIdx % dsCfg.getWalSegments() > formatted && cleanErr == null) try { wait(); + + if (cleanErr != null) + throw cleanErr; } catch (InterruptedException ignore) { interrupted.set(true); @@ -1789,7 +1796,7 @@ private void releaseWorkSegment(long absIdx) { * * @param absIdx Absolute index to archive. */ - private SegmentArchiveResult archiveSegment(long absIdx) throws IgniteCheckedException { + private SegmentArchiveResult archiveSegment(long absIdx) throws StorageException { long segIdx = absIdx % dsCfg.getWalSegments(); File origFile = new File(walWorkDir, FileDescriptor.fileName(segIdx)); @@ -1818,7 +1825,7 @@ private SegmentArchiveResult archiveSegment(long absIdx) throws IgniteCheckedExc } } catch (IOException e) { - throw new IgniteCheckedException("Failed to archive WAL segment [" + + throw new StorageException("Failed to archive WAL segment [" + "srcFile=" + origFile.getAbsolutePath() + ", dstFile=" + dstTmpFile.getAbsolutePath() + ']', e); } @@ -1841,7 +1848,7 @@ private boolean checkStop() { * Background creation of all segments except first. First segment was created in main thread by {@link * FileWriteAheadLogManager#checkOrPrepareFiles()} */ - private void allocateRemainingFiles() throws IgniteCheckedException { + private void allocateRemainingFiles() throws StorageException { checkFiles( 1, true, @@ -2235,23 +2242,23 @@ private void shutdown() throws IgniteInterruptedCheckedException { * @param startWith Start with. * @param create Flag create file. * @param p Predicate Exit condition. - * @throws IgniteCheckedException if validation or create file fail. + * @throws StorageException if validation or create file fail. */ private void checkFiles( int startWith, boolean create, @Nullable IgnitePredicate p, @Nullable IgniteInClosure completionCallback - ) throws IgniteCheckedException { + ) throws StorageException { for (int i = startWith; i < dsCfg.getWalSegments() && (p == null || p.apply(i)); i++) { File checkFile = new File(walWorkDir, FileDescriptor.fileName(i)); if (checkFile.exists()) { if (checkFile.isDirectory()) - throw new IgniteCheckedException("Failed to initialize WAL log segment (a directory with " + + throw new StorageException("Failed to initialize WAL log segment (a directory with " + "the same name already exists): " + checkFile.getAbsolutePath()); else if (checkFile.length() != dsCfg.getWalSegmentSize() && mode == WALMode.FSYNC) - throw new IgniteCheckedException("Failed to initialize WAL log segment " + + throw new StorageException("Failed to initialize WAL log segment " + "(WAL segment size change is not supported in 'DEFAULT' WAL mode) " + "[filePath=" + checkFile.getAbsolutePath() + ", fileSize=" + checkFile.length() + @@ -2651,9 +2658,8 @@ public void writeHeader() { * Flush or wait for concurrent flush completion. * * @param ptr Pointer. - * @throws IgniteCheckedException If failed. */ - private void flushOrWait(FileWALPointer ptr) throws IgniteCheckedException { + private void flushOrWait(FileWALPointer ptr) { if (ptr != null) { // If requested obsolete file index, it must be already flushed by close. if (ptr.index() != idx) @@ -2665,10 +2671,8 @@ private void flushOrWait(FileWALPointer ptr) throws IgniteCheckedException { /** * @param ptr Pointer. - * @throws IgniteCheckedException If failed. - * @throws StorageException If failed. */ - private void flush(FileWALPointer ptr) throws IgniteCheckedException, StorageException { + private void flush(FileWALPointer ptr) { if (ptr == null) { // Unconditional flush. walWriter.flushAll(); @@ -2884,7 +2888,7 @@ private boolean close(boolean rollOver) throws IgniteCheckedException, StorageEx } } catch (IOException e) { - throw new IgniteCheckedException(e); + throw new StorageException(e); } if (log.isDebugEnabled()) @@ -3365,28 +3369,29 @@ private void unparkWaiters(long pos) { /** * Forces all made changes to the file. */ - void force() throws IgniteCheckedException { + void force() { flushBuffer(FILE_FORCE); } /** * Closes file. */ - void close() throws IgniteCheckedException { + void close() { flushBuffer(FILE_CLOSE); } /** * Flushes all data from the buffer. */ - void flushAll() throws IgniteCheckedException { + void flushAll() { flushBuffer(UNCONDITIONAL_FLUSH); } /** * @param expPos Expected position. */ - void flushBuffer(long expPos) throws StorageException, IgniteCheckedException { + @SuppressWarnings("ForLoopReplaceableByForEach") + void flushBuffer(long expPos) { if (mmap) return; 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 49fbc73b9a48c..6f676fcf43284 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 @@ -487,37 +487,32 @@ private void checkWalConfiguration() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public void resumeLogging(WALPointer lastPtr) throws IgniteCheckedException { - try { - assert currentHnd == null; - assert lastPtr == null || lastPtr instanceof FileWALPointer; - - FileWALPointer filePtr = (FileWALPointer)lastPtr; + assert currentHnd == null; + assert lastPtr == null || lastPtr instanceof FileWALPointer; - currentHnd = restoreWriteHandle(filePtr); + FileWALPointer filePtr = (FileWALPointer)lastPtr; - if (currentHnd.serializer.version() != serializer.version()) { - if (log.isInfoEnabled()) - log.info("Record serializer version change detected, will start logging with a new WAL record " + - "serializer to a new WAL segment [curFile=" + currentHnd + ", newVer=" + serializer.version() + - ", oldVer=" + currentHnd.serializer.version() + ']'); + currentHnd = restoreWriteHandle(filePtr); - rollOver(currentHnd); - } + if (currentHnd.serializer.version() != serializer.version()) { + if (log.isInfoEnabled()) + log.info("Record serializer version change detected, will start logging with a new WAL record " + + "serializer to a new WAL segment [curFile=" + currentHnd + ", newVer=" + serializer.version() + + ", oldVer=" + currentHnd.serializer.version() + ']'); - if (mode == WALMode.BACKGROUND) { - backgroundFlushSchedule = cctx.time().schedule(new Runnable() { - @Override public void run() { - doFlush(); - } - }, flushFreq, flushFreq); - } - - if (walAutoArchiveAfterInactivity > 0) - scheduleNextInactivityPeriodElapsedCheck(); + rollOver(currentHnd); } - catch (StorageException e) { - throw new IgniteCheckedException(e); + + if (mode == WALMode.BACKGROUND) { + backgroundFlushSchedule = cctx.time().schedule(new Runnable() { + @Override public void run() { + doFlush(); + } + }, flushFreq, flushFreq); } + + if (walAutoArchiveAfterInactivity > 0) + scheduleNextInactivityPeriodElapsedCheck(); } /** @@ -1019,7 +1014,7 @@ private FileWriteHandle currentHandle() { * @param cur Handle that failed to fit the given entry. * @return Handle that will fit the entry. */ - private FileWriteHandle rollOver(FileWriteHandle cur) throws StorageException, IgniteCheckedException { + private FileWriteHandle rollOver(FileWriteHandle cur) throws StorageException, IgniteInterruptedCheckedException { FileWriteHandle hnd = currentHandle(); if (hnd != cur) @@ -1050,9 +1045,9 @@ private FileWriteHandle rollOver(FileWriteHandle cur) throws StorageException, I /** * @param lastReadPtr Last read WAL file pointer. * @return Initialized file write handle. - * @throws IgniteCheckedException If failed to initialize WAL write handle. + * @throws StorageException If failed to initialize WAL write handle. */ - private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws IgniteCheckedException { + private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws StorageException { long absIdx = lastReadPtr == null ? 0 : lastReadPtr.index(); long segNo = absIdx % dsCfg.getWalSegments(); @@ -1100,13 +1095,21 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws Ig return hnd; } catch (IgniteCheckedException | IOException e) { - fileIO.close(); + try { + fileIO.close(); + } + catch (IOException suppressed) { + e.addSuppressed(suppressed); + } - throw e; + if (e instanceof StorageException) + throw (StorageException) e; + + throw e instanceof IOException ? (IOException) e : new IOException(e); } } catch (IOException e) { - throw new IgniteCheckedException("Failed to restore WAL write handle: " + curFile.getAbsolutePath(), e); + throw new StorageException("Failed to restore WAL write handle: " + curFile.getAbsolutePath(), e); } } @@ -1118,9 +1121,9 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws Ig * @param curIdx current absolute segment released by WAL writer * @return Initialized file handle. * @throws StorageException If IO exception occurred. - * @throws IgniteCheckedException If failed. + * @throws IgniteInterruptedCheckedException If interrupted. */ - private FileWriteHandle initNextWriteHandle(long curIdx) throws StorageException, IgniteCheckedException { + private FileWriteHandle initNextWriteHandle(long curIdx) throws StorageException, IgniteInterruptedCheckedException { try { File nextFile = pollNextFile(curIdx); @@ -1150,9 +1153,11 @@ private FileWriteHandle initNextWriteHandle(long curIdx) throws StorageException } /** - * Deletes temp files, creates and prepares new; Creates first segment if necessary + * Deletes temp files, creates and prepares new; Creates first segment if necessary. + * + * @throws StorageException If failed. */ - private void checkOrPrepareFiles() throws IgniteCheckedException { + private void checkOrPrepareFiles() throws StorageException { // Clean temp files. { File[] tmpFiles = walWorkDir.listFiles(WAL_SEGMENT_TEMP_FILE_FILTER); @@ -1162,7 +1167,7 @@ private void checkOrPrepareFiles() throws IgniteCheckedException { boolean deleted = tmp.delete(); if (!deleted) - throw new IgniteCheckedException("Failed to delete previously created temp file " + + throw new StorageException("Failed to delete previously created temp file " + "(make sure Ignite process has enough rights): " + tmp.getAbsolutePath()); } } @@ -1171,7 +1176,7 @@ private void checkOrPrepareFiles() throws IgniteCheckedException { File[] allFiles = walWorkDir.listFiles(WAL_SEGMENT_FILE_FILTER); if (allFiles.length != 0 && allFiles.length > dsCfg.getWalSegments()) - throw new IgniteCheckedException("Failed to initialize wal (work directory contains " + + throw new StorageException("Failed to initialize wal (work directory contains " + "incorrect number of segments) [cur=" + allFiles.length + ", expected=" + dsCfg.getWalSegments() + ']'); // Allocate the first segment synchronously. All other segments will be allocated by archiver in background. @@ -1188,9 +1193,9 @@ private void checkOrPrepareFiles() throws IgniteCheckedException { * Clears whole the file, fills with zeros for Default mode. * * @param file File to format. - * @throws IgniteCheckedException if formatting failed + * @throws StorageException if formatting failed. */ - private void formatFile(File file) throws IgniteCheckedException { + private void formatFile(File file) throws StorageException { formatFile(file, dsCfg.getWalSegmentSize()); } @@ -1199,9 +1204,9 @@ private void formatFile(File file) throws IgniteCheckedException { * * @param file File to format. * @param bytesCntToFormat Count of first bytes to format. - * @throws IgniteCheckedException if formatting failed + * @throws StorageException If formatting failed. */ - private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedException { + private void formatFile(File file, int bytesCntToFormat) throws StorageException { if (log.isDebugEnabled()) log.debug("Formatting file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']'); @@ -1223,7 +1228,7 @@ private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedExc fileIO.clear(); } catch (IOException e) { - throw new IgniteCheckedException("Failed to format WAL segment file: " + file.getAbsolutePath(), e); + throw new StorageException("Failed to format WAL segment file: " + file.getAbsolutePath(), e); } } @@ -1231,9 +1236,9 @@ private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedExc * Creates a file atomically with temp file. * * @param file File to create. - * @throws IgniteCheckedException If failed. + * @throws StorageException If failed. */ - private void createFile(File file) throws IgniteCheckedException { + private void createFile(File file) throws StorageException { if (log.isDebugEnabled()) log.debug("Creating new file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']'); @@ -1245,7 +1250,7 @@ private void createFile(File file) throws IgniteCheckedException { Files.move(tmp.toPath(), file.toPath()); } catch (IOException e) { - throw new IgniteCheckedException("Failed to move temp file to a regular WAL segment file: " + + throw new StorageException("Failed to move temp file to a regular WAL segment file: " + file.getAbsolutePath(), e); } @@ -1259,9 +1264,10 @@ private void createFile(File file) throws IgniteCheckedException { * * @param curIdx Current absolute WAL segment index. * @return File ready for use as new WAL segment. - * @throws IgniteCheckedException If failed. + * @throws StorageException If exception occurred in the archiver thread. + * @throws IgniteInterruptedCheckedException If interrupted. */ - private File pollNextFile(long curIdx) throws IgniteCheckedException { + private File pollNextFile(long curIdx) throws StorageException, IgniteInterruptedCheckedException { // Signal to archiver that we are done with the segment and it can be archived. long absNextIdx = archiver.nextAbsoluteSegmentIndex(curIdx); @@ -1318,7 +1324,7 @@ private void checkNode() throws StorageException { */ private class FileArchiver extends GridWorker { /** Exception which occurred during initial creation of files or during archiving WAL segment */ - private IgniteCheckedException cleanException; + private StorageException cleanException; /** * Absolute current segment index WAL Manager writes to. Guarded by this. @@ -1426,15 +1432,17 @@ private synchronized void release(long absIdx) { try { allocateRemainingFiles(); } - catch (IgniteCheckedException e) { + catch (StorageException e) { synchronized (this) { // Stop the thread and report to starter. cleanException = e; notifyAll(); - - return; } + + cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, e)); + + return; } Throwable err = null; @@ -1515,9 +1523,10 @@ private void changeLastArchivedIndexAndWakeupCompressor(long idx) { * * @param curIdx Current absolute index that we want to increment. * @return Next index (curWalSegmIdx+1) when it is ready to be written. - * @throws IgniteCheckedException If failed (if interrupted or if exception occurred in the archiver thread). + * @throws StorageException If exception occurred in the archiver thread. + * @throws IgniteInterruptedCheckedException If interrupted. */ - private long nextAbsoluteSegmentIndex(long curIdx) throws IgniteCheckedException { + private long nextAbsoluteSegmentIndex(long curIdx) throws StorageException, IgniteInterruptedCheckedException { try { synchronized (this) { if (cleanException != null) @@ -1535,10 +1544,16 @@ private long nextAbsoluteSegmentIndex(long curIdx) throws IgniteCheckedException while ((curAbsWalIdx - lastAbsArchivedIdx > segments && cleanException == null)) wait(); + if (cleanException != null) + throw cleanException; + // Wait for formatter so that we do not open an empty file in DEFAULT mode. - while (curAbsWalIdx % dsCfg.getWalSegments() > formatted) + while (curAbsWalIdx % dsCfg.getWalSegments() > formatted && cleanException == null) wait(); + if (cleanException != null) + throw cleanException; + return curAbsWalIdx; } } @@ -1664,7 +1679,7 @@ private boolean checkStop() { * Background creation of all segments except first. First segment was created in main thread by * {@link FsyncModeFileWriteAheadLogManager#checkOrPrepareFiles()} */ - private void allocateRemainingFiles() throws IgniteCheckedException { + private void allocateRemainingFiles() throws StorageException { final FileArchiver archiver = this; checkFiles(1, @@ -2029,23 +2044,23 @@ private void shutdown() { * @param startWith Start with. * @param create Flag create file. * @param p Predicate Exit condition. - * @throws IgniteCheckedException if validation or create file fail. + * @throws StorageException if validation or create file fail. */ private void checkFiles( int startWith, boolean create, @Nullable IgnitePredicate p, @Nullable IgniteInClosure completionCallback - ) throws IgniteCheckedException { + ) throws StorageException { for (int i = startWith; i < dsCfg.getWalSegments() && (p == null || (p != null && p.apply(i))); i++) { File checkFile = new File(walWorkDir, FileDescriptor.fileName(i)); if (checkFile.exists()) { if (checkFile.isDirectory()) - throw new IgniteCheckedException("Failed to initialize WAL log segment (a directory with " + + throw new StorageException("Failed to initialize WAL log segment (a directory with " + "the same name already exists): " + checkFile.getAbsolutePath()); else if (checkFile.length() != dsCfg.getWalSegmentSize() && mode == WALMode.FSYNC) - throw new IgniteCheckedException("Failed to initialize WAL log segment " + + throw new StorageException("Failed to initialize WAL log segment " + "(WAL segment size change is not supported):" + checkFile.getAbsolutePath()); } else if (create) @@ -2408,9 +2423,9 @@ private FileWriteHandle( * Write serializer version to current handle. * NOTE: Method mutates {@code fileIO} position, written and lastFsyncPos fields. * - * @throws IOException If fail to write serializer version. + * @throws StorageException If fail to write serializer version. */ - public void writeSerializerVersion() throws IOException { + private void writeSerializerVersion() throws StorageException { try { assert fileIO.position() == 0 : "Serializer version can be written only at the begin of file " + fileIO.position(); @@ -2423,7 +2438,7 @@ public void writeSerializerVersion() throws IOException { head.set(new FakeRecord(new FileWALPointer(idx, (int)updatedPosition, 0), false)); } catch (IOException e) { - throw new IOException("Unable to write serializer version for segment " + idx, e); + throw new StorageException("Unable to write serializer version for segment " + idx, e); } } @@ -2448,9 +2463,8 @@ private boolean stopped(WALRecord record) { * @param rec Record to be added to record chain as new {@link #head} * @return Pointer or null if roll over to next segment is required or already started by other thread. * @throws StorageException If failed. - * @throws IgniteCheckedException If failed. */ - @Nullable private WALPointer addRecord(WALRecord rec) throws StorageException, IgniteCheckedException { + @Nullable private WALPointer addRecord(WALRecord rec) throws StorageException { assert rec.size() > 0 || rec.getClass() == FakeRecord.class; boolean flushed = false; @@ -2503,9 +2517,9 @@ private long nextPosition(WALRecord rec) { * Flush or wait for concurrent flush completion. * * @param ptr Pointer. - * @throws IgniteCheckedException If failed. + * @throws StorageException If failed. */ - private void flushOrWait(FileWALPointer ptr, boolean stop) throws IgniteCheckedException { + private void flushOrWait(FileWALPointer ptr, boolean stop) throws StorageException { long expWritten; if (ptr != null) { @@ -2549,10 +2563,9 @@ else if (stop) { /** * @param ptr Pointer. * @return {@code true} If the flush really happened. - * @throws IgniteCheckedException If failed. * @throws StorageException If failed. */ - private boolean flush(FileWALPointer ptr, boolean stop) throws IgniteCheckedException, StorageException { + private boolean flush(FileWALPointer ptr, boolean stop) throws StorageException { if (ptr == null) { // Unconditional flush. for (; ; ) { WALRecord expHead = head.get(); @@ -2594,10 +2607,9 @@ private long chainBeginPosition(WALRecord h) { /** * @param expHead Expected head of chain. If head was changed, flush is not performed in this thread - * @throws IgniteCheckedException If failed. * @throws StorageException If failed. */ - private boolean flush(WALRecord expHead, boolean stop) throws StorageException, IgniteCheckedException { + private boolean flush(WALRecord expHead, boolean stop) throws StorageException { if (expHead.previous() == null) { FakeRecord frHead = (FakeRecord)expHead; @@ -2643,7 +2655,8 @@ private boolean flush(WALRecord expHead, boolean stop) throws StorageException, return true; } catch (Throwable e) { - StorageException se = new StorageException("Unable to write", new IOException(e)); + StorageException se = e instanceof StorageException ? (StorageException) e : + new StorageException("Unable to write", new IOException(e)); cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, se)); @@ -2725,8 +2738,9 @@ private FileWALPointer position() { /** * @param ptr Pointer to sync. * @throws StorageException If failed. + * @throws IgniteInterruptedCheckedException If interrupted. */ - private void fsync(FileWALPointer ptr, boolean stop) throws StorageException, IgniteCheckedException { + private void fsync(FileWALPointer ptr, boolean stop) throws StorageException, IgniteInterruptedCheckedException { lock.lock(); try { @@ -2780,10 +2794,9 @@ private void fsync(FileWALPointer ptr, boolean stop) throws StorageException, Ig /** * @return {@code true} If this thread actually closed the segment. - * @throws IgniteCheckedException If failed. * @throws StorageException If failed. */ - private boolean close(boolean rollOver) throws IgniteCheckedException, StorageException { + private boolean close(boolean rollOver) throws StorageException { if (stop.compareAndSet(false, true)) { lock.lock(); @@ -2793,43 +2806,49 @@ private boolean close(boolean rollOver) throws IgniteCheckedException, StorageEx assert stopped() : "Segment is not closed after close flush: " + head.get(); try { - RecordSerializer backwardSerializer = new RecordSerializerFactoryImpl(cctx) - .createSerializer(serializerVersion); + try { + RecordSerializer backwardSerializer = new RecordSerializerFactoryImpl(cctx) + .createSerializer(serializerVersion); - SwitchSegmentRecord segmentRecord = new SwitchSegmentRecord(); + SwitchSegmentRecord segmentRecord = new SwitchSegmentRecord(); - int switchSegmentRecSize = backwardSerializer.size(segmentRecord); + int switchSegmentRecSize = backwardSerializer.size(segmentRecord); - if (rollOver && written < (maxSegmentSize - switchSegmentRecSize)) { - final ByteBuffer buf = ByteBuffer.allocate(switchSegmentRecSize); + if (rollOver && written < (maxSegmentSize - switchSegmentRecSize)) { + final ByteBuffer buf = ByteBuffer.allocate(switchSegmentRecSize); - segmentRecord.position(new FileWALPointer(idx, (int)written, switchSegmentRecSize)); - backwardSerializer.writeRecord(segmentRecord, buf); + segmentRecord.position(new FileWALPointer(idx, (int)written, switchSegmentRecSize)); + backwardSerializer.writeRecord(segmentRecord, buf); - buf.rewind(); + buf.rewind(); - int rem = buf.remaining(); + int rem = buf.remaining(); - while (rem > 0) { - int written0 = fileIO.write(buf, written); + while (rem > 0) { + int written0 = fileIO.write(buf, written); - written += written0; + written += written0; - rem -= written0; + rem -= written0; + } } } + catch (IgniteCheckedException e) { + throw new IOException(e); + } + finally { + // Do the final fsync. + if (mode == WALMode.FSYNC) { + fileIO.force(); - // Do the final fsync. - if (mode == WALMode.FSYNC) { - fileIO.force(); + lastFsyncPos = written; + } - lastFsyncPos = written; + fileIO.close(); } - - fileIO.close(); } catch (IOException e) { - throw new IgniteCheckedException(e); + throw new StorageException(e); } if (log.isDebugEnabled()) @@ -2872,9 +2891,9 @@ private void signalNextAvailable() { } /** - * @throws IgniteCheckedException If failed. + * */ - private void awaitNext() throws IgniteCheckedException { + private void awaitNext() { lock.lock(); try { @@ -2894,7 +2913,7 @@ private void awaitNext() throws IgniteCheckedException { * @throws IgniteCheckedException If failed. */ @SuppressWarnings("TooBroadScope") - private void writeBuffer(long pos, ByteBuffer buf) throws StorageException, IgniteCheckedException { + private void writeBuffer(long pos, ByteBuffer buf) throws StorageException { boolean interrupted = false; lock.lock(); diff --git a/modules/core/src/test/java/org/apache/ignite/failure/TestFailureHandler.java b/modules/core/src/test/java/org/apache/ignite/failure/TestFailureHandler.java index 1159683e6b54f..545c9ea1176d1 100644 --- a/modules/core/src/test/java/org/apache/ignite/failure/TestFailureHandler.java +++ b/modules/core/src/test/java/org/apache/ignite/failure/TestFailureHandler.java @@ -18,6 +18,7 @@ package org.apache.ignite.failure; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import org.apache.ignite.Ignite; /** @@ -33,6 +34,13 @@ public class TestFailureHandler implements FailureHandler { /** Failure context. */ volatile FailureContext failureCtx; + /** + * @param invalidate Invalidate. + */ + public TestFailureHandler(boolean invalidate) { + this(invalidate, new CountDownLatch(1)); + } + /** * @param invalidate Invalidate. * @param latch Latch. @@ -60,4 +68,15 @@ public TestFailureHandler(boolean invalidate, CountDownLatch latch) { public FailureContext failureContext() { return failureCtx; } + + /** + * @param millis Millis. + + * @return Failure context. + */ + public FailureContext awaitFailure(long millis) throws InterruptedException { + latch.await(millis, TimeUnit.MILLISECONDS); + + return failureCtx; + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFormatFileFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFormatFileFailoverTest.java new file mode 100644 index 0000000000000..379b8c32cda89 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFormatFileFailoverTest.java @@ -0,0 +1,258 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.io.IOException; +import java.nio.file.OpenOption; +import java.util.Arrays; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.CacheAtomicityMode; +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.failure.FailureHandler; +import org.apache.ignite.failure.TestFailureHandler; +import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; +import org.apache.ignite.internal.pagemem.wal.StorageException; +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.X; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static java.nio.file.StandardOpenOption.CREATE; +import static java.nio.file.StandardOpenOption.READ; +import static java.nio.file.StandardOpenOption.WRITE; + +/** + * + */ +public class IgniteWalFormatFileFailoverTest extends GridCommonAbstractTest { + /** */ + private static final String TEST_CACHE = "testCache"; + + /** */ + private static final String formatFile = "formatFile"; + + /** Fail method name reference. */ + private final AtomicReference failMtdNameRef = new AtomicReference<>(); + + /** */ + private boolean fsync; + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setCacheConfiguration(new CacheConfiguration(TEST_CACHE) + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL)); + + DataStorageConfiguration memCfg = new DataStorageConfiguration() + .setDefaultDataRegionConfiguration(new DataRegionConfiguration() + .setMaxSize(2048L * 1024 * 1024) + .setPersistenceEnabled(true)) + .setWalMode(fsync ? WALMode.FSYNC : WALMode.BACKGROUND) + .setWalBufferSize(1024 * 1024) + .setWalSegmentSize(512 * 1024) + .setFileIOFactory(new FailingFileIOFactory(failMtdNameRef)); + + cfg.setDataStorageConfiguration(memCfg); + + cfg.setFailureHandler(new TestFailureHandler(false)); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testNodeStartFailedFsync() throws Exception { + fsync = true; + + failMtdNameRef.set(formatFile); + + checkCause(GridTestUtils.assertThrows(log, () -> startGrid(0), IgniteCheckedException.class, null)); + } + + /** + * @throws Exception If failed. + */ + public void testFailureHandlerTriggeredFsync() throws Exception { + fsync = true; + + failFormatFileOnClusterActivate(); + } + + /** + * @throws Exception If failed. + */ + public void testFailureHandlerTriggered() throws Exception { + fsync = false; + + failFormatFileOnClusterActivate(); + } + + /** + * @throws Exception If failed. + */ + private void failFormatFileOnClusterActivate() throws Exception { + failMtdNameRef.set(null); + + startGrid(0); + startGrid(1); + + if (!fsync) { + setFileIOFactory(grid(0).context().cache().context().wal()); + setFileIOFactory(grid(1).context().cache().context().wal()); + } + + failMtdNameRef.set(formatFile); + + grid(0).cluster().active(true); + + checkCause(failureHandler(0).awaitFailure(2000).error()); + checkCause(failureHandler(1).awaitFailure(2000).error()); + } + + /** + * @param mtdName Method name. + */ + private static boolean isCalledFrom(String mtdName) { + return isCalledFrom(Thread.currentThread().getStackTrace(), mtdName); + } + + /** + * @param stackTrace Stack trace. + * @param mtdName Method name. + */ + private static boolean isCalledFrom(StackTraceElement[] stackTrace, String mtdName) { + return Arrays.stream(stackTrace).map(StackTraceElement::getMethodName).anyMatch(mtdName::equals); + } + + /** + * @param gridIdx Grid index. + * @return Failure handler configured for grid with given index. + */ + private TestFailureHandler failureHandler(int gridIdx) { + FailureHandler hnd = grid(gridIdx).configuration().getFailureHandler(); + + assertTrue(hnd instanceof TestFailureHandler); + + return (TestFailureHandler)hnd; + } + + /** + * @param t Throwable. + */ + private void checkCause(Throwable t) { + StorageException e = X.cause(t, StorageException.class); + + assertNotNull(e); + assertNotNull(e.getMessage()); + assertTrue(e.getMessage().contains("Failed to format WAL segment file")); + + IOException ioe = X.cause(e, IOException.class); + + assertNotNull(ioe); + assertNotNull(ioe.getMessage()); + assertTrue(ioe.getMessage().contains("No space left on device")); + + assertTrue(isCalledFrom(ioe.getStackTrace(), formatFile)); + } + + /** */ + private void setFileIOFactory(IgniteWriteAheadLogManager wal) { + if (wal instanceof FileWriteAheadLogManager) + ((FileWriteAheadLogManager)wal).setFileIOFactory(new FailingFileIOFactory(failMtdNameRef)); + else + fail(wal.getClass().toString()); + } + + /** + * Create File I/O which fails if specific method call present in stack trace. + */ + private static class FailingFileIOFactory implements FileIOFactory { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Delegate factory. */ + private final FileIOFactory delegateFactory = new RandomAccessFileIOFactory(); + + /** Fail method name reference. */ + private final AtomicReference failMtdNameRef; + + /** + * @param failMtdNameRef Fail method name reference. + */ + FailingFileIOFactory(AtomicReference failMtdNameRef) { + assertNotNull(failMtdNameRef); + + this.failMtdNameRef = failMtdNameRef; + } + + /** {@inheritDoc} */ + @Override public FileIO create(File file) throws IOException { + return create(file, CREATE, READ, WRITE); + } + + /** {@inheritDoc} */ + @Override public FileIO create(File file, OpenOption... modes) throws IOException { + final FileIO delegate = delegateFactory.create(file, modes); + + return new FileIODecorator(delegate) { + @Override public int write(byte[] buf, int off, int len) throws IOException { + conditionalFail(); + + return super.write(buf, off, len); + } + + @Override public void clear() throws IOException { + conditionalFail(); + + super.clear(); + } + + private void conditionalFail() throws IOException { + String failMtdName = failMtdNameRef.get(); + + if (failMtdName != null && isCalledFrom(failMtdName)) + throw new IOException("No space left on device"); + } + }; + } + } +} 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 27757435a7d9c..316ff92fcdab2 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 @@ -49,6 +49,7 @@ import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushFsyncWithMmapBufferSelfTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushLogOnlySelfTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushLogOnlyWithMmapBufferSelfTest; +import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFormatFileFailoverTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalHistoryReservationsTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalIteratorSwitchSegmentTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalSerializerVersionTest; @@ -146,6 +147,8 @@ public static void addRealPageStoreTests(TestSuite suite) { suite.addTestSuite(IgniteWalFlushLogOnlyWithMmapBufferSelfTest.class); + suite.addTestSuite(IgniteWalFormatFileFailoverTest.class); + // Test suite uses Standalone WAL iterator to verify PDS content. suite.addTestSuite(IgniteWalReaderTest.class); From 914dbbb0852bab40ddeae36bf6c68e7101f69877 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Mon, 18 Jun 2018 15:07:48 +0300 Subject: [PATCH 0273/1463] IGNITE-8757 idle_verify utility doesn't show both update counter and hash conflicts (cherry picked from commit 9131e4d) --- .../internal/commandline/CommandHandler.java | 95 +++++ .../cache/verify/IdleVerifyResultV2.java | 113 ++++++ .../cache/verify/PartitionHashRecordV2.java | 168 +++++++++ .../cache/verify/PartitionKeyV2.java | 127 +++++++ .../verify/VerifyBackupPartitionsTask.java | 16 +- .../verify/VerifyBackupPartitionsTaskV2.java | 347 ++++++++++++++++++ .../visor/verify/VisorIdleVerifyTask.java | 2 + .../visor/verify/VisorIdleVerifyTaskV2.java | 94 +++++ .../resources/META-INF/classnames.properties | 1 + .../core/src/main/resources/ignite.properties | 2 +- .../junits/common/GridCommonAbstractTest.java | 6 +- .../ignite/util/GridCommandHandlerTest.java | 134 +++++++ 12 files changed, 1093 insertions(+), 12 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/IdleVerifyResultV2.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/PartitionHashRecordV2.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/PartitionKeyV2.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTaskV2.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskV2.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java index c59e348f768ae..be33991fa96c0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java @@ -34,6 +34,7 @@ import java.util.stream.Collectors; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.internal.client.GridClient; import org.apache.ignite.internal.client.GridClientAuthenticationException; import org.apache.ignite.internal.client.GridClientClosedException; @@ -51,8 +52,12 @@ import org.apache.ignite.internal.commandline.cache.CacheCommand; import org.apache.ignite.internal.processors.cache.verify.CacheInfo; import org.apache.ignite.internal.processors.cache.verify.ContentionInfo; +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.PartitionHashRecordV2; import org.apache.ignite.internal.processors.cache.verify.PartitionKey; +import org.apache.ignite.internal.processors.cache.verify.PartitionKeyV2; +import org.apache.ignite.internal.processors.cache.verify.VerifyBackupPartitionsTaskV2; 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; @@ -82,6 +87,7 @@ 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.internal.visor.verify.VisorValidateIndexesJobResult; import org.apache.ignite.internal.visor.verify.VisorValidateIndexesTaskArg; import org.apache.ignite.internal.visor.verify.VisorValidateIndexesTaskResult; @@ -89,6 +95,7 @@ import org.apache.ignite.internal.visor.verify.VisorViewCacheTaskArg; import org.apache.ignite.internal.visor.verify.VisorViewCacheTaskResult; import org.apache.ignite.lang.IgniteClosure; +import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.plugin.security.SecurityCredentials; import org.apache.ignite.plugin.security.SecurityCredentialsBasicProvider; @@ -710,10 +717,39 @@ private void cacheView(GridClient client, CacheArguments cacheArgs) throws GridC } /** + * Executes appropriate version of idle_verify check. Old version will be used if there are old nodes in the cluster. + * * @param client Client. * @param cacheArgs Cache args. */ private void cacheIdleVerify(GridClient client, CacheArguments cacheArgs) throws GridClientException { + Collection nodes = client.compute().nodes(GridClientNode::connectable); + + boolean idleVerifyV2 = true; + + for (GridClientNode node : nodes) { + String nodeVerStr = node.attribute(IgniteNodeAttributes.ATTR_BUILD_VER); + + IgniteProductVersion nodeVer = IgniteProductVersion.fromString(nodeVerStr); + + if (nodeVer.compareTo(VerifyBackupPartitionsTaskV2.V2_SINCE_VER) < 0) { + idleVerifyV2 = false; + + break; + } + } + + if (idleVerifyV2) + cacheIdleVerifyV2(client, cacheArgs); + else + legacyCacheIdleVerify(client, cacheArgs); + } + + /** + * @param client Client. + * @param cacheArgs Cache args. + */ + private void legacyCacheIdleVerify(GridClient client, CacheArguments cacheArgs) throws GridClientException { VisorIdleVerifyTaskResult res = executeTask( client, VisorIdleVerifyTask.class, new VisorIdleVerifyTaskArg(cacheArgs.caches())); @@ -735,6 +771,65 @@ private void cacheIdleVerify(GridClient client, CacheArguments cacheArgs) throws } } + /** + * @param client Client. + * @param cacheArgs Cache args. + */ + private void cacheIdleVerifyV2(GridClient client, CacheArguments cacheArgs) throws GridClientException { + IdleVerifyResultV2 res = executeTask( + client, VisorIdleVerifyTaskV2.class, new VisorIdleVerifyTaskArg(cacheArgs.caches())); + + if (!res.hasConflicts()) { + log("idle_verify check has finished, no conflicts have been found."); + nl(); + } + else { + int cntrConflictsSize = res.counterConflicts().size(); + int hashConflictsSize = res.hashConflicts().size(); + + log("idle_verify check has finished, found " + (cntrConflictsSize + hashConflictsSize) + + " conflict partitions: [counterConflicts=" + cntrConflictsSize + ", hashConflicts=" + + hashConflictsSize + "]"); + nl(); + + if (!F.isEmpty(res.counterConflicts())) { + log("Update counter conflicts:"); + + for (Map.Entry> entry : res.counterConflicts().entrySet()) { + log("Conflict partition: " + entry.getKey()); + + log("Partition instances: " + entry.getValue()); + } + + nl(); + } + + if (!F.isEmpty(res.hashConflicts())) { + log("Hash conflicts:"); + + for (Map.Entry> entry : res.hashConflicts().entrySet()) { + log("Conflict partition: " + entry.getKey()); + + log("Partition instances: " + entry.getValue()); + } + + nl(); + } + } + + if (!F.isEmpty(res.movingPartitions())) { + log("Verification was skipped for " + res.movingPartitions().size() + " MOVING partitions:"); + + for (Map.Entry> entry : res.movingPartitions().entrySet()) { + log("Rebalancing partition: " + entry.getKey()); + + log("Partition instances: " + entry.getValue()); + } + + nl(); + } + } + /** * Change baseline. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/IdleVerifyResultV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/IdleVerifyResultV2.java new file mode 100644 index 0000000000000..d5815cd2b02e1 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/IdleVerifyResultV2.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.verify; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.List; +import java.util.Map; +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.internal.visor.VisorDataTransferObject; + +/** + * Encapsulates result of {@link VerifyBackupPartitionsTaskV2}. + */ +public class IdleVerifyResultV2 extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Counter conflicts. */ + private Map> cntrConflicts; + + /** Hash conflicts. */ + private Map> hashConflicts; + + /** Moving partitions. */ + private Map> movingPartitions; + + /** + * @param cntrConflicts Counter conflicts. + * @param hashConflicts Hash conflicts. + * @param movingPartitions Moving partitions. + */ + public IdleVerifyResultV2( + Map> cntrConflicts, + Map> hashConflicts, + Map> movingPartitions + ) { + this.cntrConflicts = cntrConflicts; + this.hashConflicts = hashConflicts; + this.movingPartitions = movingPartitions; + } + + /** + * Default constructor for Externalizable. + */ + public IdleVerifyResultV2() { + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + U.writeMap(out, cntrConflicts); + U.writeMap(out, hashConflicts); + U.writeMap(out, movingPartitions); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, + ObjectInput in) throws IOException, ClassNotFoundException { + cntrConflicts = U.readMap(in); + hashConflicts = U.readMap(in); + movingPartitions = U.readMap(in); + } + + /** + * @return Counter conflicts. + */ + public Map> counterConflicts() { + return cntrConflicts; + } + + /** + * @return Hash conflicts. + */ + public Map> hashConflicts() { + return hashConflicts; + } + + /** + * @return Moving partitions. + */ + public Map> movingPartitions() { + return movingPartitions; + } + + /** + * @return true if any conflicts were discovered during idle_verify check. + */ + public boolean hasConflicts() { + return !F.isEmpty(hashConflicts()) || !F.isEmpty(counterConflicts()); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(IdleVerifyResultV2.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/PartitionHashRecordV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/PartitionHashRecordV2.java new file mode 100644 index 0000000000000..c0f8121057f10 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/PartitionHashRecordV2.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.verify; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * Record containing partition checksum, primary flag and consistent ID of owner. + */ +public class PartitionHashRecordV2 extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Moving partition size. */ + public static final long MOVING_PARTITION_SIZE = Long.MIN_VALUE; + + /** Partition key. */ + @GridToStringExclude + private PartitionKeyV2 partKey; + + /** Is primary flag. */ + private boolean isPrimary; + + /** Consistent id. */ + @GridToStringInclude + private Object consistentId; + + /** Partition hash. */ + @GridToStringExclude + private int partHash; + + /** Update counter. */ + private long updateCntr; + + /** Size. */ + @GridToStringExclude + private long size; + + /** + * @param partKey Partition key. + * @param isPrimary Is primary. + * @param consistentId Consistent id. + * @param partHash Partition hash. + * @param updateCntr Update counter. + * @param size Size. + */ + public PartitionHashRecordV2(PartitionKeyV2 partKey, boolean isPrimary, + Object consistentId, int partHash, long updateCntr, long size) { + this.partKey = partKey; + this.isPrimary = isPrimary; + this.consistentId = consistentId; + this.partHash = partHash; + this.updateCntr = updateCntr; + this.size = size; + } + + /** + * Default constructor for Externalizable. + */ + public PartitionHashRecordV2() { + } + + /** + * @return Partition key. + */ + public PartitionKeyV2 partitionKey() { + return partKey; + } + + /** + * @return Is primary. + */ + public boolean isPrimary() { + return isPrimary; + } + + /** + * @return Consistent id. + */ + public Object consistentId() { + return consistentId; + } + + /** + * @return Partition hash. + */ + public int partitionHash() { + return partHash; + } + + /** + * @return Update counter. + */ + public long updateCounter() { + return updateCntr; + } + + /** + * @return Size. + */ + public long size() { + return size; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + out.writeObject(partKey); + out.writeBoolean(isPrimary); + out.writeObject(consistentId); + out.writeInt(partHash); + out.writeLong(updateCntr); + out.writeLong(size); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + partKey = (PartitionKeyV2)in.readObject(); + isPrimary = in.readBoolean(); + consistentId = in.readObject(); + partHash = in.readInt(); + updateCntr = in.readLong(); + size = in.readLong(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return size == MOVING_PARTITION_SIZE ? + S.toString(PartitionHashRecordV2.class, this, "state", "MOVING") : + S.toString(PartitionHashRecordV2.class, this, "size", size, "partHash", partHash); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + PartitionHashRecordV2 record = (PartitionHashRecordV2)o; + + return consistentId.equals(record.consistentId); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return consistentId.hashCode(); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/PartitionKeyV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/PartitionKeyV2.java new file mode 100644 index 0000000000000..1332b0a0e3057 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/PartitionKeyV2.java @@ -0,0 +1,127 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.verify; + +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.VisorDataTransferObject; + +/** + * Partition key - pair of cache group ID and partition ID. + */ +public class PartitionKeyV2 extends VisorDataTransferObject { + /** */ + private static final long serialVersionUID = 0L; + + /** Group id. */ + private int grpId; + + /** Group name. Optional field, used only for output. */ + private volatile String grpName; + + /** Partition id. */ + private int partId; + + /** + * @param grpId Group id. + * @param partId Partition id. + * @param grpName Group name. + */ + public PartitionKeyV2(int grpId, int partId, String grpName) { + this.grpId = grpId; + this.partId = partId; + this.grpName = grpName; + } + + /** + * Default constructor for Externalizable. + */ + public PartitionKeyV2() { + } + + /** + * @return Group id. + */ + public int groupId() { + return grpId; + } + + /** + * @return Partition id. + */ + public int partitionId() { + return partId; + } + + /** + * @return Group name. + */ + public String groupName() { + return grpName; + } + + /** + * @param grpName Group name. + */ + public void groupName(String grpName) { + this.grpName = grpName; + } + + /** {@inheritDoc} */ + @Override protected void writeExternalData(ObjectOutput out) throws IOException { + out.writeInt(grpId); + U.writeString(out, grpName); + out.writeInt(partId); + } + + /** {@inheritDoc} */ + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + grpId = in.readInt(); + grpName = U.readString(in); + partId = in.readInt(); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + PartitionKeyV2 key = (PartitionKeyV2)o; + + return grpId == key.grpId && partId == key.partId; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + int res = grpId; + + res = 31 * res + partId; + + return res; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(PartitionKeyV2.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTask.java index b884cb01ac934..c99681864de0c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTask.java @@ -67,8 +67,11 @@ *
    * Works properly only on idle cluster - there may be false positive conflict reports if data in cluster is being * concurrently updated. + * + * @deprecated Legacy version of {@link VerifyBackupPartitionsTaskV2}. */ @GridInternal +@Deprecated public class VerifyBackupPartitionsTask extends ComputeTaskAdapter, Map>> { /** */ @@ -154,9 +157,10 @@ public class VerifyBackupPartitionsTask extends ComputeTaskAdapter, } /** - * + * Legacy version of {@link VerifyBackupPartitionsTaskV2} internal job, kept for compatibility. */ - public static class VerifyBackupPartitionsJob extends ComputeJobAdapter { + @Deprecated + private static class VerifyBackupPartitionsJob extends ComputeJobAdapter { /** */ private static final long serialVersionUID = 0L; @@ -177,7 +181,7 @@ public static class VerifyBackupPartitionsJob extends ComputeJobAdapter { /** * @param names Names. */ - private VerifyBackupPartitionsJob(Set names) { + public VerifyBackupPartitionsJob(Set names) { cacheNames = names; } @@ -244,7 +248,7 @@ private VerifyBackupPartitionsJob(Set names) { Future> fut = partHashCalcFutures.get(i); try { - Map partHash = fut.get(10, TimeUnit.SECONDS); + Map partHash = fut.get(100, TimeUnit.MILLISECONDS); res.putAll(partHash); @@ -261,7 +265,7 @@ else if (e.getCause() instanceof IgniteException) else throw new IgniteException(e.getCause()); } - catch (TimeoutException e) { + catch (TimeoutException ignored) { if (U.currentTimeMillis() - lastProgressLogTs > 3 * 60 * 1000L) { lastProgressLogTs = U.currentTimeMillis(); @@ -289,7 +293,6 @@ private Future> calculatePartitionHashAsy }); } - /** * @param grpCtx Group context. * @param part Local partition. @@ -355,5 +358,4 @@ private Map calculatePartitionHash( return Collections.singletonMap(partKey, partRec); } } - } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTaskV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTaskV2.java new file mode 100644 index 0000000000000..826393ac09602 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTaskV2.java @@ -0,0 +1,347 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.verify; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteInterruptedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.compute.ComputeJob; +import org.apache.ignite.compute.ComputeJobAdapter; +import org.apache.ignite.compute.ComputeJobResult; +import org.apache.ignite.compute.ComputeTaskAdapter; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor; +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.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.task.GridInternal; +import org.apache.ignite.internal.util.lang.GridIterator; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.internal.visor.verify.VisorIdleVerifyTaskArg; +import org.apache.ignite.lang.IgniteProductVersion; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.resources.LoggerResource; +import org.jetbrains.annotations.Nullable; + +/** + * Task for comparing update counters and checksums between primary and backup partitions of specified caches. + *
    + * Argument: Set of cache names, 'null' will trigger verification for all caches. + *
    + * Result: {@link IdleVerifyResultV2} with conflict partitions. + *
    + * Works properly only on idle cluster - there may be false positive conflict reports if data in cluster is being + * concurrently updated. + */ +@GridInternal +public class VerifyBackupPartitionsTaskV2 extends ComputeTaskAdapter { + /** First version of Ignite that is capable of executing Idle Verify V2. */ + public static final IgniteProductVersion V2_SINCE_VER = IgniteProductVersion.fromString("2.5.3"); + + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Nullable @Override public Map map( + List subgrid, VisorIdleVerifyTaskArg arg) throws IgniteException { + Map jobs = new HashMap<>(); + + for (ClusterNode node : subgrid) + jobs.put(new VerifyBackupPartitionsJobV2(arg), node); + + return jobs; + } + + /** {@inheritDoc} */ + @Nullable @Override public IdleVerifyResultV2 reduce(List results) + throws IgniteException { + Map> clusterHashes = new HashMap<>(); + + for (ComputeJobResult res : results) { + Map nodeHashes = res.getData(); + + for (Map.Entry e : nodeHashes.entrySet()) { + List records = clusterHashes.computeIfAbsent(e.getKey(), k -> new ArrayList<>()); + + records.add(e.getValue()); + } + } + + Map> hashConflicts = new HashMap<>(); + + Map> updateCntrConflicts = new HashMap<>(); + + Map> movingParts = new HashMap<>(); + + for (Map.Entry> e : clusterHashes.entrySet()) { + Integer partHash = null; + Long updateCntr = null; + + for (PartitionHashRecordV2 record : e.getValue()) { + if (record.size() == PartitionHashRecordV2.MOVING_PARTITION_SIZE) { + List records = movingParts.computeIfAbsent( + e.getKey(), k -> new ArrayList<>()); + + records.add(record); + + continue; + } + + if (partHash == null) { + partHash = record.partitionHash(); + + updateCntr = record.updateCounter(); + } + else { + if (record.updateCounter() != updateCntr) + updateCntrConflicts.putIfAbsent(e.getKey(), e.getValue()); + + if (record.partitionHash() != partHash) + hashConflicts.putIfAbsent(e.getKey(), e.getValue()); + } + } + } + + return new IdleVerifyResultV2(updateCntrConflicts, hashConflicts, movingParts); + } + + /** + * Job that collects update counters and hashes of local partitions. + */ + private static class VerifyBackupPartitionsJobV2 extends ComputeJobAdapter { + /** */ + private static final long serialVersionUID = 0L; + + /** Ignite instance. */ + @IgniteInstanceResource + private IgniteEx ignite; + + /** Injected logger. */ + @LoggerResource + private IgniteLogger log; + + /** Idle verify arguments. */ + private VisorIdleVerifyTaskArg arg; + + /** Counter of processed partitions. */ + private final AtomicInteger completionCntr = new AtomicInteger(0); + + /** + * @param arg Argument. + */ + public VerifyBackupPartitionsJobV2(VisorIdleVerifyTaskArg arg) { + this.arg = arg; + } + + /** {@inheritDoc} */ + @Override public Map execute() throws IgniteException { + Set grpIds = new HashSet<>(); + + Set missingCaches = new HashSet<>(); + + if (arg.getCaches() != null) { + for (String cacheName : arg.getCaches()) { + DynamicCacheDescriptor desc = ignite.context().cache().cacheDescriptor(cacheName); + + if (desc == null) { + missingCaches.add(cacheName); + + continue; + } + + grpIds.add(desc.groupId()); + } + + if (!missingCaches.isEmpty()) { + StringBuilder strBuilder = new StringBuilder("The following caches do not exist: "); + + for (String name : missingCaches) + strBuilder.append(name).append(", "); + + strBuilder.delete(strBuilder.length() - 2, strBuilder.length()); + + throw new IgniteException(strBuilder.toString()); + } + } + else { + Collection groups = ignite.context().cache().cacheGroups(); + + for (CacheGroupContext grp : groups) { + if (!grp.systemCache() && !grp.isLocal()) + grpIds.add(grp.groupId()); + } + } + + List>> partHashCalcFutures = new ArrayList<>(); + + completionCntr.set(0); + + for (Integer grpId : grpIds) { + CacheGroupContext grpCtx = ignite.context().cache().cacheGroup(grpId); + + if (grpCtx == null) + continue; + + List parts = grpCtx.topology().localPartitions(); + + for (GridDhtLocalPartition part : parts) + partHashCalcFutures.add(calculatePartitionHashAsync(grpCtx, part)); + } + + Map res = new HashMap<>(); + + long lastProgressLogTs = U.currentTimeMillis(); + + for (int i = 0; i < partHashCalcFutures.size(); ) { + Future> fut = partHashCalcFutures.get(i); + + try { + Map partHash = fut.get(100, TimeUnit.MILLISECONDS); + + res.putAll(partHash); + + i++; + } + catch (InterruptedException | ExecutionException e) { + for (int j = i + 1; j < partHashCalcFutures.size(); j++) + partHashCalcFutures.get(j).cancel(false); + + if (e instanceof InterruptedException) + throw new IgniteInterruptedException((InterruptedException)e); + else if (e.getCause() instanceof IgniteException) + throw (IgniteException)e.getCause(); + else + throw new IgniteException(e.getCause()); + } + catch (TimeoutException ignored) { + if (U.currentTimeMillis() - lastProgressLogTs > 3 * 60 * 1000L) { + lastProgressLogTs = U.currentTimeMillis(); + + log.warning("idle_verify is still running, processed " + completionCntr.get() + " of " + + partHashCalcFutures.size() + " local partitions"); + } + } + } + + return res; + } + + /** + * @param grpCtx Group context. + * @param part Local partition. + */ + private Future> calculatePartitionHashAsync( + final CacheGroupContext grpCtx, + final GridDhtLocalPartition part + ) { + return ForkJoinPool.commonPool().submit(new Callable>() { + @Override public Map call() throws Exception { + return calculatePartitionHash(grpCtx, part); + } + }); + } + + + /** + * @param grpCtx Group context. + * @param part Local partition. + */ + private Map calculatePartitionHash( + CacheGroupContext grpCtx, + GridDhtLocalPartition part + ) { + if (!part.reserve()) + return Collections.emptyMap(); + + int partHash = 0; + long partSize; + long updateCntrBefore = part.updateCounter(); + + PartitionKeyV2 partKey = new PartitionKeyV2(grpCtx.groupId(), part.id(), grpCtx.cacheOrGroupName()); + + Object consId = ignite.context().discovery().localNode().consistentId(); + + boolean isPrimary = part.primary(grpCtx.topology().readyTopologyVersion()); + + try { + if (part.state() == GridDhtPartitionState.MOVING) { + PartitionHashRecordV2 movingHashRecord = new PartitionHashRecordV2(partKey, isPrimary, consId, + partHash, updateCntrBefore, PartitionHashRecordV2.MOVING_PARTITION_SIZE); + + return Collections.singletonMap(partKey, movingHashRecord); + } + else if (part.state() != GridDhtPartitionState.OWNING) + return Collections.emptyMap(); + + partSize = part.dataStore().fullSize(); + + GridIterator it = grpCtx.offheap().partitionIterator(part.id()); + + while (it.hasNextX()) { + CacheDataRow row = it.nextX(); + + partHash += row.key().hashCode(); + + partHash += Arrays.hashCode(row.value().valueBytes(grpCtx.cacheObjectContext())); + } + + long updateCntrAfter = part.updateCounter(); + + if (updateCntrBefore != updateCntrAfter) { + throw new IgniteException("Cluster is not idle: update counter of partition [grpId=" + + grpCtx.groupId() + ", partId=" + part.id() + "] changed during hash calculation [before=" + + updateCntrBefore + ", after=" + updateCntrAfter + "]"); + } + } + catch (IgniteCheckedException e) { + U.error(log, "Can't calculate partition hash [grpId=" + grpCtx.groupId() + + ", partId=" + part.id() + "]", e); + + return Collections.emptyMap(); + } + finally { + part.release(); + } + + PartitionHashRecordV2 partRec = new PartitionHashRecordV2( + partKey, isPrimary, consId, partHash, updateCntrBefore, partSize); + + completionCntr.incrementAndGet(); + + return Collections.singletonMap(partKey, partRec); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTask.java index 05f2621310366..e67c681acea45 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTask.java @@ -37,6 +37,7 @@ * Task to verify checksums of backup partitions. */ @GridInternal +@Deprecated public class VisorIdleVerifyTask extends VisorOneNodeTask { /** */ private static final long serialVersionUID = 0L; @@ -49,6 +50,7 @@ public class VisorIdleVerifyTask extends VisorOneNodeTask { /** */ private static final long serialVersionUID = 0L; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskV2.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskV2.java new file mode 100644 index 0000000000000..b9250ef28c543 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/VisorIdleVerifyTaskV2.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.visor.verify; + +import org.apache.ignite.IgniteException; +import org.apache.ignite.compute.ComputeJobContext; +import org.apache.ignite.compute.ComputeTaskFuture; +import org.apache.ignite.internal.processors.cache.verify.IdleVerifyResultV2; +import org.apache.ignite.internal.processors.cache.verify.VerifyBackupPartitionsTaskV2; +import org.apache.ignite.internal.processors.task.GridInternal; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.visor.VisorJob; +import org.apache.ignite.internal.visor.VisorOneNodeTask; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.resources.JobContextResource; + +/** + * Task to verify checksums of backup partitions. + */ +@GridInternal +public class VisorIdleVerifyTaskV2 extends VisorOneNodeTask { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override protected VisorJob job(VisorIdleVerifyTaskArg arg) { + return new VisorIdleVerifyJobV2(arg, debug); + } + + /** + * + */ + private static class VisorIdleVerifyJobV2 extends VisorJob { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private ComputeTaskFuture fut; + + /** Auto-inject job context. */ + @JobContextResource + protected transient ComputeJobContext jobCtx; + + /** + * @param arg Argument. + * @param debug Debug. + */ + private VisorIdleVerifyJobV2(VisorIdleVerifyTaskArg arg, boolean debug) { + super(arg, debug); + } + + /** {@inheritDoc} */ + @Override protected IdleVerifyResultV2 run(VisorIdleVerifyTaskArg arg) throws IgniteException { + if (fut == null) { + fut = ignite.compute().executeAsync(VerifyBackupPartitionsTaskV2.class, arg); + + if (!fut.isDone()) { + jobCtx.holdcc(); + + fut.listen(new IgniteInClosure>() { + @Override public void apply(IgniteFuture f) { + jobCtx.callcc(); + } + }); + + return null; + } + } + + return fut.get(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorIdleVerifyJobV2.class, this); + } + } +} diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties index f1aec976e4c53..6c91de43700cb 100644 --- a/modules/core/src/main/resources/META-INF/classnames.properties +++ b/modules/core/src/main/resources/META-INF/classnames.properties @@ -1138,6 +1138,7 @@ org.apache.ignite.internal.processors.cache.verify.CollectConflictPartitionKeysT org.apache.ignite.internal.processors.cache.verify.CollectConflictPartitionKeysTask$CollectPartitionEntryHashesJob org.apache.ignite.internal.processors.cache.verify.ContentionClosure org.apache.ignite.internal.processors.cache.verify.ContentionInfo +org.apache.ignite.internal.processors.cache.verify.IdleVerifyResultV2 org.apache.ignite.internal.processors.cache.verify.PartitionEntryHashRecord org.apache.ignite.internal.processors.cache.verify.PartitionHashRecord org.apache.ignite.internal.processors.cache.verify.PartitionKey diff --git a/modules/core/src/main/resources/ignite.properties b/modules/core/src/main/resources/ignite.properties index e70660b1c6a64..785f962378054 100644 --- a/modules/core/src/main/resources/ignite.properties +++ b/modules/core/src/main/resources/ignite.properties @@ -15,7 +15,7 @@ # limitations under the License. # -ignite.version=2.5.0-SNAPSHOT +ignite.version=2.6.0-SNAPSHOT ignite.build=0 ignite.revision=DEV ignite.rel.date=01011970 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 71abb95c47ac3..5b34f16813809 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 @@ -799,10 +799,8 @@ protected void verifyBackupPartitions(Ignite ig, Set cacheNames) throws Map> conflicts = ig.compute().execute( new VerifyBackupPartitionsTask(), cacheNames); - if (!conflicts.isEmpty()) { - throw new IgniteCheckedException("Partition checksums are different for backups " + - "of the following partitions: " + conflicts.keySet()); - } + if (!conflicts.isEmpty()) + throw new IgniteCheckedException("Conflict partitions: " + conflicts.keySet()); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java index f6d1e1930d6af..3835ef4f4d1ad 100644 --- a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java @@ -48,7 +48,14 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.commandline.CommandHandler; import org.apache.ignite.internal.commandline.cache.CacheCommand; +import org.apache.ignite.internal.pagemem.wal.record.DataEntry; +import org.apache.ignite.internal.processors.cache.CacheObjectImpl; +import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheFuture; +import org.apache.ignite.internal.processors.cache.GridCacheOperation; +import org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl; +import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.X; @@ -602,6 +609,77 @@ public void testCacheIdleVerify() throws Exception { assertTrue(testOut.toString().contains("conflict partitions")); } + /** + * Tests that both update counter and hash conflicts are detected. + * + * @throws Exception If failed. + */ + public void testCacheIdleVerifyTwoConflictTypes() throws Exception { + IgniteEx ignite = (IgniteEx)startGrids(2); + + ignite.cluster().active(true); + + int parts = 32; + + IgniteCache cache = ignite.createCache(new CacheConfiguration<>() + .setAffinity(new RendezvousAffinityFunction(false, parts)) + .setBackups(1) + .setName(DEFAULT_CACHE_NAME)); + + for (int i = 0; i < 100; i++) + cache.put(i, i); + + injectTestSystemOut(); + + assertEquals(EXIT_CODE_OK, execute("--cache", "idle_verify")); + + assertTrue(testOut.toString().contains("no conflicts have been found")); + + GridCacheContext cacheCtx = ignite.cachex(DEFAULT_CACHE_NAME).context(); + + corruptDataEntry(cacheCtx, 1, true, false); + + corruptDataEntry(cacheCtx, 1 + parts / 2, false, true); + + assertEquals(EXIT_CODE_OK, execute("--cache", "idle_verify")); + + assertTrue(testOut.toString().contains("found 2 conflict partitions")); + } + + /** + * @throws Exception If failed. + */ + public void testCacheIdleVerifyMovingParts() throws Exception { + IgniteEx ignite = (IgniteEx)startGrids(2); + + ignite.cluster().active(true); + + int parts = 32; + + IgniteCache cache = ignite.createCache(new CacheConfiguration<>() + .setAffinity(new RendezvousAffinityFunction(false, parts)) + .setBackups(1) + .setName(DEFAULT_CACHE_NAME) + .setRebalanceDelay(10_000)); + + for (int i = 0; i < 100; i++) + cache.put(i, i); + + injectTestSystemOut(); + + assertEquals(EXIT_CODE_OK, execute("--cache", "idle_verify")); + + assertTrue(testOut.toString().contains("no conflicts have been found")); + + startGrid(2); + + resetBaselineTopology(); + + assertEquals(EXIT_CODE_OK, execute("--cache", "idle_verify")); + + assertTrue(testOut.toString().contains("MOVING partitions")); + } + /** * */ @@ -925,4 +1003,60 @@ private void checkFutures() { assertTrue("Expecting no active futures: node=" + ig.localNode().id(), futs.isEmpty()); } } + + /** + * Corrupts data entry. + * + * @param ctx Context. + * @param key Key. + * @param breakCntr Break counter. + * @param breakData Break data. + */ + private void corruptDataEntry( + GridCacheContext ctx, + int key, + boolean breakCntr, + boolean breakData + ) { + int partId = ctx.affinity().partition(key); + + try { + long updateCntr = ctx.topology().localPartition(partId).updateCounter(); + + Object valToPut = ctx.cache().keepBinary().get(key); + + if (breakCntr) + updateCntr++; + + if (breakData) + valToPut = valToPut.toString() + " broken"; + + // Create data entry + DataEntry dataEntry = new DataEntry( + ctx.cacheId(), + new KeyCacheObjectImpl(key, null, partId), + new CacheObjectImpl(valToPut, null), + GridCacheOperation.UPDATE, + new GridCacheVersion(), + new GridCacheVersion(), + 0L, + partId, + updateCntr + ); + + GridCacheDatabaseSharedManager db = (GridCacheDatabaseSharedManager)ctx.shared().database(); + + db.checkpointReadLock(); + + try { + U.invoke(GridCacheDatabaseSharedManager.class, db, "applyUpdate", ctx, dataEntry); + } + finally { + db.checkpointReadUnlock(); + } + } + catch (IgniteCheckedException e) { + e.printStackTrace(); + } + } } From 80c7cecd146fca25e71c621e3b18086271937881 Mon Sep 17 00:00:00 2001 From: Anton Kalashnikov Date: Mon, 18 Jun 2018 18:53:43 +0300 Subject: [PATCH 0274/1463] IGNITE-8707 DataStorageMetrics.getTotalAllocatedSize metric does not account reserved partition page header - Fixes #4202. Signed-off-by: Ivan Rakov (cherry picked from commit ceade87) --- .../cache/persistence/file/FilePageStore.java | 8 +- .../db/IgnitePdsDataRegionMetricsTest.java | 95 ++++++++++--------- 2 files changed, 50 insertions(+), 53 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java index 852eb0da870b1..f614032c58b15 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java @@ -279,7 +279,7 @@ public void truncate(int tag) throws PersistentStorageIOException { throw new PersistentStorageIOException("Failed to delete partition file: " + cfgFile.getPath(), e); } finally { - allocatedTracker.updateTotalAllocatedPages(-1L * pages()); + allocatedTracker.updateTotalAllocatedPages(-1L * allocated.get() / pageSize); allocated.set(0); @@ -440,11 +440,7 @@ private void init() throws PersistentStorageIOException { assert allocated.get() == 0; - long delta = newSize - headerSize(); - - assert delta % pageSize == 0; - - allocatedTracker.updateTotalAllocatedPages(delta / pageSize); + allocatedTracker.updateTotalAllocatedPages(newSize / pageSize); allocated.set(newSize); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java index 18a47814755d1..268d2fb53dda3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsDataRegionMetricsTest.java @@ -17,13 +17,14 @@ package org.apache.ignite.internal.processors.cache.persistence.db; +import java.io.File; +import java.nio.file.DirectoryStream; +import java.nio.file.Path; +import java.util.HashMap; import java.util.Map; -import java.util.Set; -import java.util.UUID; import java.util.Random; -import java.util.HashMap; +import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; -import java.util.stream.Collectors; import org.apache.ignite.DataRegionMetrics; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; @@ -34,20 +35,19 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.processors.cache.CacheGroupContext; -import org.apache.ignite.internal.processors.cache.persistence.DataRegion; import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl; import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.util.future.GridFutureAdapter; -import org.apache.ignite.internal.util.typedef.PA; import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.CU; 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.junit.Assert; +import static java.nio.file.Files.newDirectoryStream; import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_DATA_REG_DEFAULT_NAME; /** @@ -136,9 +136,6 @@ public void testMemoryUsageSingleNode() throws Exception { final IgniteCache cache = node.getOrCreateCache(DEFAULT_CACHE_NAME); - final Set grpIds = node.context().cache().cacheGroups() - .stream().map(CacheGroupContext::groupId).collect(Collectors.toSet()); - Map map = new HashMap<>(); for (int batch = 0; batch < BATCHES; batch++) { @@ -149,7 +146,9 @@ public void testMemoryUsageSingleNode() throws Exception { cache.putAll(map); - checkMetricsConsistency(node, grpIds); + forceCheckpoint(); + + checkMetricsConsistency(node, DEFAULT_CACHE_NAME); } currMetrics = getDfltRegionMetrics(node); @@ -169,22 +168,21 @@ public void testMemoryUsageMultipleNodes() throws Exception { node0.cluster().active(true); - final IgniteCache cache = node0.getOrCreateCache(DEFAULT_CACHE_NAME); + final IgniteCache cache = node0.getOrCreateCache(DEFAULT_CACHE_NAME); - final Set grpIds = node0.context().cache().cacheGroups() - .stream().map(CacheGroupContext::groupId).collect(Collectors.toSet()); - - Map map = new HashMap<>(); + Map map = new HashMap<>(); for (int i = 0; i < 10_000; i++) - map.put(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + map.put(i, UUID.randomUUID().toString()); cache.putAll(map); awaitPartitionMapExchange(true, true, null); - checkMetricsConsistency(node0, grpIds); - checkMetricsConsistency(node1, grpIds); + forceCheckpoint(); + + checkMetricsConsistency(node0, DEFAULT_CACHE_NAME); + checkMetricsConsistency(node1, DEFAULT_CACHE_NAME); IgniteEx node2 = startGrid(2); @@ -192,9 +190,11 @@ public void testMemoryUsageMultipleNodes() throws Exception { awaitPartitionMapExchange(true, true, null); - checkMetricsConsistency(node0, grpIds); - checkMetricsConsistency(node1, grpIds); - checkMetricsConsistency(node2, grpIds); + forceCheckpoint(); + + checkMetricsConsistency(node0, DEFAULT_CACHE_NAME); + checkMetricsConsistency(node1, DEFAULT_CACHE_NAME); + checkMetricsConsistency(node2, DEFAULT_CACHE_NAME); stopGrid(1, true); @@ -202,8 +202,10 @@ public void testMemoryUsageMultipleNodes() throws Exception { awaitPartitionMapExchange(true, true, null); - checkMetricsConsistency(node0, grpIds); - checkMetricsConsistency(node2, grpIds); + forceCheckpoint(); + + checkMetricsConsistency(node0, DEFAULT_CACHE_NAME); + checkMetricsConsistency(node2, DEFAULT_CACHE_NAME); } /** @@ -289,29 +291,28 @@ private static DataRegionMetrics getDfltRegionMetrics(Ignite node) { } /** */ - private static void checkMetricsConsistency( - final IgniteEx node, - final Set grpIds) throws Exception { - boolean storageMatches = GridTestUtils.waitForCondition((PA)() -> { - long pagesInStore = 0; - long allocated = 0; - - for (int grpId : grpIds) { - DataRegion region = node.context().cache().cacheGroup(grpId).dataRegion(); - - if (!region.config().isMetricsEnabled()) - continue; - - pagesInStore += node.context().cache().context().pageStore().pagesAllocated(grpId); - allocated += region.memoryMetrics().getTotalAllocatedPages(); - } - - assert 0 != pagesInStore; - assert 0 != allocated; + private void checkMetricsConsistency(final IgniteEx node, String cacheName) throws Exception { + FilePageStoreManager pageStoreManager = (FilePageStoreManager)node.context().cache().context().pageStore(); + + long totalPersistanceSize = 0; + File cacheWorkDir = pageStoreManager.cacheWorkDir( + node.getOrCreateCache(cacheName).getConfiguration(CacheConfiguration.class) + ); + + try (DirectoryStream files = newDirectoryStream( + cacheWorkDir.toPath(), entry -> entry.toFile().getName().endsWith(".bin")) + ) { + for (Path path : files) + totalPersistanceSize += path.toFile().length(); + } - return allocated == pagesInStore; - }, 1000); + long totalAllocatedPagesFromMetrics = node.context().cache().context() + .cacheContext(CU.cacheId(DEFAULT_CACHE_NAME)) + .group() + .dataRegion() + .memoryMetrics() + .getTotalAllocatedPages(); - assertTrue(storageMatches); + assertEquals(totalPersistanceSize / pageStoreManager.pageSize(), totalAllocatedPagesFromMetrics); } } From 2cb10a7ccbf18d7893450a0529f95743c0e395df Mon Sep 17 00:00:00 2001 From: Sergey Kosarev Date: Mon, 18 Jun 2018 18:48:05 +0300 Subject: [PATCH 0275/1463] IGNITE-8815 Ease enabling WAL management in control.sh - Fixes #4212. Signed-off-by: Alexey Goncharuk (cherry picked from commit b80482d125d08a4779c8f7b1ea75e4dd1a4c4bde) --- bin/control.sh | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/bin/control.sh b/bin/control.sh index e2b75afbd1a07..7f84696831c50 100755 --- a/bin/control.sh +++ b/bin/control.sh @@ -92,6 +92,11 @@ if [ -z "$JVM_OPTS" ] ; then fi fi +# +# Uncomment to enable experimental commands [--wal] +# +# JVM_OPTS="${JVM_OPTS} -DIGNITE_ENABLE_EXPERIMENTAL_COMMAND=true" + # # Uncomment the following GC settings if you see spikes in your throughput due to Garbage Collection. # From 1d34cf3f4ce3880b4ce3ca8829b49d08710773e2 Mon Sep 17 00:00:00 2001 From: devozerov Date: Fri, 15 Jun 2018 11:43:01 +0300 Subject: [PATCH 0276/1463] IGNITE-8800: Binary: print type name and existing schema IDs in case of exception due to missing schema. This closes #4190. (cherry picked from commit 7a39efb) --- .../internal/binary/BinaryReaderExImpl.java | 31 ++++++++++++++----- 1 file changed, 24 insertions(+), 7 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 2b7964c609a3d..f88e3c3234b41 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 @@ -23,8 +23,10 @@ import java.math.BigDecimal; import java.sql.Time; import java.sql.Timestamp; +import java.util.ArrayList; import java.util.Collection; import java.util.Date; +import java.util.List; import java.util.Map; import java.util.UUID; import org.apache.ignite.binary.BinaryCollectionFactory; @@ -2004,21 +2006,36 @@ public BinarySchema getOrCreateSchema() { if (fieldIdLen != BinaryUtils.FIELD_ID_LEN) { BinaryTypeImpl type = (BinaryTypeImpl) ctx.metadata(typeId, schemaId); - if (type == null || type.metadata() == null) + BinaryMetadata meta = type != null ? type.metadata() : null; + + if (type == null || meta == null) throw new BinaryObjectException("Cannot find metadata for object with compact footer: " + typeId); - for (BinarySchema typeSchema : type.metadata().schemas()) { - if (schemaId == typeSchema.schemaId()) { - schema = typeSchema; + Collection existingSchemas = meta.schemas(); + + for (BinarySchema existingSchema : existingSchemas) { + if (schemaId == existingSchema.schemaId()) { + schema = existingSchema; break; } } - if (schema == null) - throw new BinaryObjectException("Cannot find schema for object with compact footer [" + - "typeId=" + typeId + ", schemaId=" + schemaId + ']'); + if (schema == null) { + List existingSchemaIds = new ArrayList<>(existingSchemas.size()); + + for (BinarySchema existingSchema : existingSchemas) + existingSchemaIds.add(existingSchema.schemaId()); + + + throw new BinaryObjectException("Cannot find schema for object with compact footer" + + " [typeName=" + type.typeName() + + ", typeId=" + typeId + + ", missingSchemaId=" + schemaId + + ", existingSchemaIds=" + existingSchemaIds + ']' + ); + } } else schema = createSchema(); From 2d554b704a365023a5c910d312786de8286cf219 Mon Sep 17 00:00:00 2001 From: a-polyakov Date: Mon, 18 Jun 2018 19:11:32 +0300 Subject: [PATCH 0277/1463] IGNITE-8601 Add to control.sh utility information about transaction start time - Fixes #4070. (cherry picked from commit 4776a1a) --- .../internal/TransactionsMXBeanImpl.java | 2 +- .../internal/commandline/CommandHandler.java | 6 ++- .../ignite/internal/visor/tx/VisorTxInfo.java | 37 +++++++++++++++-- .../internal/visor/tx/VisorTxSortOrder.java | 8 +++- .../ignite/internal/visor/tx/VisorTxTask.java | 20 ++++++++- .../ignite/util/GridCommandHandlerTest.java | 41 +++++++++++-------- 6 files changed, 89 insertions(+), 25 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/TransactionsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/TransactionsMXBeanImpl.java index 6937ebd146c4a..210a32008d313 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/TransactionsMXBeanImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/TransactionsMXBeanImpl.java @@ -25,7 +25,6 @@ import java.util.UUID; import java.util.stream.Collectors; import org.apache.ignite.IgniteCompute; -import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; @@ -107,6 +106,7 @@ else if ("SIZE".equals(order)) w.println(" Tx: [xid=" + info.getXid() + ", label=" + info.getLabel() + ", state=" + info.getState() + + ", startTime=" + info.getFormattedStartTime() + ", duration=" + info.getDuration() / 1000 + ", isolation=" + info.getIsolation() + ", concurrency=" + info.getConcurrency() + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java index be33991fa96c0..ed85f0c850ba9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java @@ -236,6 +236,9 @@ public class CommandHandler { /** */ private static final String TX_ORDER = "order"; + /** */ + public static final String CMD_TX_ORDER_START_TIME="START_TIME"; + /** */ private static final String TX_SERVERS = "servers"; @@ -1079,6 +1082,7 @@ else if (arg.getOperation() == VisorTxOperation.KILL) log(" Tx: [xid=" + info.getXid() + ", label=" + info.getLabel() + ", state=" + info.getState() + + ", startTime=" + info.getFormattedStartTime() + ", duration=" + info.getDuration() / 1000 + ", isolation=" + info.getIsolation() + ", concurrency=" + info.getConcurrency() + @@ -1826,7 +1830,7 @@ public int execute(List rawArgs) { usage(" Set baseline topology based on version:", BASELINE, " version topologyVersion [--force]"); usage(" List or kill transactions:", TX, " [xid XID] [minDuration SECONDS] " + "[minSize SIZE] [label PATTERN_REGEX] [servers|clients] " + - "[nodes consistentId1[,consistentId2,....,consistentIdN] [limit NUMBER] [order DURATION|SIZE] [kill] [--force]"); + "[nodes consistentId1[,consistentId2,....,consistentIdN] [limit NUMBER] [order DURATION|SIZE|", CMD_TX_ORDER_START_TIME, "] [kill] [--force]"); if(enableExperimental) { usage(" Print absolute paths of unused archived wal segments on each node:", WAL, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxInfo.java index 89bf274653165..ecf3e0d79fb4c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxInfo.java @@ -20,8 +20,11 @@ import java.io.IOException; import java.io.ObjectInput; import java.io.ObjectOutput; -import java.io.Serializable; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; import java.util.Collection; +import java.util.TimeZone; import java.util.UUID; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -37,9 +40,17 @@ public class VisorTxInfo extends VisorDataTransferObject { /** */ private static final long serialVersionUID = 0L; + /** */ + private static DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS"); + /** */ private IgniteUuid xid; + /** + * Transaction start time. + */ + private long startTime; + /** */ private long duration; @@ -73,6 +84,7 @@ public VisorTxInfo() { /** * @param xid Xid. + * @param startTime Start time of transaction. * @param duration Duration. * @param isolation Isolation. * @param concurrency Concurrency. @@ -82,10 +94,11 @@ public VisorTxInfo() { * @param state State. * @param size Size. */ - public VisorTxInfo(IgniteUuid xid, long duration, TransactionIsolation isolation, + public VisorTxInfo(IgniteUuid xid, long startTime, long duration, TransactionIsolation isolation, TransactionConcurrency concurrency, long timeout, String lb, Collection primaryNodes, TransactionState state, int size) { this.xid = xid; + this.startTime = startTime; this.duration = duration; this.isolation = isolation; this.concurrency = concurrency; @@ -101,6 +114,16 @@ public IgniteUuid getXid() { return xid; } + /** */ + public long getStartTime() { + return startTime; + } + + /** */ + public String getFormattedStartTime() { + return dateTimeFormatter.format(LocalDateTime.ofInstant(Instant.ofEpochMilli(startTime), TimeZone.getDefault().toZoneId())); + } + /** */ public long getDuration() { return duration; @@ -141,6 +164,11 @@ public int getSize() { return size; } + /** {@inheritDoc} */ + @Override public byte getProtocolVersion() { + return V2; + } + /** {@inheritDoc} */ @Override protected void writeExternalData(ObjectOutput out) throws IOException { U.writeGridUuid(out, xid); @@ -152,10 +180,12 @@ public int getSize() { U.writeCollection(out, primaryNodes); U.writeEnum(out, state); out.writeInt(size); + out.writeLong(startTime); } /** {@inheritDoc} */ - @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { + @Override protected void readExternalData(byte protoVer, + ObjectInput in) throws IOException, ClassNotFoundException { xid = U.readGridUuid(in); duration = in.readLong(); isolation = TransactionIsolation.fromOrdinal(in.readByte()); @@ -165,6 +195,7 @@ public int getSize() { primaryNodes = U.readCollection(in); state = TransactionState.fromOrdinal(in.readByte()); size = in.readInt(); + startTime = protoVer >= V2 ? in.readLong() : 0L; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxSortOrder.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxSortOrder.java index 382cf9152207b..9a18882c24958 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxSortOrder.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxSortOrder.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.visor.tx; +import org.apache.ignite.internal.commandline.CommandHandler; import org.jetbrains.annotations.Nullable; /** @@ -25,7 +26,9 @@ public enum VisorTxSortOrder { /** Sort by duration. */ DURATION, /** Sort by size. */ - SIZE; + SIZE, + /** Sort by startTime */ + START_TIME; /** Enumerated values. */ private static final VisorTxSortOrder[] VALS = values(); @@ -50,6 +53,9 @@ public static VisorTxSortOrder fromString(String name) { if (SIZE.toString().equals(name)) return SIZE; + if (CommandHandler.CMD_TX_ORDER_START_TIME.equals(name)) + return START_TIME; + throw new IllegalArgumentException("Sort order is unknown: " + name); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java index 579abbe9261a9..25a69d1d67e06 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java @@ -194,7 +194,7 @@ private VisorTxJob(VisorTxTaskArg arg, boolean debug) { if (arg.getMinSize() != null && size < arg.getMinSize()) continue; - infos.add(new VisorTxInfo(locTx.xid(), duration, locTx.isolation(), locTx.concurrency(), + infos.add(new VisorTxInfo(locTx.xid(), locTx.startTime(), duration, locTx.isolation(), locTx.concurrency(), locTx.timeout(), locTx.label(), mappings, locTx.state(), size)); if (arg.getOperation() == VisorTxOperation.KILL) @@ -218,6 +218,11 @@ private VisorTxJob(VisorTxTaskArg arg, boolean debug) { break; + case START_TIME: + comp = TxStartTimeComparator.INSTANCE; + + break; + default: } } @@ -228,6 +233,19 @@ private VisorTxJob(VisorTxTaskArg arg, boolean debug) { } } + /** + * + */ + private static class TxStartTimeComparator implements Comparator { + /** Instance. */ + public static final TxStartTimeComparator INSTANCE = new TxStartTimeComparator(); + + /** {@inheritDoc} */ + @Override public int compare(VisorTxInfo o1, VisorTxInfo o2) { + return Long.compare(o2.getStartTime(), o1.getStartTime()); + } + } + /** * */ diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java index 3835ef4f4d1ad..467e04620521e 100644 --- a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java @@ -456,16 +456,13 @@ else if (entry.getKey().equals(node2)) { validate(h, map -> { VisorTxTaskResult res = map.get(grid(0).localNode()); - for (VisorTxInfo info:res.getInfos()){ + for (VisorTxInfo info : res.getInfos()) assertNull(info.getLabel()); - } - }, "--tx", "label", "null"); - // test check minSize - int minSize=10; + int minSize = 10; validate(h, map -> { VisorTxTaskResult res = map.get(grid(0).localNode()); @@ -482,7 +479,7 @@ else if (entry.getKey().equals(node2)) { validate(h, map -> { VisorTxTaskResult res = map.get(grid(0).localNode()); - assertTrue(res.getInfos().get(0).getSize() >= res.getInfos().get(1).getSize()); + assertTrue(res.getInfos().get(0).getSize() >= res.getInfos().get(1).getSize()); }, "--tx", "order", "SIZE"); @@ -490,10 +487,18 @@ else if (entry.getKey().equals(node2)) { validate(h, map -> { VisorTxTaskResult res = map.get(grid(0).localNode()); - assertTrue(res.getInfos().get(0).getDuration() >= res.getInfos().get(1).getDuration()); + assertTrue(res.getInfos().get(0).getDuration() >= res.getInfos().get(1).getDuration()); }, "--tx", "order", "DURATION"); + // test order by start_time. + validate(h, map -> { + VisorTxTaskResult res = map.get(grid(0).localNode()); + + for (int i = res.getInfos().size() - 1; i > 1; i--) + assertTrue(res.getInfos().get(i - 1).getStartTime() >= res.getInfos().get(i).getStartTime()); + }, "--tx", "order", CommandHandler.CMD_TX_ORDER_START_TIME); + // Trigger topology change and test connection. IgniteInternalFuture startFut = multithreadedAsync(() -> { try { @@ -846,16 +851,16 @@ private void validate(CommandHandler h, IgniteInClosure generate(int from, int cnt) { Map map = new TreeMap<>(); - for (int i = 0; i < cnt; i++ ) + for (int i = 0; i < cnt; i++) map.put(i + from, i + from); return map; } /** - * Test execution of --wal print command. + * Test execution of --wal print command. * - * @throws Exception if failed. + * @throws Exception if failed. */ public void testUnusedWalPrint() throws Exception { Ignite ignite = startGrids(2); @@ -864,14 +869,14 @@ public void testUnusedWalPrint() throws Exception { List nodes = new ArrayList<>(2); - for (ClusterNode node: ignite.cluster().forServers().nodes()) + for (ClusterNode node : ignite.cluster().forServers().nodes()) nodes.add(node.consistentId().toString()); injectTestSystemOut(); assertEquals(EXIT_CODE_OK, execute("--wal", "print")); - for(String id: nodes) + for (String id : nodes) assertTrue(testOut.toString().contains(id)); assertTrue(!testOut.toString().contains("error")); @@ -886,9 +891,9 @@ public void testUnusedWalPrint() throws Exception { } /** - * Test execution of --wal delete command. + * Test execution of --wal delete command. * - * @throws Exception if failed. + * @throws Exception if failed. */ public void testUnusedWalDelete() throws Exception { Ignite ignite = startGrids(2); @@ -897,14 +902,14 @@ public void testUnusedWalDelete() throws Exception { List nodes = new ArrayList<>(2); - for (ClusterNode node: ignite.cluster().forServers().nodes()) + for (ClusterNode node : ignite.cluster().forServers().nodes()) nodes.add(node.consistentId().toString()); injectTestSystemOut(); assertEquals(EXIT_CODE_OK, execute("--wal", "delete")); - for(String id: nodes) + for (String id : nodes) assertTrue(testOut.toString().contains(id)); assertTrue(!testOut.toString().contains("error")); @@ -919,11 +924,11 @@ public void testUnusedWalDelete() throws Exception { } /** - * * @param lockLatch Lock latch. * @param unlockLatch Unlock latch. */ - private IgniteInternalFuture startTransactions(CountDownLatch lockLatch, CountDownLatch unlockLatch) throws Exception { + private IgniteInternalFuture startTransactions(CountDownLatch lockLatch, + CountDownLatch unlockLatch) throws Exception { IgniteEx client = grid("client"); AtomicInteger idx = new AtomicInteger(); From 84582af1df76063ff66be1a2ad2049b09d45e393 Mon Sep 17 00:00:00 2001 From: Andrey Gura Date: Fri, 15 Jun 2018 17:12:09 +0300 Subject: [PATCH 0278/1463] IGNITE-8789 Invoke failure processor in case of message processing error --- .../processors/cache/GridCacheIoManager.java | 16 +- .../igfs/IgfsPrimaryMultiNodeSelfTest.java | 7 + ...ryRelaxedConsistencyMultiNodeSelfTest.java | 7 + .../GridCacheMessageSelfTest.java | 248 ++++++++++++++++++ 4 files changed, 275 insertions(+), 3 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 3182192678cbc..01344211f36e9 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 @@ -34,6 +34,8 @@ import org.apache.ignite.IgniteLogger; import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.failure.FailureContext; +import org.apache.ignite.failure.FailureType; import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; @@ -1057,10 +1059,18 @@ private void processMessage(UUID nodeId, GridCacheMessage msg, IgniteBiInClosure log.debug("Finished processing cache communication message [nodeId=" + nodeId + ", msg=" + msg + ']'); } catch (Throwable e) { - U.error(log, "Failed processing message [senderId=" + nodeId + ", msg=" + msg + ']', e); + try { + U.error(log, "Failed processing message [senderId=" + nodeId + ", msg=" + msg + ']', e); + } + catch (Throwable e0) { + U.error(log, "Failed processing message [senderId=" + nodeId + ", msg=(failed to log message)", e); - if (e instanceof Error) - throw e; + U.error(log, "Failed to log message due to an error: ", e0); + } + + cctx.kernalContext().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e)); + + throw e; } finally { onMessageProcessed(msg); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryMultiNodeSelfTest.java index 1ee6d5ac500e8..f004d4069bbe2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryMultiNodeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryMultiNodeSelfTest.java @@ -25,4 +25,11 @@ public class IgfsPrimaryMultiNodeSelfTest extends IgfsPrimarySelfTest { @Override protected int nodeCount() { return 4; } + + /** + * @throws Exception If failed. + */ + @Override public void testCreateConsistencyMultithreaded() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8823"); + } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedConsistencyMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedConsistencyMultiNodeSelfTest.java index 73a14c3c2470b..d35237cacd0fb 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedConsistencyMultiNodeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedConsistencyMultiNodeSelfTest.java @@ -25,4 +25,11 @@ public class IgfsPrimaryRelaxedConsistencyMultiNodeSelfTest extends IgfsPrimaryR @Override protected int nodeCount() { return 4; } + + @Override + public void testCreateConsistencyMultithreaded() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-8823"); + + super.testCreateConsistencyMultithreaded(); + } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridCacheMessageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridCacheMessageSelfTest.java index 435af8f4b0e9c..587be71d25e2e 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridCacheMessageSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridCacheMessageSelfTest.java @@ -22,10 +22,14 @@ import java.util.Collection; import java.util.UUID; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import org.apache.ignite.Ignite; import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.failure.FailureContext; +import org.apache.ignite.failure.FailureHandler; import org.apache.ignite.internal.GridDirectCollection; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.managers.communication.GridIoManager; @@ -33,6 +37,7 @@ import org.apache.ignite.internal.managers.communication.GridIoPolicy; import org.apache.ignite.internal.managers.communication.GridMessageListener; import org.apache.ignite.internal.processors.cache.GridCacheMessage; +import org.apache.ignite.internal.util.typedef.CI2; import org.apache.ignite.internal.util.typedef.CO; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType; @@ -56,6 +61,9 @@ public class GridCacheMessageSelfTest extends GridCommonAbstractTest { /** Sample count. */ private static final int SAMPLE_CNT = 1; + /** Latch on failure processor. */ + private static CountDownLatch failureLatch; + /** */ public static final String TEST_BODY = "Test body"; @@ -86,6 +94,12 @@ public class GridCacheMessageSelfTest extends GridCommonAbstractTest { return new TestMessage2(); } }); + + GridIoMessageFactory.registerCustom(TestBadMessage.DIRECT_TYPE, new CO() { + @Override public Message apply() { + return new TestBadMessage(); + } + }); } /** {@inheritDoc} */ @@ -100,6 +114,8 @@ public class GridCacheMessageSelfTest extends GridCommonAbstractTest { cfg.setIncludeEventTypes((int[])null); + cfg.setFailureHandler(new TestFailureHandler()); + CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME); ccfg.setCacheMode(CacheMode.PARTITIONED); @@ -113,6 +129,11 @@ public class GridCacheMessageSelfTest extends GridCommonAbstractTest { return cfg; } + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + failureLatch = new CountDownLatch(1); + } + /** * @throws Exception If failed. */ @@ -127,6 +148,42 @@ public void testSendMessage() throws Exception { } } + /** + * @throws Exception If failed. + */ + public void testSendBadMessage() throws Exception { + try { + startGrids(2); + + Ignite ignite0 = grid(0); + Ignite ignite1 = grid(1); + + ((IgniteKernal)ignite0).context().cache().context().io().addCacheHandler( + 0, TestBadMessage.class, new CI2() { + @Override public void apply(UUID nodeId, GridCacheMessage msg) { + throw new RuntimeException("Test bad message exception"); + } + }); + + ((IgniteKernal)ignite1).context().cache().context().io().addCacheHandler( + 0, TestBadMessage.class, new CI2() { + @Override public void apply(UUID nodeId, GridCacheMessage msg) { + throw new RuntimeException("Test bad message exception"); + } + }); + + ((IgniteKernal)ignite0).context().cache().context().io().send( + ((IgniteKernal)ignite1).localNode().id(), new TestBadMessage(), (byte)2); + + boolean res = failureLatch.await(5, TimeUnit.SECONDS); + + assertTrue(res); + } + finally { + stopAllGrids(); + } + } + /** * @throws Exception If failed. */ @@ -506,6 +563,185 @@ public int id() { return 7; } + /** {@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.writeUuid("nodeId", nodeId)) + return false; + + writer.incrementState(); + + case 4: + if (!writer.writeInt("id", id)) + return false; + + writer.incrementState(); + + case 5: + if (!writer.writeString("body", body)) + return false; + + writer.incrementState(); + + case 6: + if (!writer.writeMessage("msg", 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: + nodeId = reader.readUuid("nodeId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 4: + id = reader.readInt("id"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 5: + body = reader.readString("body"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 6: + msg = reader.readMessage("msg"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return true; + } + } + + /** + * Test message class. + */ + static class TestBadMessage extends GridCacheMessage { + /** */ + public static final short DIRECT_TYPE = 204; + + /** Node id. */ + private UUID nodeId; + + /** Integer field. */ + private int id; + + /** Body. */ + private String body; + + /** */ + private Message msg; + + /** + * @param mes Message. + */ + public void init(Message mes, UUID nodeId, int id, String body) { + this.nodeId = nodeId; + this.id = id; + this.msg = mes; + this.body = body; + } + + /** {@inheritDoc} */ + @Override public int handlerId() { + return 0; + } + + /** {@inheritDoc} */ + @Override public boolean cacheGroupMessage() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean addDeploymentInfo() { + return false; + } + + /** + * @return Body. + */ + public String body() { + return body; + } + + /** + * @return Message. + */ + public Message message() { + return msg; + } + + /** + * @return Node id. + */ + public UUID nodeId() { + return nodeId; + } + + /** + * @return Id. + */ + public int id() { + return id; + } + + /** {@inheritDoc} */ + @Override public short directType() { + return DIRECT_TYPE; + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 7; + } + + /** {@inheritDoc} */ + @Override public String toString() { + throw new RuntimeException("Exception while log message"); + } + /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { writer.setBuffer(buf); @@ -598,4 +834,16 @@ public int id() { return true; } } + + /** + * + */ + private static class TestFailureHandler implements FailureHandler { + /** {@inheritDoc} */ + @Override public boolean onFailure(Ignite ignite, FailureContext failureCtx) { + failureLatch.countDown(); + + return false; + } + } } From e4f6a4fa8f65d7cea2a67e980c4d408591896eb8 Mon Sep 17 00:00:00 2001 From: Ivan Daschinskiy Date: Mon, 18 Jun 2018 20:44:00 +0300 Subject: [PATCH 0279/1463] IGNITE-8755 Throw a correct error message when client optimized marshaller is trying to serialize too large object (cherry picked from commit 61b3897) --- .../util/io/GridUnsafeDataOutput.java | 36 ++++++++- .../optimized/OptimizedMarshallerTest.java | 80 +++++++++++++++++++ 2 files changed, 114 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/io/GridUnsafeDataOutput.java b/modules/core/src/main/java/org/apache/ignite/internal/util/io/GridUnsafeDataOutput.java index c45b8fdd9fe6c..ad94889e52498 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/io/GridUnsafeDataOutput.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/io/GridUnsafeDataOutput.java @@ -120,7 +120,12 @@ public void bytes(byte[] bytes, int off) { /** * @param size Size. */ - private void requestFreeSize(int size) { + private void requestFreeSize(int size) throws IOException { + // If arithmetic overflow occurs, off + size should be less than size. + if (off + size < size) + throw new IOException("Failed to allocate required memory (arithmetic overflow detected) " + + "[length=" + size + ", offset=" + off + ']'); + size = off + size; maxOff = Math.max(maxOff, size); @@ -128,7 +133,7 @@ private void requestFreeSize(int size) { long now = U.currentTimeMillis(); if (size > bytes.length) { - byte[] newBytes = new byte[size << 1]; // Grow. + byte[] newBytes = new byte[Math.max(size << 1, size)]; // Grow. System.arraycopy(bytes, 0, newBytes, 0, off); @@ -185,6 +190,8 @@ private void onWrite(int size) throws IOException { int bytesToCp = arr.length << 3; + checkArrayAllocationOverflow(bytesToCp, arr.length, "double"); + requestFreeSize(bytesToCp); if (BIG_ENDIAN) { @@ -217,6 +224,8 @@ private void onWrite(int size) throws IOException { int bytesToCp = arr.length << 1; + checkArrayAllocationOverflow(bytesToCp, arr.length, "char"); + requestFreeSize(bytesToCp); if (BIG_ENDIAN) { @@ -240,6 +249,8 @@ private void onWrite(int size) throws IOException { int bytesToCp = arr.length << 3; + checkArrayAllocationOverflow(bytesToCp, arr.length, "long"); + requestFreeSize(bytesToCp); if (BIG_ENDIAN) { @@ -263,6 +274,8 @@ private void onWrite(int size) throws IOException { int bytesToCp = arr.length << 2; + checkArrayAllocationOverflow(bytesToCp, arr.length, "float"); + requestFreeSize(bytesToCp); if (BIG_ENDIAN) { @@ -304,6 +317,8 @@ private void onWrite(int size) throws IOException { int bytesToCp = arr.length << 1; + checkArrayAllocationOverflow(bytesToCp, arr.length, "short"); + requestFreeSize(bytesToCp); if (BIG_ENDIAN) { @@ -327,6 +342,8 @@ private void onWrite(int size) throws IOException { int bytesToCp = arr.length << 2; + checkArrayAllocationOverflow(bytesToCp, arr.length, "int"); + requestFreeSize(bytesToCp); if (BIG_ENDIAN) { @@ -471,6 +488,21 @@ private void onWrite(int size) throws IOException { writeUTF(s, utfLength(s)); } + /** + * Check for possible arithmetic overflow when trying to serialize a humongous array. + * + * @param bytesToAlloc Bytes to allocate. + * @param arrLen Array length. + * @param type Type of an array. + * @throws IOException If oveflow presents and data corruption can occur. + */ + private void checkArrayAllocationOverflow(int bytesToAlloc, int arrLen, String type) throws IOException { + // If arithmetic overflow occurs, bytesToAlloc should be less than arrLen. + if (bytesToAlloc < arrLen) + throw new IOException("Failed to allocate required memory for " + type + " array " + + "(arithmetic overflow detected) [length=" + arrLen + ']'); + } + /** * * Returns the length in bytes of the UTF encoding of the given string. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/marshaller/optimized/OptimizedMarshallerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/marshaller/optimized/OptimizedMarshallerTest.java index 14b49a4f8b2ae..79496ae1163c0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/marshaller/optimized/OptimizedMarshallerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/marshaller/optimized/OptimizedMarshallerTest.java @@ -28,6 +28,7 @@ import java.net.InetSocketAddress; import java.util.Arrays; import java.util.Collection; +import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentMap; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCheckedException; @@ -41,6 +42,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinderAdapter; 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.jetbrains.annotations.Nullable; @@ -386,6 +388,84 @@ private void checkPerformance(int cnt, int tries) throws Exception { info(">>> Finished performance check <<<"); } + /** + * Tests checks for arithmetic overflow when trying to serialize huge object. + * WARNING! Requires a lot of heap space. Should not be run on CI. + */ + public void _testAllocationOverflow() { + allocationOverflowCheck(() -> marshaller().marshal(new HugeObject())); + + allocationOverflowCheck(() -> { + marshaller().marshal(new short[1<<30]); + marshaller().marshal(new short[1<<30]); + return null; + }); + + allocationOverflowCheck(() -> { + marshaller().marshal(new char[1<<30]); + marshaller().marshal(new char[1<<30]); + return null; + }); + + allocationOverflowCheck(() -> { + marshaller().marshal(new int[1<<30]); + marshaller().marshal(new int[1<<30]); + return null; + }); + + allocationOverflowCheck(() -> { + marshaller().marshal(new int[1<<30]); + marshaller().marshal(new int[1<<30]); + return null; + }); + + allocationOverflowCheck(() -> { + marshaller().marshal(new float[1<<29]); + marshaller().marshal(new float[1<<29]); + return null; + }); + + allocationOverflowCheck(() -> { + marshaller().marshal(new long[1<<29]); + marshaller().marshal(new long[1<<29]); + return null; + }); + + allocationOverflowCheck(() -> { + marshaller().marshal(new double[1<<29]); + marshaller().marshal(new double[1<<29]); + return null; + }); + } + + /** + * Asserts that {@link IOException} will be thrown. + * + * @param call Callable that cause allocation overflow. + */ + private void allocationOverflowCheck(Callable call) { + GridTestUtils.assertThrowsAnyCause(log, call, IOException.class, "Impossible to allocate required memory"); + } + + /** + * + */ + public static class HugeObject implements Externalizable { + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + out.write(new byte[1 << 31 - 2]); + out.write(new byte[1 << 31 - 2]); + out.write(new byte[1 << 31 - 2]); + out.write(new byte[1 << 31 - 2]); + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + + } + } + /** * Some non-serializable class. */ From 8779e1180aa20e67de0afc55c920cd713e5bef52 Mon Sep 17 00:00:00 2001 From: Ilya Lantukh Date: Thu, 26 Apr 2018 18:31:47 +0300 Subject: [PATCH 0280/1463] GG-13652 PITR : handle situation when WAL got temporarily disabled for rebalancing Signed-off-by: EdShangGG (cherry picked from commit 92dc88d) Signed-off-by: EdShangGG --- .../cache/persistence/GridCacheDatabaseSharedManager.java | 6 +++--- 1 file changed, 3 insertions(+), 3 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 4a44f23fe693a..e2fe57260e35e 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 @@ -4332,7 +4332,7 @@ private void fillWalDisabledGroups() { * @param grpId Group ID. * @return Key. */ - private static String walGroupIdToKey(int grpId, boolean local) { + public static String walGroupIdToKey(int grpId, boolean local) { if (local) return WAL_LOCAL_KEY_PREFIX + grpId; else @@ -4354,9 +4354,9 @@ private static String checkpointInapplicableCpAndGroupIdToKey(long cpTs, int grp * Convert WAL state key to cache group ID. * * @param key Key. - * @return Group ID. + * @return Group ID or {@code null} if key is not WAL state key. */ - private static T2 walKeyToGroupIdAndLocalFlag(String key) { + @Nullable public static T2 walKeyToGroupIdAndLocalFlag(String key) { if (key.startsWith(WAL_LOCAL_KEY_PREFIX)) return new T2<>(Integer.parseInt(key.substring(WAL_LOCAL_KEY_PREFIX.length())), true); else if (key.startsWith(WAL_GLOBAL_KEY_PREFIX)) From c89a60089026df0cfa0fa83376986e2b278ade16 Mon Sep 17 00:00:00 2001 From: Ivan Rakov Date: Tue, 19 Jun 2018 12:38:24 +0300 Subject: [PATCH 0281/1463] IGNITE-8749 Exception for no space left situation should be propagated to FailureHandler - rollback --- .../wal/FileWriteAheadLogManager.java | 167 ++++++------ .../FsyncModeFileWriteAheadLogManager.java | 215 +++++++-------- .../ignite/failure/TestFailureHandler.java | 19 -- .../wal/IgniteWalFormatFileFailoverTest.java | 258 ------------------ .../testsuites/IgnitePdsTestSuite2.java | 3 - 5 files changed, 179 insertions(+), 483 deletions(-) delete mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFormatFileFailoverTest.java 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 09a08c9d4bce5..9b39987edf42a 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 @@ -605,43 +605,48 @@ private void checkWalConfiguration() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public void resumeLogging(WALPointer lastPtr) throws IgniteCheckedException { - assert currHnd == null; - assert lastPtr == null || lastPtr instanceof FileWALPointer; + try { + assert currHnd == null; + assert lastPtr == null || lastPtr instanceof FileWALPointer; - FileWALPointer filePtr = (FileWALPointer)lastPtr; + FileWALPointer filePtr = (FileWALPointer)lastPtr; walWriter = new WALWriter(log); if (!mmap) new IgniteThread(walWriter).start(); - currHnd = restoreWriteHandle(filePtr); + currHnd = restoreWriteHandle(filePtr); - // For new handle write serializer version to it. - if (filePtr == null) - currHnd.writeHeader(); + // For new handle write serializer version to it. + if (filePtr == null) + currHnd.writeHeader(); - if (currHnd.serializer.version() != serializer.version()) { - if (log.isInfoEnabled()) - log.info("Record serializer version change detected, will start logging with a new WAL record " + - "serializer to a new WAL segment [curFile=" + currHnd + ", newVer=" + serializer.version() + - ", oldVer=" + currHnd.serializer.version() + ']'); + if (currHnd.serializer.version() != serializer.version()) { + if (log.isInfoEnabled()) + log.info("Record serializer version change detected, will start logging with a new WAL record " + + "serializer to a new WAL segment [curFile=" + currHnd + ", newVer=" + serializer.version() + + ", oldVer=" + currHnd.serializer.version() + ']'); - rollOver(currHnd); - } + rollOver(currHnd); + } - currHnd.resume = false; + currHnd.resume = false; - if (mode == WALMode.BACKGROUND) { - backgroundFlushSchedule = cctx.time().schedule(new Runnable() { - @Override public void run() { - doFlush(); - } - }, flushFreq, flushFreq); - } + if (mode == WALMode.BACKGROUND) { + backgroundFlushSchedule = cctx.time().schedule(new Runnable() { + @Override public void run() { + doFlush(); + } + }, flushFreq, flushFreq); + } - if (walAutoArchiveAfterInactivity > 0) - scheduleNextInactivityPeriodElapsedCheck(); + if (walAutoArchiveAfterInactivity > 0) + scheduleNextInactivityPeriodElapsedCheck(); + } + catch (StorageException e) { + throw new IgniteCheckedException(e); + } } /** @@ -1126,9 +1131,9 @@ private FileWriteHandle rollOver(FileWriteHandle cur) throws StorageException, I /** * @param lastReadPtr Last read WAL file pointer. * @return Initialized file write handle. - * @throws StorageException If failed to initialize WAL write handle. + * @throws IgniteCheckedException If failed to initialize WAL write handle. */ - private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws StorageException { + private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws IgniteCheckedException { long absIdx = lastReadPtr == null ? 0 : lastReadPtr.index(); @Nullable FileArchiver archiver0 = archiver; @@ -1170,9 +1175,14 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws St SegmentedRingByteBuffer rbuf; if (mmap) { - MappedByteBuffer buf = fileIO.map((int)maxWalSegmentSize); + try { + MappedByteBuffer buf = fileIO.map((int)maxWalSegmentSize); - rbuf = new SegmentedRingByteBuffer(buf, metrics); + rbuf = new SegmentedRingByteBuffer(buf, metrics); + } + catch (IOException e) { + throw new IgniteCheckedException(e); + } } else rbuf = new SegmentedRingByteBuffer(dsCfg.getWalBufferSize(), maxWalSegmentSize, DIRECT, metrics); @@ -1196,21 +1206,13 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws St return hnd; } catch (IgniteCheckedException | IOException e) { - try { - fileIO.close(); - } - catch (IOException suppressed) { - e.addSuppressed(suppressed); - } - - if (e instanceof StorageException) - throw (StorageException) e; + fileIO.close(); - throw e instanceof IOException ? (IOException) e : new IOException(e); + throw e; } } catch (IOException e) { - throw new StorageException("Failed to restore WAL write handle: " + curFile.getAbsolutePath(), e); + throw new IgniteCheckedException("Failed to restore WAL write handle: " + curFile.getAbsolutePath(), e); } } @@ -1221,8 +1223,9 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws St * @param cur Current file write handle released by WAL writer * @return Initialized file handle. * @throws StorageException If IO exception occurred. + * @throws IgniteCheckedException If failed. */ - private FileWriteHandle initNextWriteHandle(FileWriteHandle cur) throws StorageException { + private FileWriteHandle initNextWriteHandle(FileWriteHandle cur) throws StorageException, IgniteCheckedException { try { File nextFile = pollNextFile(cur.idx); @@ -1307,10 +1310,8 @@ private FileWriteHandle initNextWriteHandle(FileWriteHandle cur) throws StorageE /** * Deletes temp files, creates and prepares new; Creates first segment if necessary - * - * @throws StorageException If failed. */ - private void checkOrPrepareFiles() throws StorageException { + private void checkOrPrepareFiles() throws IgniteCheckedException { // Clean temp files. { File[] tmpFiles = walWorkDir.listFiles(WAL_SEGMENT_TEMP_FILE_FILTER); @@ -1320,7 +1321,7 @@ private void checkOrPrepareFiles() throws StorageException { boolean deleted = tmp.delete(); if (!deleted) - throw new StorageException("Failed to delete previously created temp file " + + throw new IgniteCheckedException("Failed to delete previously created temp file " + "(make sure Ignite process has enough rights): " + tmp.getAbsolutePath()); } } @@ -1330,7 +1331,7 @@ private void checkOrPrepareFiles() throws StorageException { if(isArchiverEnabled()) if (allFiles.length != 0 && allFiles.length > dsCfg.getWalSegments()) - throw new StorageException("Failed to initialize wal (work directory contains " + + throw new IgniteCheckedException("Failed to initialize wal (work directory contains " + "incorrect number of segments) [cur=" + allFiles.length + ", expected=" + dsCfg.getWalSegments() + ']'); // Allocate the first segment synchronously. All other segments will be allocated by archiver in background. @@ -1370,9 +1371,9 @@ public void cleanupWalDirectories() throws IgniteCheckedException { * Clears whole the file, fills with zeros for Default mode. * * @param file File to format. - * @throws StorageException if formatting failed + * @throws IgniteCheckedException if formatting failed */ - private void formatFile(File file) throws StorageException { + private void formatFile(File file) throws IgniteCheckedException { formatFile(file, dsCfg.getWalSegmentSize()); } @@ -1381,9 +1382,9 @@ private void formatFile(File file) throws StorageException { * * @param file File to format. * @param bytesCntToFormat Count of first bytes to format. - * @throws StorageException if formatting failed + * @throws IgniteCheckedException if formatting failed */ - private void formatFile(File file, int bytesCntToFormat) throws StorageException { + private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Formatting file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']'); @@ -1395,7 +1396,7 @@ private void formatFile(File file, int bytesCntToFormat) throws StorageException int toWrite = Math.min(FILL_BUF.length, left); if (fileIO.write(FILL_BUF, 0, toWrite) < toWrite) { - final StorageException ex = new StorageException("Failed to extend WAL segment file: " + + final IgniteCheckedException ex = new IgniteCheckedException("Failed to extend WAL segment file: " + file.getName() + ". Probably disk is too busy, please check your device."); if (failureProcessor != null) @@ -1413,7 +1414,7 @@ private void formatFile(File file, int bytesCntToFormat) throws StorageException fileIO.clear(); } catch (IOException e) { - throw new StorageException("Failed to format WAL segment file: " + file.getAbsolutePath(), e); + throw new IgniteCheckedException("Failed to format WAL segment file: " + file.getAbsolutePath(), e); } } @@ -1421,9 +1422,9 @@ private void formatFile(File file, int bytesCntToFormat) throws StorageException * Creates a file atomically with temp file. * * @param file File to create. - * @throws StorageException If failed. + * @throws IgniteCheckedException If failed. */ - private void createFile(File file) throws StorageException { + private void createFile(File file) throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Creating new file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']'); @@ -1435,7 +1436,7 @@ private void createFile(File file) throws StorageException { Files.move(tmp.toPath(), file.toPath()); } catch (IOException e) { - throw new StorageException("Failed to move temp file to a regular WAL segment file: " + + throw new IgniteCheckedException("Failed to move temp file to a regular WAL segment file: " + file.getAbsolutePath(), e); } @@ -1448,9 +1449,9 @@ private void createFile(File file) throws StorageException { * * @param curIdx Current absolute WAL segment index. * @return File ready for use as new WAL segment. - * @throws StorageException If exception occurred in the archiver thread. + * @throws IgniteCheckedException If failed. */ - private File pollNextFile(long curIdx) throws StorageException { + private File pollNextFile(long curIdx) throws IgniteCheckedException { FileArchiver archiver0 = archiver; if (archiver0 == null) { @@ -1526,7 +1527,7 @@ public long maxWalSegmentSize() { */ private class FileArchiver extends GridWorker { /** Exception which occurred during initial creation of files or during archiving WAL segment */ - private StorageException cleanErr; + private IgniteCheckedException cleanErr; /** * Absolute current segment index WAL Manager writes to. Guarded by this. Incremented during @@ -1598,17 +1599,15 @@ private synchronized boolean locked(long absIdx) { try { allocateRemainingFiles(); } - catch (StorageException e) { + catch (IgniteCheckedException e) { synchronized (this) { // Stop the thread and report to starter. cleanErr = e; notifyAll(); - } - - cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, e)); - return; + return; + } } Throwable err = null; @@ -1692,9 +1691,9 @@ private void changeLastArchivedIndexAndNotifyWaiters(long idx) { * * @param curIdx Current absolute index that we want to increment. * @return Next index (curWalSegmIdx+1) when it is ready to be written. - * @throws StorageException If exception occurred in the archiver thread. + * @throws IgniteCheckedException If failed (if interrupted or if exception occurred in the archiver thread). */ - private long nextAbsoluteSegmentIndex(long curIdx) throws StorageException { + private long nextAbsoluteSegmentIndex(long curIdx) throws IgniteCheckedException { synchronized (this) { if (cleanErr != null) throw cleanErr; @@ -1709,9 +1708,6 @@ private long nextAbsoluteSegmentIndex(long curIdx) throws StorageException { while (curAbsWalIdx - lastAbsArchivedIdx > dsCfg.getWalSegments() && cleanErr == null) { try { wait(); - - if (cleanErr != null) - throw cleanErr; } catch (InterruptedException ignore) { interrupted.set(true); @@ -1719,12 +1715,9 @@ private long nextAbsoluteSegmentIndex(long curIdx) throws StorageException { } // Wait for formatter so that we do not open an empty file in DEFAULT mode. - while (curAbsWalIdx % dsCfg.getWalSegments() > formatted && cleanErr == null) + while (curAbsWalIdx % dsCfg.getWalSegments() > formatted) try { wait(); - - if (cleanErr != null) - throw cleanErr; } catch (InterruptedException ignore) { interrupted.set(true); @@ -1796,7 +1789,7 @@ private void releaseWorkSegment(long absIdx) { * * @param absIdx Absolute index to archive. */ - private SegmentArchiveResult archiveSegment(long absIdx) throws StorageException { + private SegmentArchiveResult archiveSegment(long absIdx) throws IgniteCheckedException { long segIdx = absIdx % dsCfg.getWalSegments(); File origFile = new File(walWorkDir, FileDescriptor.fileName(segIdx)); @@ -1825,7 +1818,7 @@ private SegmentArchiveResult archiveSegment(long absIdx) throws StorageException } } catch (IOException e) { - throw new StorageException("Failed to archive WAL segment [" + + throw new IgniteCheckedException("Failed to archive WAL segment [" + "srcFile=" + origFile.getAbsolutePath() + ", dstFile=" + dstTmpFile.getAbsolutePath() + ']', e); } @@ -1848,7 +1841,7 @@ private boolean checkStop() { * Background creation of all segments except first. First segment was created in main thread by {@link * FileWriteAheadLogManager#checkOrPrepareFiles()} */ - private void allocateRemainingFiles() throws StorageException { + private void allocateRemainingFiles() throws IgniteCheckedException { checkFiles( 1, true, @@ -2242,23 +2235,23 @@ private void shutdown() throws IgniteInterruptedCheckedException { * @param startWith Start with. * @param create Flag create file. * @param p Predicate Exit condition. - * @throws StorageException if validation or create file fail. + * @throws IgniteCheckedException if validation or create file fail. */ private void checkFiles( int startWith, boolean create, @Nullable IgnitePredicate p, @Nullable IgniteInClosure completionCallback - ) throws StorageException { + ) throws IgniteCheckedException { for (int i = startWith; i < dsCfg.getWalSegments() && (p == null || p.apply(i)); i++) { File checkFile = new File(walWorkDir, FileDescriptor.fileName(i)); if (checkFile.exists()) { if (checkFile.isDirectory()) - throw new StorageException("Failed to initialize WAL log segment (a directory with " + + throw new IgniteCheckedException("Failed to initialize WAL log segment (a directory with " + "the same name already exists): " + checkFile.getAbsolutePath()); else if (checkFile.length() != dsCfg.getWalSegmentSize() && mode == WALMode.FSYNC) - throw new StorageException("Failed to initialize WAL log segment " + + throw new IgniteCheckedException("Failed to initialize WAL log segment " + "(WAL segment size change is not supported in 'DEFAULT' WAL mode) " + "[filePath=" + checkFile.getAbsolutePath() + ", fileSize=" + checkFile.length() + @@ -2658,8 +2651,9 @@ public void writeHeader() { * Flush or wait for concurrent flush completion. * * @param ptr Pointer. + * @throws IgniteCheckedException If failed. */ - 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) @@ -2671,8 +2665,10 @@ private void flushOrWait(FileWALPointer ptr) { /** * @param ptr Pointer. + * @throws IgniteCheckedException If failed. + * @throws StorageException If failed. */ - private void flush(FileWALPointer ptr) { + private void flush(FileWALPointer ptr) throws IgniteCheckedException, StorageException { if (ptr == null) { // Unconditional flush. walWriter.flushAll(); @@ -2888,7 +2884,7 @@ private boolean close(boolean rollOver) throws IgniteCheckedException, StorageEx } } catch (IOException e) { - throw new StorageException(e); + throw new IgniteCheckedException(e); } if (log.isDebugEnabled()) @@ -3369,29 +3365,28 @@ 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); } /** * @param expPos Expected position. */ - @SuppressWarnings("ForLoopReplaceableByForEach") - void flushBuffer(long expPos) { + void flushBuffer(long expPos) throws StorageException, IgniteCheckedException { if (mmap) return; 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 6f676fcf43284..49fbc73b9a48c 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 @@ -487,32 +487,37 @@ private void checkWalConfiguration() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public void resumeLogging(WALPointer lastPtr) throws IgniteCheckedException { - assert currentHnd == null; - assert lastPtr == null || lastPtr instanceof FileWALPointer; + try { + assert currentHnd == null; + assert lastPtr == null || lastPtr instanceof FileWALPointer; - FileWALPointer filePtr = (FileWALPointer)lastPtr; + FileWALPointer filePtr = (FileWALPointer)lastPtr; - currentHnd = restoreWriteHandle(filePtr); + currentHnd = restoreWriteHandle(filePtr); - if (currentHnd.serializer.version() != serializer.version()) { - if (log.isInfoEnabled()) - log.info("Record serializer version change detected, will start logging with a new WAL record " + - "serializer to a new WAL segment [curFile=" + currentHnd + ", newVer=" + serializer.version() + - ", oldVer=" + currentHnd.serializer.version() + ']'); + if (currentHnd.serializer.version() != serializer.version()) { + if (log.isInfoEnabled()) + log.info("Record serializer version change detected, will start logging with a new WAL record " + + "serializer to a new WAL segment [curFile=" + currentHnd + ", newVer=" + serializer.version() + + ", oldVer=" + currentHnd.serializer.version() + ']'); - rollOver(currentHnd); - } + rollOver(currentHnd); + } - if (mode == WALMode.BACKGROUND) { - backgroundFlushSchedule = cctx.time().schedule(new Runnable() { - @Override public void run() { - doFlush(); - } - }, flushFreq, flushFreq); - } + if (mode == WALMode.BACKGROUND) { + backgroundFlushSchedule = cctx.time().schedule(new Runnable() { + @Override public void run() { + doFlush(); + } + }, flushFreq, flushFreq); + } - if (walAutoArchiveAfterInactivity > 0) - scheduleNextInactivityPeriodElapsedCheck(); + if (walAutoArchiveAfterInactivity > 0) + scheduleNextInactivityPeriodElapsedCheck(); + } + catch (StorageException e) { + throw new IgniteCheckedException(e); + } } /** @@ -1014,7 +1019,7 @@ private FileWriteHandle currentHandle() { * @param cur Handle that failed to fit the given entry. * @return Handle that will fit the entry. */ - private FileWriteHandle rollOver(FileWriteHandle cur) throws StorageException, IgniteInterruptedCheckedException { + private FileWriteHandle rollOver(FileWriteHandle cur) throws StorageException, IgniteCheckedException { FileWriteHandle hnd = currentHandle(); if (hnd != cur) @@ -1045,9 +1050,9 @@ private FileWriteHandle rollOver(FileWriteHandle cur) throws StorageException, I /** * @param lastReadPtr Last read WAL file pointer. * @return Initialized file write handle. - * @throws StorageException If failed to initialize WAL write handle. + * @throws IgniteCheckedException If failed to initialize WAL write handle. */ - private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws StorageException { + private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws IgniteCheckedException { long absIdx = lastReadPtr == null ? 0 : lastReadPtr.index(); long segNo = absIdx % dsCfg.getWalSegments(); @@ -1095,21 +1100,13 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws St return hnd; } catch (IgniteCheckedException | IOException e) { - try { - fileIO.close(); - } - catch (IOException suppressed) { - e.addSuppressed(suppressed); - } - - if (e instanceof StorageException) - throw (StorageException) e; + fileIO.close(); - throw e instanceof IOException ? (IOException) e : new IOException(e); + throw e; } } catch (IOException e) { - throw new StorageException("Failed to restore WAL write handle: " + curFile.getAbsolutePath(), e); + throw new IgniteCheckedException("Failed to restore WAL write handle: " + curFile.getAbsolutePath(), e); } } @@ -1121,9 +1118,9 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws St * @param curIdx current absolute segment released by WAL writer * @return Initialized file handle. * @throws StorageException If IO exception occurred. - * @throws IgniteInterruptedCheckedException If interrupted. + * @throws IgniteCheckedException If failed. */ - private FileWriteHandle initNextWriteHandle(long curIdx) throws StorageException, IgniteInterruptedCheckedException { + private FileWriteHandle initNextWriteHandle(long curIdx) throws StorageException, IgniteCheckedException { try { File nextFile = pollNextFile(curIdx); @@ -1153,11 +1150,9 @@ private FileWriteHandle initNextWriteHandle(long curIdx) throws StorageException } /** - * Deletes temp files, creates and prepares new; Creates first segment if necessary. - * - * @throws StorageException If failed. + * Deletes temp files, creates and prepares new; Creates first segment if necessary */ - private void checkOrPrepareFiles() throws StorageException { + private void checkOrPrepareFiles() throws IgniteCheckedException { // Clean temp files. { File[] tmpFiles = walWorkDir.listFiles(WAL_SEGMENT_TEMP_FILE_FILTER); @@ -1167,7 +1162,7 @@ private void checkOrPrepareFiles() throws StorageException { boolean deleted = tmp.delete(); if (!deleted) - throw new StorageException("Failed to delete previously created temp file " + + throw new IgniteCheckedException("Failed to delete previously created temp file " + "(make sure Ignite process has enough rights): " + tmp.getAbsolutePath()); } } @@ -1176,7 +1171,7 @@ private void checkOrPrepareFiles() throws StorageException { File[] allFiles = walWorkDir.listFiles(WAL_SEGMENT_FILE_FILTER); if (allFiles.length != 0 && allFiles.length > dsCfg.getWalSegments()) - throw new StorageException("Failed to initialize wal (work directory contains " + + throw new IgniteCheckedException("Failed to initialize wal (work directory contains " + "incorrect number of segments) [cur=" + allFiles.length + ", expected=" + dsCfg.getWalSegments() + ']'); // Allocate the first segment synchronously. All other segments will be allocated by archiver in background. @@ -1193,9 +1188,9 @@ private void checkOrPrepareFiles() throws StorageException { * Clears whole the file, fills with zeros for Default mode. * * @param file File to format. - * @throws StorageException if formatting failed. + * @throws IgniteCheckedException if formatting failed */ - private void formatFile(File file) throws StorageException { + private void formatFile(File file) throws IgniteCheckedException { formatFile(file, dsCfg.getWalSegmentSize()); } @@ -1204,9 +1199,9 @@ private void formatFile(File file) throws StorageException { * * @param file File to format. * @param bytesCntToFormat Count of first bytes to format. - * @throws StorageException If formatting failed. + * @throws IgniteCheckedException if formatting failed */ - private void formatFile(File file, int bytesCntToFormat) throws StorageException { + private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Formatting file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']'); @@ -1228,7 +1223,7 @@ private void formatFile(File file, int bytesCntToFormat) throws StorageException fileIO.clear(); } catch (IOException e) { - throw new StorageException("Failed to format WAL segment file: " + file.getAbsolutePath(), e); + throw new IgniteCheckedException("Failed to format WAL segment file: " + file.getAbsolutePath(), e); } } @@ -1236,9 +1231,9 @@ private void formatFile(File file, int bytesCntToFormat) throws StorageException * Creates a file atomically with temp file. * * @param file File to create. - * @throws StorageException If failed. + * @throws IgniteCheckedException If failed. */ - private void createFile(File file) throws StorageException { + private void createFile(File file) throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Creating new file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']'); @@ -1250,7 +1245,7 @@ private void createFile(File file) throws StorageException { Files.move(tmp.toPath(), file.toPath()); } catch (IOException e) { - throw new StorageException("Failed to move temp file to a regular WAL segment file: " + + throw new IgniteCheckedException("Failed to move temp file to a regular WAL segment file: " + file.getAbsolutePath(), e); } @@ -1264,10 +1259,9 @@ private void createFile(File file) throws StorageException { * * @param curIdx Current absolute WAL segment index. * @return File ready for use as new WAL segment. - * @throws StorageException If exception occurred in the archiver thread. - * @throws IgniteInterruptedCheckedException If interrupted. + * @throws IgniteCheckedException If failed. */ - private File pollNextFile(long curIdx) throws StorageException, IgniteInterruptedCheckedException { + private File pollNextFile(long curIdx) throws IgniteCheckedException { // Signal to archiver that we are done with the segment and it can be archived. long absNextIdx = archiver.nextAbsoluteSegmentIndex(curIdx); @@ -1324,7 +1318,7 @@ private void checkNode() throws StorageException { */ private class FileArchiver extends GridWorker { /** Exception which occurred during initial creation of files or during archiving WAL segment */ - private StorageException cleanException; + private IgniteCheckedException cleanException; /** * Absolute current segment index WAL Manager writes to. Guarded by this. @@ -1432,17 +1426,15 @@ private synchronized void release(long absIdx) { try { allocateRemainingFiles(); } - catch (StorageException e) { + catch (IgniteCheckedException e) { synchronized (this) { // Stop the thread and report to starter. cleanException = e; notifyAll(); - } - cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, e)); - - return; + return; + } } Throwable err = null; @@ -1523,10 +1515,9 @@ private void changeLastArchivedIndexAndWakeupCompressor(long idx) { * * @param curIdx Current absolute index that we want to increment. * @return Next index (curWalSegmIdx+1) when it is ready to be written. - * @throws StorageException If exception occurred in the archiver thread. - * @throws IgniteInterruptedCheckedException If interrupted. + * @throws IgniteCheckedException If failed (if interrupted or if exception occurred in the archiver thread). */ - private long nextAbsoluteSegmentIndex(long curIdx) throws StorageException, IgniteInterruptedCheckedException { + private long nextAbsoluteSegmentIndex(long curIdx) throws IgniteCheckedException { try { synchronized (this) { if (cleanException != null) @@ -1544,16 +1535,10 @@ private long nextAbsoluteSegmentIndex(long curIdx) throws StorageException, Igni while ((curAbsWalIdx - lastAbsArchivedIdx > segments && cleanException == null)) wait(); - if (cleanException != null) - throw cleanException; - // Wait for formatter so that we do not open an empty file in DEFAULT mode. - while (curAbsWalIdx % dsCfg.getWalSegments() > formatted && cleanException == null) + while (curAbsWalIdx % dsCfg.getWalSegments() > formatted) wait(); - if (cleanException != null) - throw cleanException; - return curAbsWalIdx; } } @@ -1679,7 +1664,7 @@ private boolean checkStop() { * Background creation of all segments except first. First segment was created in main thread by * {@link FsyncModeFileWriteAheadLogManager#checkOrPrepareFiles()} */ - private void allocateRemainingFiles() throws StorageException { + private void allocateRemainingFiles() throws IgniteCheckedException { final FileArchiver archiver = this; checkFiles(1, @@ -2044,23 +2029,23 @@ private void shutdown() { * @param startWith Start with. * @param create Flag create file. * @param p Predicate Exit condition. - * @throws StorageException if validation or create file fail. + * @throws IgniteCheckedException if validation or create file fail. */ private void checkFiles( int startWith, boolean create, @Nullable IgnitePredicate p, @Nullable IgniteInClosure completionCallback - ) throws StorageException { + ) throws IgniteCheckedException { for (int i = startWith; i < dsCfg.getWalSegments() && (p == null || (p != null && p.apply(i))); i++) { File checkFile = new File(walWorkDir, FileDescriptor.fileName(i)); if (checkFile.exists()) { if (checkFile.isDirectory()) - throw new StorageException("Failed to initialize WAL log segment (a directory with " + + throw new IgniteCheckedException("Failed to initialize WAL log segment (a directory with " + "the same name already exists): " + checkFile.getAbsolutePath()); else if (checkFile.length() != dsCfg.getWalSegmentSize() && mode == WALMode.FSYNC) - throw new StorageException("Failed to initialize WAL log segment " + + throw new IgniteCheckedException("Failed to initialize WAL log segment " + "(WAL segment size change is not supported):" + checkFile.getAbsolutePath()); } else if (create) @@ -2423,9 +2408,9 @@ private FileWriteHandle( * Write serializer version to current handle. * NOTE: Method mutates {@code fileIO} position, written and lastFsyncPos fields. * - * @throws StorageException If fail to write serializer version. + * @throws IOException If fail to write serializer version. */ - private void writeSerializerVersion() throws StorageException { + public void writeSerializerVersion() throws IOException { try { assert fileIO.position() == 0 : "Serializer version can be written only at the begin of file " + fileIO.position(); @@ -2438,7 +2423,7 @@ private void writeSerializerVersion() throws StorageException { head.set(new FakeRecord(new FileWALPointer(idx, (int)updatedPosition, 0), false)); } catch (IOException e) { - throw new StorageException("Unable to write serializer version for segment " + idx, e); + throw new IOException("Unable to write serializer version for segment " + idx, e); } } @@ -2463,8 +2448,9 @@ private boolean stopped(WALRecord record) { * @param rec Record to be added to record chain as new {@link #head} * @return Pointer or null if roll over to next segment is required or already started by other thread. * @throws StorageException If failed. + * @throws IgniteCheckedException If failed. */ - @Nullable private WALPointer addRecord(WALRecord rec) throws StorageException { + @Nullable private WALPointer addRecord(WALRecord rec) throws StorageException, IgniteCheckedException { assert rec.size() > 0 || rec.getClass() == FakeRecord.class; boolean flushed = false; @@ -2517,9 +2503,9 @@ private long nextPosition(WALRecord rec) { * Flush or wait for concurrent flush completion. * * @param ptr Pointer. - * @throws StorageException If failed. + * @throws IgniteCheckedException If failed. */ - private void flushOrWait(FileWALPointer ptr, boolean stop) throws StorageException { + private void flushOrWait(FileWALPointer ptr, boolean stop) throws IgniteCheckedException { long expWritten; if (ptr != null) { @@ -2563,9 +2549,10 @@ else if (stop) { /** * @param ptr Pointer. * @return {@code true} If the flush really happened. + * @throws IgniteCheckedException If failed. * @throws StorageException If failed. */ - private boolean flush(FileWALPointer ptr, boolean stop) throws StorageException { + private boolean flush(FileWALPointer ptr, boolean stop) throws IgniteCheckedException, StorageException { if (ptr == null) { // Unconditional flush. for (; ; ) { WALRecord expHead = head.get(); @@ -2607,9 +2594,10 @@ private long chainBeginPosition(WALRecord h) { /** * @param expHead Expected head of chain. If head was changed, flush is not performed in this thread + * @throws IgniteCheckedException If failed. * @throws StorageException If failed. */ - private boolean flush(WALRecord expHead, boolean stop) throws StorageException { + private boolean flush(WALRecord expHead, boolean stop) throws StorageException, IgniteCheckedException { if (expHead.previous() == null) { FakeRecord frHead = (FakeRecord)expHead; @@ -2655,8 +2643,7 @@ private boolean flush(WALRecord expHead, boolean stop) throws StorageException { return true; } catch (Throwable e) { - StorageException se = e instanceof StorageException ? (StorageException) e : - new StorageException("Unable to write", new IOException(e)); + StorageException se = new StorageException("Unable to write", new IOException(e)); cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, se)); @@ -2738,9 +2725,8 @@ private FileWALPointer position() { /** * @param ptr Pointer to sync. * @throws StorageException If failed. - * @throws IgniteInterruptedCheckedException If interrupted. */ - private void fsync(FileWALPointer ptr, boolean stop) throws StorageException, IgniteInterruptedCheckedException { + private void fsync(FileWALPointer ptr, boolean stop) throws StorageException, IgniteCheckedException { lock.lock(); try { @@ -2794,9 +2780,10 @@ private void fsync(FileWALPointer ptr, boolean stop) throws StorageException, Ig /** * @return {@code true} If this thread actually closed the segment. + * @throws IgniteCheckedException If failed. * @throws StorageException If failed. */ - private boolean close(boolean rollOver) throws StorageException { + private boolean close(boolean rollOver) throws IgniteCheckedException, StorageException { if (stop.compareAndSet(false, true)) { lock.lock(); @@ -2806,49 +2793,43 @@ private boolean close(boolean rollOver) throws StorageException { assert stopped() : "Segment is not closed after close flush: " + head.get(); try { - try { - RecordSerializer backwardSerializer = new RecordSerializerFactoryImpl(cctx) - .createSerializer(serializerVersion); + RecordSerializer backwardSerializer = new RecordSerializerFactoryImpl(cctx) + .createSerializer(serializerVersion); - SwitchSegmentRecord segmentRecord = new SwitchSegmentRecord(); + SwitchSegmentRecord segmentRecord = new SwitchSegmentRecord(); - int switchSegmentRecSize = backwardSerializer.size(segmentRecord); + int switchSegmentRecSize = backwardSerializer.size(segmentRecord); - if (rollOver && written < (maxSegmentSize - switchSegmentRecSize)) { - final ByteBuffer buf = ByteBuffer.allocate(switchSegmentRecSize); + if (rollOver && written < (maxSegmentSize - switchSegmentRecSize)) { + final ByteBuffer buf = ByteBuffer.allocate(switchSegmentRecSize); - segmentRecord.position(new FileWALPointer(idx, (int)written, switchSegmentRecSize)); - backwardSerializer.writeRecord(segmentRecord, buf); + segmentRecord.position(new FileWALPointer(idx, (int)written, switchSegmentRecSize)); + backwardSerializer.writeRecord(segmentRecord, buf); - buf.rewind(); + buf.rewind(); - int rem = buf.remaining(); + int rem = buf.remaining(); - while (rem > 0) { - int written0 = fileIO.write(buf, written); + while (rem > 0) { + int written0 = fileIO.write(buf, written); - written += written0; + written += written0; - rem -= written0; - } + rem -= written0; } } - catch (IgniteCheckedException e) { - throw new IOException(e); - } - finally { - // Do the final fsync. - if (mode == WALMode.FSYNC) { - fileIO.force(); - lastFsyncPos = written; - } + // Do the final fsync. + if (mode == WALMode.FSYNC) { + fileIO.force(); - fileIO.close(); + lastFsyncPos = written; } + + fileIO.close(); } catch (IOException e) { - throw new StorageException(e); + throw new IgniteCheckedException(e); } if (log.isDebugEnabled()) @@ -2891,9 +2872,9 @@ private void signalNextAvailable() { } /** - * + * @throws IgniteCheckedException If failed. */ - private void awaitNext() { + private void awaitNext() throws IgniteCheckedException { lock.lock(); try { @@ -2913,7 +2894,7 @@ private void awaitNext() { * @throws IgniteCheckedException If failed. */ @SuppressWarnings("TooBroadScope") - private void writeBuffer(long pos, ByteBuffer buf) throws StorageException { + private void writeBuffer(long pos, ByteBuffer buf) throws StorageException, IgniteCheckedException { boolean interrupted = false; lock.lock(); diff --git a/modules/core/src/test/java/org/apache/ignite/failure/TestFailureHandler.java b/modules/core/src/test/java/org/apache/ignite/failure/TestFailureHandler.java index 545c9ea1176d1..1159683e6b54f 100644 --- a/modules/core/src/test/java/org/apache/ignite/failure/TestFailureHandler.java +++ b/modules/core/src/test/java/org/apache/ignite/failure/TestFailureHandler.java @@ -18,7 +18,6 @@ package org.apache.ignite.failure; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; import org.apache.ignite.Ignite; /** @@ -34,13 +33,6 @@ public class TestFailureHandler implements FailureHandler { /** Failure context. */ volatile FailureContext failureCtx; - /** - * @param invalidate Invalidate. - */ - public TestFailureHandler(boolean invalidate) { - this(invalidate, new CountDownLatch(1)); - } - /** * @param invalidate Invalidate. * @param latch Latch. @@ -68,15 +60,4 @@ public TestFailureHandler(boolean invalidate, CountDownLatch latch) { public FailureContext failureContext() { return failureCtx; } - - /** - * @param millis Millis. - - * @return Failure context. - */ - public FailureContext awaitFailure(long millis) throws InterruptedException { - latch.await(millis, TimeUnit.MILLISECONDS); - - return failureCtx; - } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFormatFileFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFormatFileFailoverTest.java deleted file mode 100644 index 379b8c32cda89..0000000000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFormatFileFailoverTest.java +++ /dev/null @@ -1,258 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.processors.cache.persistence.db.wal; - -import java.io.File; -import java.io.IOException; -import java.nio.file.OpenOption; -import java.util.Arrays; -import java.util.concurrent.atomic.AtomicReference; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.cache.CacheAtomicityMode; -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.failure.FailureHandler; -import org.apache.ignite.failure.TestFailureHandler; -import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; -import org.apache.ignite.internal.pagemem.wal.StorageException; -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.X; -import org.apache.ignite.testframework.GridTestUtils; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; - -import static java.nio.file.StandardOpenOption.CREATE; -import static java.nio.file.StandardOpenOption.READ; -import static java.nio.file.StandardOpenOption.WRITE; - -/** - * - */ -public class IgniteWalFormatFileFailoverTest extends GridCommonAbstractTest { - /** */ - private static final String TEST_CACHE = "testCache"; - - /** */ - private static final String formatFile = "formatFile"; - - /** Fail method name reference. */ - private final AtomicReference failMtdNameRef = new AtomicReference<>(); - - /** */ - private boolean fsync; - - /** {@inheritDoc} */ - @Override protected void beforeTest() throws Exception { - cleanPersistenceDir(); - } - - /** {@inheritDoc} */ - @Override protected void afterTest() throws Exception { - stopAllGrids(); - - cleanPersistenceDir(); - } - - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - cfg.setCacheConfiguration(new CacheConfiguration(TEST_CACHE) - .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL)); - - DataStorageConfiguration memCfg = new DataStorageConfiguration() - .setDefaultDataRegionConfiguration(new DataRegionConfiguration() - .setMaxSize(2048L * 1024 * 1024) - .setPersistenceEnabled(true)) - .setWalMode(fsync ? WALMode.FSYNC : WALMode.BACKGROUND) - .setWalBufferSize(1024 * 1024) - .setWalSegmentSize(512 * 1024) - .setFileIOFactory(new FailingFileIOFactory(failMtdNameRef)); - - cfg.setDataStorageConfiguration(memCfg); - - cfg.setFailureHandler(new TestFailureHandler(false)); - - return cfg; - } - - /** - * @throws Exception If failed. - */ - public void testNodeStartFailedFsync() throws Exception { - fsync = true; - - failMtdNameRef.set(formatFile); - - checkCause(GridTestUtils.assertThrows(log, () -> startGrid(0), IgniteCheckedException.class, null)); - } - - /** - * @throws Exception If failed. - */ - public void testFailureHandlerTriggeredFsync() throws Exception { - fsync = true; - - failFormatFileOnClusterActivate(); - } - - /** - * @throws Exception If failed. - */ - public void testFailureHandlerTriggered() throws Exception { - fsync = false; - - failFormatFileOnClusterActivate(); - } - - /** - * @throws Exception If failed. - */ - private void failFormatFileOnClusterActivate() throws Exception { - failMtdNameRef.set(null); - - startGrid(0); - startGrid(1); - - if (!fsync) { - setFileIOFactory(grid(0).context().cache().context().wal()); - setFileIOFactory(grid(1).context().cache().context().wal()); - } - - failMtdNameRef.set(formatFile); - - grid(0).cluster().active(true); - - checkCause(failureHandler(0).awaitFailure(2000).error()); - checkCause(failureHandler(1).awaitFailure(2000).error()); - } - - /** - * @param mtdName Method name. - */ - private static boolean isCalledFrom(String mtdName) { - return isCalledFrom(Thread.currentThread().getStackTrace(), mtdName); - } - - /** - * @param stackTrace Stack trace. - * @param mtdName Method name. - */ - private static boolean isCalledFrom(StackTraceElement[] stackTrace, String mtdName) { - return Arrays.stream(stackTrace).map(StackTraceElement::getMethodName).anyMatch(mtdName::equals); - } - - /** - * @param gridIdx Grid index. - * @return Failure handler configured for grid with given index. - */ - private TestFailureHandler failureHandler(int gridIdx) { - FailureHandler hnd = grid(gridIdx).configuration().getFailureHandler(); - - assertTrue(hnd instanceof TestFailureHandler); - - return (TestFailureHandler)hnd; - } - - /** - * @param t Throwable. - */ - private void checkCause(Throwable t) { - StorageException e = X.cause(t, StorageException.class); - - assertNotNull(e); - assertNotNull(e.getMessage()); - assertTrue(e.getMessage().contains("Failed to format WAL segment file")); - - IOException ioe = X.cause(e, IOException.class); - - assertNotNull(ioe); - assertNotNull(ioe.getMessage()); - assertTrue(ioe.getMessage().contains("No space left on device")); - - assertTrue(isCalledFrom(ioe.getStackTrace(), formatFile)); - } - - /** */ - private void setFileIOFactory(IgniteWriteAheadLogManager wal) { - if (wal instanceof FileWriteAheadLogManager) - ((FileWriteAheadLogManager)wal).setFileIOFactory(new FailingFileIOFactory(failMtdNameRef)); - else - fail(wal.getClass().toString()); - } - - /** - * Create File I/O which fails if specific method call present in stack trace. - */ - private static class FailingFileIOFactory implements FileIOFactory { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** Delegate factory. */ - private final FileIOFactory delegateFactory = new RandomAccessFileIOFactory(); - - /** Fail method name reference. */ - private final AtomicReference failMtdNameRef; - - /** - * @param failMtdNameRef Fail method name reference. - */ - FailingFileIOFactory(AtomicReference failMtdNameRef) { - assertNotNull(failMtdNameRef); - - this.failMtdNameRef = failMtdNameRef; - } - - /** {@inheritDoc} */ - @Override public FileIO create(File file) throws IOException { - return create(file, CREATE, READ, WRITE); - } - - /** {@inheritDoc} */ - @Override public FileIO create(File file, OpenOption... modes) throws IOException { - final FileIO delegate = delegateFactory.create(file, modes); - - return new FileIODecorator(delegate) { - @Override public int write(byte[] buf, int off, int len) throws IOException { - conditionalFail(); - - return super.write(buf, off, len); - } - - @Override public void clear() throws IOException { - conditionalFail(); - - super.clear(); - } - - private void conditionalFail() throws IOException { - String failMtdName = failMtdNameRef.get(); - - if (failMtdName != null && isCalledFrom(failMtdName)) - throw new IOException("No space left on device"); - } - }; - } - } -} 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 316ff92fcdab2..27757435a7d9c 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 @@ -49,7 +49,6 @@ import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushFsyncWithMmapBufferSelfTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushLogOnlySelfTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushLogOnlyWithMmapBufferSelfTest; -import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFormatFileFailoverTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalHistoryReservationsTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalIteratorSwitchSegmentTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalSerializerVersionTest; @@ -147,8 +146,6 @@ public static void addRealPageStoreTests(TestSuite suite) { suite.addTestSuite(IgniteWalFlushLogOnlyWithMmapBufferSelfTest.class); - suite.addTestSuite(IgniteWalFormatFileFailoverTest.class); - // Test suite uses Standalone WAL iterator to verify PDS content. suite.addTestSuite(IgniteWalReaderTest.class); From 802ddafd3113421aef5cc2159521868ed80e302f Mon Sep 17 00:00:00 2001 From: Dmitriy Sorokin Date: Tue, 19 Jun 2018 15:35:51 +0300 Subject: [PATCH 0282/1463] IGNITE-8769 JVM crash in Basic1 suite in master branch on TC - Fixes #4206. Signed-off-by: Ivan Rakov (cherry picked from commit b37f8a2) --- .../cache/persistence/freelist/PagesList.java | 23 +++++++++++++++---- 1 file changed, 18 insertions(+), 5 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java index ed77674a8e024..78dc91f512c57 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java @@ -635,8 +635,19 @@ protected final void put( continue; } - assert PageIO.getPageId(tailAddr) == tailId : "pageId = " + PageIO.getPageId(tailAddr) + ", tailId = " + tailId; - assert PageIO.getType(tailAddr) == PageIO.T_PAGE_LIST_NODE; + if (stripe.tailId != tailId) { + // Another thread took the last page. + writeUnlock(tailId, tailPage, tailAddr, false); + + lockAttempt--; // Ignore current attempt. + + continue; + } + + assert PageIO.getPageId(tailAddr) == tailId + : "tailId = " + U.hexLong(tailId) + ", pageId = " + U.hexLong(PageIO.getPageId(tailAddr)); + assert PageIO.getType(tailAddr) == PageIO.T_PAGE_LIST_NODE + : "tailId = " + U.hexLong(tailId) + ", type = " + PageIO.getType(tailAddr); boolean ok = false; @@ -1032,7 +1043,7 @@ protected final long takeEmptyPage(int bucket, @Nullable IOVersions initIoVers) if (tailAddr == 0L) continue; - if (stripe.empty) { + if (stripe.empty || stripe.tailId != tailId) { // Another thread took the last page. writeUnlock(tailId, tailPage, tailAddr, false); @@ -1045,8 +1056,10 @@ protected final long takeEmptyPage(int bucket, @Nullable IOVersions initIoVers) return 0L; } - assert PageIO.getPageId(tailAddr) == tailId : "tailId = " + tailId + ", tailPageId = " + PageIO.getPageId(tailAddr); - assert PageIO.getType(tailAddr) == PageIO.T_PAGE_LIST_NODE; + assert PageIO.getPageId(tailAddr) == tailId + : "tailId = " + U.hexLong(tailId) + ", pageId = " + U.hexLong(PageIO.getPageId(tailAddr)); + assert PageIO.getType(tailAddr) == PageIO.T_PAGE_LIST_NODE + : "tailId = " + U.hexLong(tailId) + ", type = " + PageIO.getType(tailAddr); boolean dirty = false; long dataPageId; From 77f857e15e59535d5715e0022e9d8caf38ef8bee Mon Sep 17 00:00:00 2001 From: Alexander Menshikov Date: Wed, 25 Apr 2018 14:03:24 +0300 Subject: [PATCH 0283/1463] IGNITE-6565 Use long type for size and keySize in cache metrics Signed-off-by: Anton Vinogradov (cherry picked from commit 0dc906f) --- .../org/apache/ignite/cache/CacheMetrics.java | 11 ++++++ .../cache/CacheClusterMetricsMXBeanImpl.java | 5 +++ .../cache/CacheLocalMetricsMXBeanImpl.java | 5 +++ .../processors/cache/CacheMetricsImpl.java | 30 +++++++++++++- .../cache/CacheMetricsSnapshot.java | 10 +++++ .../platform/cache/PlatformCache.java | 1 + .../visor/cache/VisorCacheMetrics.java | 17 ++++++++ .../ignite/mxbean/CacheMetricsMXBean.java | 4 ++ .../GridCacheAbstractMetricsSelfTest.java | 39 +++++++++++++++++++ .../PlatformCacheWriteMetricsTask.java | 5 +++ .../Cache/CacheMetricsTest.cs | 7 ++++ .../Apache.Ignite.Core/Cache/ICacheMetrics.cs | 8 ++++ .../Impl/Cache/CacheMetricsImpl.cs | 7 ++++ 13 files changed, 148 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java index 0b1cb87c3a301..eae7989b0e8c9 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java @@ -240,14 +240,25 @@ public interface CacheMetrics { * Gets number of non-{@code null} values in the cache. * * @return Number of non-{@code null} values in the cache. + * @deprecated Can overflow. Use {@link CacheMetrics#getCacheSize()} instead. */ + @Deprecated public int getSize(); + /** + * Gets number of non-{@code null} values in the cache as a long value. + * + * @return Number of non-{@code null} values in the cache. + */ + public long getCacheSize(); + /** * Gets number of keys in the cache, possibly with {@code null} values. * * @return Number of keys in the cache. + * @deprecated Can overflow. Use {@link CacheMetrics#getCacheSize()} instead. */ + @Deprecated public int getKeySize(); /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClusterMetricsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClusterMetricsMXBeanImpl.java index ce6416fa0c766..32603cbf940d5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClusterMetricsMXBeanImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClusterMetricsMXBeanImpl.java @@ -114,6 +114,11 @@ class CacheClusterMetricsMXBeanImpl implements CacheMetricsMXBean { return cache.clusterMetrics().getSize(); } + /** {@inheritDoc} */ + @Override public long getCacheSize() { + return cache.clusterMetrics().getCacheSize(); + } + /** {@inheritDoc} */ @Override public int getKeySize() { return cache.clusterMetrics().getKeySize(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLocalMetricsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLocalMetricsMXBeanImpl.java index 438c8c666c01b..d3060d3ab3835 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLocalMetricsMXBeanImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLocalMetricsMXBeanImpl.java @@ -115,6 +115,11 @@ class CacheLocalMetricsMXBeanImpl implements CacheMetricsMXBean { return cache.metrics0().getSize(); } + /** {@inheritDoc} */ + @Override public long getCacheSize() { + return cache.metrics0().getCacheSize(); + } + /** {@inheritDoc} */ @Override public int getKeySize() { return cache.metrics0().getKeySize(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java index b402ff2bb9170..96f40bfd72ea3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java @@ -254,6 +254,11 @@ public void delegate(CacheMetricsImpl delegate) { return getEntriesStat().size(); } + /** {@inheritDoc} */ + @Override public long getCacheSize() { + return getEntriesStat().cacheSize(); + } + /** {@inheritDoc} */ @Override public int getKeySize() { return getEntriesStat().keySize(); @@ -754,6 +759,7 @@ public EntriesStatMetrics getEntriesStat() { long offHeapBackupEntriesCnt = 0L; long heapEntriesCnt = 0L; int size = 0; + long sizeLong = 0L; boolean isEmpty; try { @@ -765,8 +771,9 @@ public EntriesStatMetrics getEntriesStat() { offHeapBackupEntriesCnt = offHeapEntriesCnt; size = cctx.cache().size(); + sizeLong = cctx.cache().sizeLong(); - heapEntriesCnt = size; + heapEntriesCnt = sizeLong; } } else { @@ -806,6 +813,8 @@ public EntriesStatMetrics getEntriesStat() { heapEntriesCnt += part.publicSize(cctx.cacheId()); } + + sizeLong = offHeapEntriesCnt; } } catch (Exception e) { @@ -816,6 +825,7 @@ public EntriesStatMetrics getEntriesStat() { offHeapBackupEntriesCnt = -1L; heapEntriesCnt = -1L; size = -1; + sizeLong = -1L; } isEmpty = (offHeapEntriesCnt == 0); @@ -827,6 +837,7 @@ public EntriesStatMetrics getEntriesStat() { stat.offHeapBackupEntriesCount(offHeapBackupEntriesCnt); stat.heapEntriesCount(heapEntriesCnt); stat.size(size); + stat.cacheSize(sizeLong); stat.keySize(size); stat.isEmpty(isEmpty); stat.totalPartitionsCount(owningPartCnt + movingPartCnt); @@ -1039,6 +1050,9 @@ public static class EntriesStatMetrics { /** Size. */ private int size; + /** Long size. */ + private long cacheSize; + /** Key size. */ private int keySize; @@ -1157,6 +1171,20 @@ public void keySize(int keySize) { this.keySize = keySize; } + /** + * @return Long size. + */ + public long cacheSize() { + return cacheSize; + } + + /** + * @param cacheSize Size long. + */ + public void cacheSize(long cacheSize) { + this.cacheSize = cacheSize; + } + /** * @return Is empty. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java index e69372001f215..539ad59a818a1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java @@ -110,6 +110,9 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable { /** Number of non-{@code null} values in the cache. */ private int size; + /** Number of non-{@code null} values in the cache as long value as a long value. */ + private long cacheSize; + /** Number of keys in the cache, possibly with {@code null} values. */ private int keySize; @@ -286,6 +289,7 @@ public CacheMetricsSnapshot(CacheMetricsImpl m) { offHeapAllocatedSize = m.getOffHeapAllocatedSize(); size = entriesStat.size(); + cacheSize = entriesStat.cacheSize(); keySize = entriesStat.keySize(); isEmpty = entriesStat.isEmpty(); @@ -351,6 +355,7 @@ public CacheMetricsSnapshot(CacheMetrics loc, Collection metrics) writeBehindStoreBatchSize = loc.getWriteBehindStoreBatchSize(); writeBehindBufSize = loc.getWriteBehindBufferSize(); size = loc.getSize(); + cacheSize = loc.getCacheSize(); keySize = loc.getKeySize(); keyType = loc.getKeyType(); @@ -633,6 +638,11 @@ public CacheMetricsSnapshot(CacheMetrics loc, Collection metrics) return size; } + /** {@inheritDoc} */ + @Override public long getCacheSize() { + return cacheSize; + } + /** {@inheritDoc} */ @Override public int getKeySize() { return keySize; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java index c78f2d62d488a..818fd67ebe63d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java @@ -1503,6 +1503,7 @@ public static void writeCacheMetrics(BinaryRawWriter writer, CacheMetrics metric writer.writeLong(metrics.getEstimatedRebalancingFinishTime()); writer.writeLong(metrics.getRebalancingStartTime()); writer.writeLong(metrics.getRebalanceClearingPartitionsLeft()); + writer.writeLong(metrics.getCacheSize()); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java index 5d8bc8151cfbf..59f16b2a5e7cd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java @@ -55,6 +55,9 @@ public class VisorCacheMetrics extends VisorDataTransferObject { /** Gets number of keys in the cache, possibly with {@code null} values. */ private int keySize; + /** Number of non-{@code null} values in the cache as a long value. */ + private long cacheSize; + /** Total number of reads of the owning entity (either cache or entry). */ private long reads; @@ -224,6 +227,8 @@ public VisorCacheMetrics(IgniteEx ignite, String cacheName) { size = m.getSize(); keySize = m.getKeySize(); + cacheSize = m.getCacheSize(); + reads = m.getCacheGets(); writes = m.getCachePuts() + m.getCacheRemovals(); hits = m.getCacheHits(); @@ -456,6 +461,13 @@ public int getKeySize() { return keySize; } + /** + * @return Number of non-{@code null} values in the cache as a long value. + */ + public long getCacheSize() { + return cacheSize; + } + /** * @return Gets query metrics for cache. */ @@ -694,6 +706,8 @@ public long getRebalancingBytesRate() { out.writeLong(rebalancingBytesRate); out.writeObject(qryMetrics); + + out.writeLong(cacheSize); } /** {@inheritDoc} */ @@ -751,6 +765,9 @@ public long getRebalancingBytesRate() { rebalancingBytesRate = in.readLong(); qryMetrics = (VisorQueryMetrics)in.readObject(); + + if (in.available() > 0) + cacheSize = in.readLong(); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java index ae03a5a8fc91f..16bdedeb2c471 100644 --- a/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java +++ b/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java @@ -151,6 +151,10 @@ public interface CacheMetricsMXBean extends CacheStatisticsMXBean, CacheMXBean, @MXBeanDescription("Number of non-null values in the cache.") public int getSize(); + /** {@inheritDoc} */ + @MXBeanDescription("Number of non-null values in the cache as a long value.") + public long getCacheSize(); + /** {@inheritDoc} */ @MXBeanDescription("Number of keys in the cache (possibly with null values).") public int getKeySize(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java index e20715826e7b9..7948569bf9327 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java @@ -674,6 +674,45 @@ public void testRemoves() throws Exception { assertEquals(1L, cache.localMetrics().getCacheRemovals()); } + /** + * Test {@link CacheMetrics#getSize()} and {@link CacheMetrics#getCacheSize()} work equally. + * + * @throws Exception If failed. + */ + public void testCacheSizeWorksAsSize() throws Exception { + IgniteCache cache = grid(0).cache(DEFAULT_CACHE_NAME); + + assertEquals(cache.metrics().getSize(), cache.metrics().getCacheSize()); + + for (int i = 0; i < KEY_CNT; i++) { + cache.put(i, i); + + CacheMetrics metrics = cache.metrics(); + + assertEquals(metrics.getSize(), metrics.getCacheSize()); + + CacheMetrics localMetrics = cache.localMetrics(); + + assertEquals(localMetrics.getSize(), localMetrics.getCacheSize()); + } + + for (int i = 0; i < KEY_CNT / 2; i++) { + cache.remove(i, i); + + CacheMetrics metrics = cache.metrics(); + + assertEquals(metrics.getSize(), metrics.getCacheSize()); + + CacheMetrics localMetrics = cache.localMetrics(); + + assertEquals(localMetrics.getSize(), localMetrics.getCacheSize()); + } + + cache.removeAll(); + + assertEquals(cache.metrics().getSize(), cache.metrics().getCacheSize()); + } + /** * @throws Exception If failed. */ diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java index 3b7bb8905c2eb..44a15d500ca44 100644 --- a/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java +++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java @@ -468,5 +468,10 @@ private static class TestCacheMetrics implements CacheMetrics { @Override public long getRebalanceClearingPartitionsLeft() { return 64; } + + /** {@inheritDoc} */ + @Override public long getCacheSize() { + return 65; + } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs index 3f671d9d71150..129b4b5de662e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs @@ -67,10 +67,12 @@ public void TestLocalMetrics() var remoteMetrics = metrics.Item2; Assert.AreEqual(1, localMetrics.Size); + Assert.AreEqual(1, localMetrics.CacheSize); Assert.AreEqual(1, localMetrics.CacheGets); Assert.AreEqual(1, localMetrics.CachePuts); Assert.AreEqual(0, remoteMetrics.Size); + Assert.AreEqual(0, remoteMetrics.CacheSize); Assert.AreEqual(0, remoteMetrics.CacheGets); Assert.AreEqual(0, remoteMetrics.CachePuts); } @@ -87,10 +89,12 @@ public void TestGlobalMetrics() var remoteMetrics = metrics.Item2; Assert.AreEqual(1, localMetrics.Size); + Assert.AreEqual(1, localMetrics.CacheSize); Assert.AreEqual(1, localMetrics.CacheGets); Assert.AreEqual(1, localMetrics.CachePuts); Assert.AreEqual(0, remoteMetrics.Size); + Assert.AreEqual(0, remoteMetrics.CacheSize); Assert.AreEqual(1, remoteMetrics.CacheGets); Assert.AreEqual(1, remoteMetrics.CachePuts); } @@ -111,10 +115,12 @@ public void TestClusterGroupMetrics() var remoteMetrics = metrics.Item1; Assert.AreEqual(1, localMetrics.Size); + Assert.AreEqual(1, localMetrics.CacheSize); Assert.AreEqual(1, localMetrics.CacheGets); Assert.AreEqual(1, localMetrics.CachePuts); Assert.AreEqual(1, remoteMetrics.Size); + Assert.AreEqual(1, remoteMetrics.CacheSize); Assert.AreEqual(0, remoteMetrics.CacheGets); Assert.AreEqual(0, remoteMetrics.CachePuts); } @@ -203,6 +209,7 @@ public void TestMetricsPropagation() Assert.AreEqual(59, metrics.HeapEntriesCount); Assert.AreEqual(62, metrics.EstimatedRebalancingFinishTime); Assert.AreEqual(63, metrics.RebalancingStartTime); + Assert.AreEqual(65, metrics.CacheSize); Assert.AreEqual("foo", metrics.KeyType); Assert.AreEqual("bar", metrics.ValueType); Assert.AreEqual(true, metrics.IsStoreByValue); diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs index a328cf5adc8ac..d775c05691076 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs @@ -256,6 +256,14 @@ public interface ICacheMetrics /// int Size { get; } + /// + /// Gets number of non-null values in the cache. + /// + /// + /// Number of non-null values in the cache. + /// + long CacheSize { get; } + /// /// Gets number of keys in the cache, possibly with null values. /// diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs index fbc5d4cbba7cd..1fdc8771f832e 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs @@ -115,6 +115,9 @@ internal class CacheMetricsImpl : ICacheMetrics /** */ private readonly int _keySize; + /** */ + private readonly long _cacheSize; + /** */ private readonly bool _isEmpty; @@ -323,6 +326,7 @@ public CacheMetricsImpl(IBinaryRawReader reader) _estimatedRebalancingFinishTime = reader.ReadLong(); _rebalancingStartTime = reader.ReadLong(); _rebalancingClearingPartitionsLeft = reader.ReadLong(); + _cacheSize = reader.ReadLong(); } /** */ @@ -412,6 +416,9 @@ public CacheMetricsImpl(IBinaryRawReader reader) /** */ public int Size { get { return _size; } } + /** */ + public long CacheSize { get { return _cacheSize; } } + /** */ public int KeySize { get { return _keySize; } } From bb94a869c818f349fea1d0bc54ad70b4a05f03bd Mon Sep 17 00:00:00 2001 From: Ilya Lantukh Date: Tue, 19 Jun 2018 19:33:24 +0300 Subject: [PATCH 0284/1463] IGNITE-8554 Cache metrics: expose metrics with rebalance info about keys - Fixes #4094. Signed-off-by: Ivan Rakov (cherry picked from commit cf09e76) --- .../org/apache/ignite/cache/CacheMetrics.java | 10 +++ .../cache/CacheAffinitySharedManager.java | 3 +- .../cache/CacheClusterMetricsMXBeanImpl.java | 10 +++ .../cache/CacheLocalMetricsMXBeanImpl.java | 8 ++ .../processors/cache/CacheMetricsImpl.java | 15 +++- .../cache/CacheMetricsSnapshot.java | 30 +++++++ .../GridCachePartitionExchangeManager.java | 13 ++- .../dht/GridClientPartitionTopology.java | 35 ++++++++ .../dht/GridDhtPartitionTopology.java | 11 +++ .../dht/GridDhtPartitionTopologyImpl.java | 34 ++++++++ .../preloader/GridDhtPartitionDemander.java | 28 +++++-- .../GridDhtPartitionSupplyMessage.java | 9 +- .../GridDhtPartitionsExchangeFuture.java | 61 ++++++++++++-- .../GridDhtPartitionsFullMessage.java | 80 ++++++++++++++++-- .../GridDhtPartitionsSingleMessage.java | 24 +++--- .../platform/cache/PlatformCache.java | 2 + .../visor/cache/VisorCacheMetrics.java | 35 ++++++++ .../visor/node/VisorNodeDataCollectorJob.java | 20 ++++- .../CacheGroupsMetricsRebalanceTest.java | 82 +++++++++++-------- .../PlatformCacheWriteMetricsTask.java | 10 +++ .../Apache.Ignite.Core/Cache/ICacheMetrics.cs | 18 ++++ .../Impl/Cache/CacheMetricsImpl.cs | 14 ++++ 22 files changed, 468 insertions(+), 84 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java index eae7989b0e8c9..15e56c789c946 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java @@ -501,6 +501,16 @@ public interface CacheMetrics { */ public int getRebalancingPartitionsCount(); + /** + * @return Number of already rebalanced keys. + */ + public long getRebalancedKeys(); + + /** + * @return Number estimated to rebalance keys. + */ + public long getEstimatedRebalancingKeys(); + /** * @return Estimated number of keys to be rebalanced on current node. */ 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 92b8d3ea87019..08eb43f5291c8 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 @@ -472,6 +472,7 @@ void onCacheGroupCreated(CacheGroupContext grp) { clientTop.partitionMap(true), clientTop.fullUpdateCounters(), Collections.emptySet(), + null, null); } @@ -530,7 +531,7 @@ else if (!fetchFuts.containsKey(grp.groupId())) { grp.topology().updateTopologyVersion(topFut, discoCache, -1, false); - grp.topology().update(topVer, partMap, null, Collections.emptySet(), null); + grp.topology().update(topVer, partMap, null, Collections.emptySet(), null, null); topFut.validate(grp, discoCache.allNodes()); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClusterMetricsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClusterMetricsMXBeanImpl.java index 32603cbf940d5..3d5278cc96496 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClusterMetricsMXBeanImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheClusterMetricsMXBeanImpl.java @@ -369,6 +369,16 @@ class CacheClusterMetricsMXBeanImpl implements CacheMetricsMXBean { return cache.clusterMetrics().getTotalPartitionsCount(); } + /** {@inheritDoc} */ + @Override public long getRebalancedKeys() { + return cache.clusterMetrics().getRebalancedKeys(); + } + + /** {@inheritDoc} */ + @Override public long getEstimatedRebalancingKeys() { + return cache.clusterMetrics().getEstimatedRebalancingKeys(); + } + /** {@inheritDoc} */ @Override public int getRebalancingPartitionsCount() { return cache.clusterMetrics().getRebalancingPartitionsCount(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLocalMetricsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLocalMetricsMXBeanImpl.java index d3060d3ab3835..212c7a07c4620 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLocalMetricsMXBeanImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLocalMetricsMXBeanImpl.java @@ -370,6 +370,14 @@ class CacheLocalMetricsMXBeanImpl implements CacheMetricsMXBean { return cache.metrics0().getTotalPartitionsCount(); } + @Override public long getRebalancedKeys() { + return cache.metrics0().getRebalancedKeys(); + } + + @Override public long getEstimatedRebalancingKeys() { + return cache.metrics0().getEstimatedRebalancingKeys(); + } + /** {@inheritDoc} */ @Override public int getRebalancingPartitionsCount() { return cache.metrics0().getRebalancingPartitionsCount(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java index 96f40bfd72ea3..0f6d06f7edd56 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java @@ -856,6 +856,16 @@ public EntriesStatMetrics getEntriesStat() { return getEntriesStat().rebalancingPartitionsCount(); } + /** {@inheritDoc} */ + @Override public long getRebalancedKeys() { + return rebalancedKeys.get(); + } + + /** {@inheritDoc} */ + @Override public long getEstimatedRebalancingKeys() { + return estimatedRebalancingKeys.get(); + } + /** {@inheritDoc} */ @Override public long getKeysToRebalanceLeft() { return Math.max(0, estimatedRebalancingKeys.get() - rebalancedKeys.get()); @@ -935,7 +945,10 @@ public void rebalanceClearingPartitions(int partitions) { * First rebalance supply message callback. * @param keysCnt Estimated number of keys. */ - public void onRebalancingKeysCountEstimateReceived(long keysCnt) { + public void onRebalancingKeysCountEstimateReceived(Long keysCnt) { + if (keysCnt == null) + return; + estimatedRebalancingKeys.addAndGet(keysCnt); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java index 539ad59a818a1..8a0f0e4b326ec 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java @@ -197,6 +197,12 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable { /** Rebalancing partitions count. */ private int rebalancingPartitionsCnt; + /** Number of already rebalanced keys. */ + private long rebalancedKeys; + + /** Number estimated to rebalance keys. */ + private long estimatedRebalancingKeys; + /** Keys to rebalance left. */ private long keysToRebalanceLeft; @@ -331,6 +337,8 @@ public CacheMetricsSnapshot(CacheMetricsImpl m) { totalPartitionsCnt = entriesStat.totalPartitionsCount(); rebalancingPartitionsCnt = entriesStat.rebalancingPartitionsCount(); + rebalancedKeys = m.getRebalancedKeys(); + estimatedRebalancingKeys = m.getEstimatedRebalancingKeys(); keysToRebalanceLeft = m.getKeysToRebalanceLeft(); rebalancingBytesRate = m.getRebalancingBytesRate(); rebalancingKeysRate = m.getRebalancingKeysRate(); @@ -459,6 +467,8 @@ public CacheMetricsSnapshot(CacheMetrics loc, Collection metrics) else writeBehindErrorRetryCnt = -1; + rebalancedKeys += e.getRebalancedKeys(); + estimatedRebalancingKeys += e.getEstimatedRebalancingKeys(); totalPartitionsCnt += e.getTotalPartitionsCount(); rebalancingPartitionsCnt += e.getRebalancingPartitionsCount(); keysToRebalanceLeft += e.getKeysToRebalanceLeft(); @@ -733,6 +743,14 @@ public CacheMetricsSnapshot(CacheMetrics loc, Collection metrics) return totalPartitionsCnt; } + @Override public long getRebalancedKeys() { + return rebalancedKeys; + } + + @Override public long getEstimatedRebalancingKeys() { + return estimatedRebalancingKeys; + } + /** {@inheritDoc} */ @Override public int getRebalancingPartitionsCount() { return rebalancingPartitionsCnt; @@ -926,6 +944,12 @@ public CacheMetricsSnapshot(CacheMetrics loc, Collection metrics) out.writeLong(keysToRebalanceLeft); out.writeLong(rebalancingBytesRate); out.writeLong(rebalancingKeysRate); + + out.writeLong(rebalancedKeys); + out.writeLong(estimatedRebalancingKeys); + out.writeLong(rebalanceStartTime); + out.writeLong(rebalanceFinishTime); + out.writeLong(rebalanceClearingPartitionsLeft); } /** {@inheritDoc} */ @@ -981,5 +1005,11 @@ public CacheMetricsSnapshot(CacheMetrics loc, Collection metrics) keysToRebalanceLeft = in.readLong(); rebalancingBytesRate = in.readLong(); rebalancingKeysRate = in.readLong(); + + rebalancedKeys = in.readLong(); + estimatedRebalancingKeys = in.readLong(); + rebalanceStartTime = in.readLong(); + rebalanceFinishTime = in.readLong(); + rebalanceClearingPartitionsLeft = in.readLong(); } } 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 38ddaf661de92..715c2907b2506 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 @@ -1121,6 +1121,8 @@ public GridDhtPartitionsFullMessage createPartitionsFullMessage( affCache.similarAffinityKey()); } + m.addPartitionSizes(grp.groupId(), grp.topology().globalPartSizes()); + if (exchId != null) { CachePartitionFullCountersMap cntrsMap = grp.topology().fullUpdateCounters(); @@ -1154,6 +1156,8 @@ public GridDhtPartitionsFullMessage createPartitionsFullMessage( m.addPartitionUpdateCounters(top.groupId(), cntrsMap); else m.addPartitionUpdateCounters(top.groupId(), CachePartitionFullCountersMap.toCountersMap(cntrsMap)); + + m.addPartitionSizes(top.groupId(), top.globalPartSizes()); } } @@ -1264,9 +1268,9 @@ public GridDhtPartitionsSingleMessage createPartitionsSingleMessage( m.addPartitionUpdateCounters(grp.groupId(), newCntrMap ? cntrsMap : CachePartitionPartialCountersMap.toCountersMap(cntrsMap)); - - m.addPartitionSizes(grp.groupId(), grp.topology().partitionSizes()); } + + m.addPartitionSizes(grp.groupId(), grp.topology().partitionSizes()); } } @@ -1288,9 +1292,9 @@ public GridDhtPartitionsSingleMessage createPartitionsSingleMessage( m.addPartitionUpdateCounters(top.groupId(), newCntrMap ? cntrsMap : CachePartitionPartialCountersMap.toCountersMap(cntrsMap)); - - m.addPartitionSizes(top.groupId(), top.partitionSizes()); } + + m.addPartitionSizes(top.groupId(), top.partitionSizes()); } return m; @@ -1482,6 +1486,7 @@ else if (!grp.isLocal()) entry.getValue(), null, msg.partsToReload(cctx.localNodeId(), grpId), + msg.partitionSizes(grpId), msg.topologyVersion()); } } 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 b3652331b210b..fc80bbc0916c7 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 @@ -119,6 +119,9 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology { /** */ private final int parts; + /** */ + private volatile Map globalPartSizes; + /** * @param cctx Context. * @param discoCache Discovery data cache. @@ -707,6 +710,7 @@ private boolean shouldOverridePartitionMap(GridDhtPartitionMap currentMap, GridD GridDhtPartitionFullMap partMap, @Nullable CachePartitionFullCountersMap cntrMap, Set partsToReload, + @Nullable Map partSizes, @Nullable AffinityTopologyVersion msgTopVer) { if (log.isDebugEnabled()) log.debug("Updating full partition map [exchVer=" + exchangeVer + ", parts=" + fullMapString() + ']'); @@ -810,6 +814,9 @@ private boolean shouldOverridePartitionMap(GridDhtPartitionMap currentMap, GridD if (cntrMap != null) this.cntrMap = new CachePartitionFullCountersMap(cntrMap); + if (partSizes != null) + this.globalPartSizes = partSizes; + consistencyCheck(); if (log.isDebugEnabled()) @@ -1222,6 +1229,34 @@ private void removeNode(UUID nodeId) { return Collections.emptyMap(); } + /** {@inheritDoc} */ + @Override @Nullable public Map globalPartSizes() { + lock.readLock().lock(); + + try { + if (globalPartSizes == null) + return Collections.emptyMap(); + + return Collections.unmodifiableMap(globalPartSizes); + } + finally { + lock.readLock().unlock(); + } + } + + /** {@inheritDoc} */ + @Override public void globalPartSizes(@Nullable Map partSizes) { + lock.writeLock().lock(); + + try { + this.globalPartSizes = partSizes; + } + finally { + lock.writeLock().unlock(); + } + } + + /** {@inheritDoc} */ @Override public boolean rebalanceFinished(AffinityTopologyVersion topVer) { assert false : "Should not be called on non-affinity node"; 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 d6c54506e1208..b77dbd6e0d719 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 @@ -287,6 +287,7 @@ public boolean update( GridDhtPartitionFullMap partMap, @Nullable CachePartitionFullCountersMap cntrMap, Set partsToReload, + @Nullable Map partSizes, @Nullable AffinityTopologyVersion msgTopVer); /** @@ -382,6 +383,16 @@ public boolean update(@Nullable GridDhtPartitionExchangeId exchId, */ public void printMemoryStats(int threshold); + /** + * @return Sizes of up-to-date partition versions in topology. + */ + Map globalPartSizes(); + + /** + * @param partSizes Sizes of up-to-date partition versions in topology. + */ + void globalPartSizes(@Nullable Map partSizes); + /** * @param topVer Topology version. * @return {@code True} if rebalance process finished. 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 2ab838433729e..45e85f9142a86 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 @@ -137,6 +137,9 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology { /** Partition update counter. */ private final CachePartitionFullCountersMap cntrMap; + /** */ + private volatile Map globalPartSizes; + /** */ private volatile AffinityTopologyVersion rebalancedTopVer = AffinityTopologyVersion.NONE; @@ -1329,6 +1332,7 @@ private boolean shouldOverridePartitionMap(GridDhtPartitionMap currentMap, GridD GridDhtPartitionFullMap partMap, @Nullable CachePartitionFullCountersMap incomeCntrMap, Set partsToReload, + @Nullable Map partSizes, @Nullable AffinityTopologyVersion msgTopVer) { if (log.isDebugEnabled()) { log.debug("Updating full partition map [grp=" + grp.cacheOrGroupName() + ", exchVer=" + exchangeVer + @@ -1544,6 +1548,9 @@ else if (state == MOVING) { updateRebalanceVersion(aff.assignment()); } + if (partSizes != null) + this.globalPartSizes = partSizes; + consistencyCheck(); if (log.isDebugEnabled()) { @@ -2624,6 +2631,33 @@ private void removeNode(UUID nodeId) { } } + /** {@inheritDoc} */ + @Override public Map globalPartSizes() { + lock.readLock().lock(); + + try { + if (globalPartSizes == null) + return Collections.emptyMap(); + + return Collections.unmodifiableMap(globalPartSizes); + } + finally { + lock.readLock().unlock(); + } + } + + /** {@inheritDoc} */ + @Override public void globalPartSizes(@Nullable Map partSizes) { + lock.writeLock().lock(); + + try { + this.globalPartSizes = partSizes; + } + finally { + lock.writeLock().unlock(); + } + } + /** {@inheritDoc} */ @Override public boolean rebalanceFinished(AffinityTopologyVersion topVer) { AffinityTopologyVersion curTopVer = this.readyTopVer; 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 c94f511d91887..3cfc25f6b7fbf 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 @@ -307,9 +307,22 @@ Runnable addAssignments( metrics.clearRebalanceCounters(); - metrics.startRebalance(0); + for (GridDhtPartitionDemandMessage msg : assignments.values()) { + for (Integer partId : msg.partitions().fullSet()) { + metrics.onRebalancingKeysCountEstimateReceived(grp.topology().globalPartSizes().get(partId)); + } + + CachePartitionPartialCountersMap histMap = msg.partitions().historicalMap(); - rebalanceFut.listen(f -> metrics.clearRebalanceCounters()); + for (int i = 0; i < histMap.size(); i++) { + long from = histMap.initialUpdateCounterAt(i); + long to = histMap.updateCounterAt(i); + + metrics.onRebalancingKeysCountEstimateReceived(to - from); + } + } + + metrics.startRebalance(0); } } @@ -714,8 +727,6 @@ public void handleSupplyMessage( try { AffinityAssignment aff = grp.affinity().cachedAffinity(topVer); - GridCacheContext cctx = grp.sharedGroup() ? null : grp.singleCacheContext(); - ctx.database().checkpointReadLock(); try { @@ -749,11 +760,10 @@ public void handleSupplyMessage( break; } - if (grp.sharedGroup() && (cctx == null || cctx.cacheId() != entry.cacheId())) - cctx = ctx.cacheContext(entry.cacheId()); - - if (cctx != null && cctx.statisticsEnabled()) - cctx.cache().metrics0().onRebalanceKeyReceived(); + for (GridCacheContext cctx : grp.caches()) { + if (cctx.statisticsEnabled()) + cctx.cache().metrics0().onRebalanceKeyReceived(); + } } // If message was last for this partition, 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 77baa38cc52c3..4ecffc492c7b4 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 @@ -443,7 +443,7 @@ public int size() { * @return Estimated keys count. */ public long estimatedKeysCount() { - return estimatedKeysCnt; + return -1; } /** @@ -457,12 +457,7 @@ public void addEstimatedKeysCount(long cnt) { * @return Estimated keys count for a given cache ID. */ public long keysForCache(int cacheId) { - if (this.keysPerCache == null) - return -1; - - Long cnt = this.keysPerCache.get(cacheId); - - return cnt != null ? cnt : 0; + return -1; } /** 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 c00cd0c510704..07f5329921e0c 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 @@ -873,6 +873,7 @@ private void updateTopologies(boolean crd) throws IgniteCheckedException { clientTop.partitionMap(true), clientTop.fullUpdateCounters(), Collections.emptySet(), + null, null); } } @@ -2338,6 +2339,37 @@ private void onAffinityInitialized(IgniteInternalFuture partSizes = new HashMap<>(); + + for (Map.Entry e : msgs.entrySet()) { + GridDhtPartitionsSingleMessage singleMsg = e.getValue(); + + GridDhtPartitionMap partMap = singleMsg.partitions().get(top.groupId()); + + if (partMap == null) + continue; + + for (Map.Entry e0 : partMap.entrySet()) { + int p = e0.getKey(); + GridDhtPartitionState state = e0.getValue(); + + if (state == GridDhtPartitionState.OWNING) + partSizes.put(p, singleMsg.partitionSizes(top.groupId()).get(p)); + } + } + + for (GridDhtLocalPartition locPart : top.currentLocalPartitions()) { + if (locPart.state() == GridDhtPartitionState.OWNING) + partSizes.put(locPart.id(), locPart.fullSize()); + } + + top.globalPartSizes(partSizes); + } + /** * Collects and determines new owners of partitions for all nodes for given {@code top}. * @@ -2378,7 +2410,7 @@ private void assignPartitionStates(GridDhtPartitionTopology top) { CounterWithNodes maxCntr = maxCntrs.get(p); if (maxCntr == null || cntr > maxCntr.cnt) - maxCntrs.put(p, new CounterWithNodes(cntr, uuid)); + maxCntrs.put(p, new CounterWithNodes(cntr, e.getValue().partitionSizes(top.groupId()).get(p), uuid)); else if (cntr == maxCntr.cnt) maxCntr.nodes.add(uuid); } @@ -2404,7 +2436,7 @@ else if (cntr == maxCntr.cnt) CounterWithNodes maxCntr = maxCntrs.get(part.id()); if (maxCntr == null && cntr == 0) { - CounterWithNodes cntrObj = new CounterWithNodes(0, cctx.localNodeId()); + CounterWithNodes cntrObj = new CounterWithNodes(0, 0L, cctx.localNodeId()); for (UUID nodeId : msgs.keySet()) { if (top.partitionState(nodeId, part.id()) == GridDhtPartitionState.OWNING) @@ -2414,7 +2446,7 @@ else if (cntr == maxCntr.cnt) maxCntrs.put(part.id(), cntrObj); } else if (maxCntr == null || cntr > maxCntr.cnt) - maxCntrs.put(part.id(), new CounterWithNodes(cntr, cctx.localNodeId())); + maxCntrs.put(part.id(), new CounterWithNodes(cntr, part.fullSize(), cctx.localNodeId())); else if (cntr == maxCntr.cnt) maxCntr.nodes.add(cctx.localNodeId()); } @@ -2475,6 +2507,13 @@ else if (cntr == maxCntr.cnt) for (int part : parts) partsToReload.put(nodeId, top.groupId(), part); } + + Map partSizes = new HashMap<>(maxCntrs.size()); + + for (Map.Entry e : maxCntrs.entrySet()) + partSizes.put(e.getKey(), e.getValue().size); + + top.globalPartSizes(partSizes); } /** @@ -2879,16 +2918,16 @@ private void assignPartitionsStates() { if (grpDesc.config().getCacheMode() == CacheMode.LOCAL) continue; - if (!CU.isPersistentCache(grpDesc.config(), cctx.gridConfig().getDataStorageConfiguration())) - continue; - CacheGroupContext grpCtx = cctx.cache().cacheGroup(e.getKey()); GridDhtPartitionTopology top = grpCtx != null ? grpCtx.topology() : cctx.exchange().clientTopology(e.getKey(), events().discoveryCache()); - assignPartitionStates(top); + if (!CU.isPersistentCache(grpDesc.config(), cctx.gridConfig().getDataStorageConfiguration())) + assignPartitionSizes(top); + else + assignPartitionStates(top); } } @@ -3279,6 +3318,7 @@ private void updatePartitionFullMap(AffinityTopologyVersion resTopVer, GridDhtPa entry.getValue(), cntrMap, msg.partsToReload(cctx.localNodeId(), grpId), + msg.partitionSizes(grpId), null); } else { @@ -3294,6 +3334,7 @@ private void updatePartitionFullMap(AffinityTopologyVersion resTopVer, GridDhtPa entry.getValue(), cntrMap, Collections.emptySet(), + null, null); } } @@ -3878,6 +3919,9 @@ private static class CounterWithNodes { /** */ private final long cnt; + /** */ + private final long size; + /** */ private final Set nodes = new HashSet<>(); @@ -3885,8 +3929,9 @@ private static class CounterWithNodes { * @param cnt Count. * @param firstNode Node ID. */ - private CounterWithNodes(long cnt, UUID firstNode) { + private CounterWithNodes(long cnt, @Nullable Long size, UUID firstNode) { this.cnt = cnt; + this.size = size != null ? size : 0; nodes.add(firstNode); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java index 4a449d18a09d8..59624687b9606 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java @@ -93,6 +93,14 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa /** Serialized partitions that must be cleared and re-loaded. */ private byte[] partsToReloadBytes; + /** Partitions sizes. */ + @GridToStringInclude + @GridDirectTransient + private Map> partsSizes; + + /** Serialized partitions sizes. */ + private byte[] partsSizesBytes; + /** Topology version. */ private AffinityTopologyVersion topVer; @@ -164,6 +172,8 @@ public GridDhtPartitionsFullMessage(@Nullable GridDhtPartitionExchangeId id, cp.partHistSuppliersBytes = partHistSuppliersBytes; cp.partsToReload = partsToReload; cp.partsToReloadBytes = partsToReloadBytes; + cp.partsSizes = partsSizes; + cp.partsSizesBytes = partsSizesBytes; cp.topVer = topVer; cp.errs = errs; cp.errsBytes = errsBytes; @@ -331,6 +341,9 @@ public IgniteDhtPartitionHistorySuppliersMap partitionHistorySuppliers() { return partHistSuppliers; } + /** + * + */ public Set partsToReload(UUID nodeId, int grpId) { if (partsToReload == null) return Collections.emptySet(); @@ -338,6 +351,35 @@ public Set partsToReload(UUID nodeId, int grpId) { return partsToReload.get(nodeId, grpId); } + /** + * Adds partition sizes map for specified {@code grpId} to the current message. + * + * @param grpId Group id. + * @param partSizesMap Partition sizes map. + */ + public void addPartitionSizes(int grpId, Map partSizesMap) { + if (partSizesMap.isEmpty()) + return; + + if (partsSizes == null) + partsSizes = new HashMap<>(); + + partsSizes.put(grpId, partSizesMap); + } + + /** + * Returns partition sizes map for specified {@code grpId}. + * + * @param grpId Group id. + * @return Partition sizes map (partId, partSize). + */ + public Map partitionSizes(int grpId) { + if (partsSizes == null) + return Collections.emptyMap(); + + return partsSizes.getOrDefault(grpId, Collections.emptyMap()); + } + /** * @return Errors map. */ @@ -369,6 +411,7 @@ void setErrorsMap(Map errs) { byte[] partCntrsBytes20 = null; byte[] partHistSuppliersBytes0 = null; byte[] partsToReloadBytes0 = null; + byte[] partsSizesBytes0 = null; byte[] errsBytes0 = null; if (!F.isEmpty(parts) && partsBytes == null) @@ -386,6 +429,9 @@ void setErrorsMap(Map errs) { if (partsToReload != null && partsToReloadBytes == null) partsToReloadBytes0 = U.marshal(ctx, partsToReload); + if (partsSizes != null && partsSizesBytes == null) + partsSizesBytes0 = U.marshal(ctx, partsSizes); + if (!F.isEmpty(errs) && errsBytes == null) errsBytes0 = U.marshal(ctx, errs); @@ -398,6 +444,7 @@ void setErrorsMap(Map errs) { byte[] partCntrsBytes2Zip = U.zip(partCntrsBytes20); byte[] partHistSuppliersBytesZip = U.zip(partHistSuppliersBytes0); byte[] partsToReloadBytesZip = U.zip(partsToReloadBytes0); + byte[] partsSizesBytesZip = U.zip(partsSizesBytes0); byte[] exsBytesZip = U.zip(errsBytes0); partsBytes0 = partsBytesZip; @@ -405,6 +452,7 @@ void setErrorsMap(Map errs) { partCntrsBytes20 = partCntrsBytes2Zip; partHistSuppliersBytes0 = partHistSuppliersBytesZip; partsToReloadBytes0 = partsToReloadBytesZip; + partsSizesBytes0 = partsSizesBytesZip; errsBytes0 = exsBytesZip; compressed(true); @@ -419,6 +467,7 @@ void setErrorsMap(Map errs) { partCntrsBytes2 = partCntrsBytes20; partHistSuppliersBytes = partHistSuppliersBytes0; partsToReloadBytes = partsToReloadBytes0; + partsSizesBytes = partsSizesBytes0; errsBytes = errsBytes0; } } @@ -506,6 +555,13 @@ public void topologyVersion(AffinityTopologyVersion topVer) { partsToReload = U.unmarshal(ctx, partsToReloadBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); } + if (partsSizesBytes != null && partsSizes == null) { + if (compressed()) + partsSizes = U.unmarshalZip(ctx.marshaller(), partsSizesBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + else + partsSizes = U.unmarshal(ctx, partsSizesBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + } + if (partCntrs == null) partCntrs = new IgniteDhtPartitionCountersMap(); @@ -584,18 +640,24 @@ public void topologyVersion(AffinityTopologyVersion topVer) { writer.incrementState(); case 13: - if (!writer.writeByteArray("partsToReloadBytes", partsToReloadBytes)) + if (!writer.writeByteArray("partsSizesBytes", partsSizesBytes)) return false; writer.incrementState(); case 14: - if (!writer.writeMessage("resTopVer", resTopVer)) + if (!writer.writeByteArray("partsToReloadBytes", partsToReloadBytes)) return false; writer.incrementState(); case 15: + if (!writer.writeMessage("resTopVer", resTopVer)) + return false; + + writer.incrementState(); + + case 16: if (!writer.writeMessage("topVer", topVer)) return false; @@ -682,7 +744,7 @@ public void topologyVersion(AffinityTopologyVersion topVer) { reader.incrementState(); case 13: - partsToReloadBytes = reader.readByteArray("partsToReloadBytes"); + partsSizesBytes = reader.readByteArray("partsSizesBytes"); if (!reader.isLastRead()) return false; @@ -690,7 +752,7 @@ public void topologyVersion(AffinityTopologyVersion topVer) { reader.incrementState(); case 14: - resTopVer = reader.readMessage("resTopVer"); + partsToReloadBytes = reader.readByteArray("partsToReloadBytes"); if (!reader.isLastRead()) return false; @@ -698,6 +760,14 @@ public void topologyVersion(AffinityTopologyVersion topVer) { reader.incrementState(); case 15: + resTopVer = reader.readMessage("resTopVer"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 16: topVer = reader.readMessage("topVer"); if (!reader.isLastRead()) @@ -717,7 +787,7 @@ public void topologyVersion(AffinityTopologyVersion topVer) { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 16; + return 17; } /** {@inheritDoc} */ 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 b60070e6b4dd2..804cc030489f7 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 @@ -70,7 +70,7 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes /** Partitions sizes. */ @GridToStringInclude @GridDirectTransient - private Map> partSizes; + private Map> partsSizes; /** Serialized partitions counters. */ private byte[] partsSizesBytes; @@ -237,10 +237,10 @@ public void addPartitionSizes(int grpId, Map partSizesMap) { if (partSizesMap.isEmpty()) return; - if (partSizes == null) - partSizes = new HashMap<>(); + if (partsSizes == null) + partsSizes = new HashMap<>(); - partSizes.put(grpId, partSizesMap); + partsSizes.put(grpId, partSizesMap); } /** @@ -250,10 +250,10 @@ public void addPartitionSizes(int grpId, Map partSizesMap) { * @return Partition sizes map (partId, partSize). */ public Map partitionSizes(int grpId) { - if (partSizes == null) + if (partsSizes == null) return Collections.emptyMap(); - return partSizes.getOrDefault(grpId, Collections.emptyMap()); + return partsSizes.getOrDefault(grpId, Collections.emptyMap()); } /** @@ -324,7 +324,7 @@ public void setError(Exception ex) { boolean marshal = (parts != null && partsBytes == null) || (partCntrs != null && partCntrsBytes == null) || (partHistCntrs != null && partHistCntrsBytes == null) || - (partSizes != null && partsSizesBytes == null) || + (partsSizes != null && partsSizesBytes == null) || (err != null && errBytes == null); if (marshal) { @@ -343,8 +343,8 @@ public void setError(Exception ex) { if (partHistCntrs != null && partHistCntrsBytes == null) partHistCntrsBytes0 = U.marshal(ctx, partHistCntrs); - if (partSizes != null && partsSizesBytes == null) - partSizesBytes0 = U.marshal(ctx, partSizes); + if (partsSizes != null && partsSizesBytes == null) + partSizesBytes0 = U.marshal(ctx, partsSizes); if (err != null && errBytes == null) errBytes0 = U.marshal(ctx, err); @@ -405,11 +405,11 @@ public void setError(Exception ex) { partHistCntrs = U.unmarshal(ctx, partHistCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); } - if (partsSizesBytes != null && partSizes == null) { + if (partsSizesBytes != null && partsSizes == null) { if (compressed()) - partSizes = U.unmarshalZip(ctx.marshaller(), partsSizesBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + partsSizes = U.unmarshalZip(ctx.marshaller(), partsSizesBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); else - partSizes = U.unmarshal(ctx, partsSizesBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); + partsSizes = U.unmarshal(ctx, partsSizesBytes, U.resolveClassLoader(ldr, ctx.gridConfig())); } if (errBytes != null && err == null) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java index 818fd67ebe63d..d930c6bf89b3f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java @@ -1504,6 +1504,8 @@ public static void writeCacheMetrics(BinaryRawWriter writer, CacheMetrics metric writer.writeLong(metrics.getRebalancingStartTime()); writer.writeLong(metrics.getRebalanceClearingPartitionsLeft()); writer.writeLong(metrics.getCacheSize()); + writer.writeLong(metrics.getRebalancedKeys()); + writer.writeLong(metrics.getEstimatedRebalancingKeys()); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java index 59f16b2a5e7cd..854bbd7ad1e2c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java @@ -175,6 +175,12 @@ public class VisorCacheMetrics extends VisorDataTransferObject { /** Total number of partitions on current node. */ private int totalPartsCnt; + /** Number of already rebalanced keys. */ + private long rebalancedKeys; + + /** Number estimated to rebalance keys. */ + private long estimatedRebalancingKeys; + /** Number of currently rebalancing partitions on current node. */ private int rebalancingPartsCnt; @@ -276,6 +282,8 @@ public VisorCacheMetrics(IgniteEx ignite, String cacheName) { offHeapPrimaryEntriesCnt = m.getOffHeapPrimaryEntriesCount(); totalPartsCnt = m.getTotalPartitionsCount(); + rebalancedKeys = m.getRebalancedKeys(); + estimatedRebalancingKeys = m.getEstimatedRebalancingKeys(); rebalancingPartsCnt = m.getRebalancingPartitionsCount(); keysToRebalanceLeft = m.getKeysToRebalanceLeft(); rebalancingKeysRate = m.getRebalancingKeysRate(); @@ -622,6 +630,20 @@ public int getTotalPartitionsCount() { return totalPartsCnt; } + /** + * @return Number of already rebalanced keys. + */ + public long getRebalancedKeys() { + return rebalancedKeys; + } + + /** + * @return Number estimated to rebalance keys. + */ + public long getEstimatedRebalancingKeys() { + return estimatedRebalancingKeys; + } + /** * @return Number of currently rebalancing partitions on current node. */ @@ -650,6 +672,11 @@ public long getRebalancingBytesRate() { return rebalancingBytesRate; } + /** {@inheritDoc} */ + @Override public byte getProtocolVersion() { + return V2; + } + /** {@inheritDoc} */ @Override protected void writeExternalData(ObjectOutput out) throws IOException { U.writeString(out, name); @@ -708,6 +735,9 @@ public long getRebalancingBytesRate() { out.writeObject(qryMetrics); out.writeLong(cacheSize); + + out.writeLong(rebalancedKeys); + out.writeLong(estimatedRebalancingKeys); } /** {@inheritDoc} */ @@ -768,6 +798,11 @@ public long getRebalancingBytesRate() { if (in.available() > 0) cacheSize = in.readLong(); + + if (protoVer > V1) { + rebalancedKeys = in.readLong(); + estimatedRebalancingKeys = in.readLong(); + } } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java index fda23a2a917c6..14b928164978c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java @@ -183,8 +183,9 @@ protected void caches(VisorNodeDataCollectorJobResult res, VisorNodeDataCollecto List resCaches = res.getCaches(); + int partitions = 0; double total = 0; - double moving = 0; + double ready = 0; for (String cacheName : cacheProc.cacheNames()) { if (proxyCache(cacheName)) @@ -201,8 +202,16 @@ protected void caches(VisorNodeDataCollectorJobResult res, VisorNodeDataCollecto CacheMetrics cm = ca.localMetrics(); - total += cm.getTotalPartitionsCount(); - moving += cm.getRebalancingPartitionsCount(); + partitions += cm.getTotalPartitionsCount(); + + long partTotal = cm.getEstimatedRebalancingKeys(); + long partReady = cm.getRebalancedKeys(); + + if (partReady >= partTotal) + partReady = Math.max(partTotal - 1, 0); + + total += partTotal; + ready += partReady; resCaches.add(new VisorCache(ignite, ca, arg.isCollectCacheMetrics())); } @@ -217,7 +226,10 @@ protected void caches(VisorNodeDataCollectorJobResult res, VisorNodeDataCollecto } } - res.setRebalance(total > 0 ? (total - moving) / total : -1); + if (partitions == 0) + res.setRebalance(-1); + else + res.setRebalance(total > 0 ? ready / total : 1); } catch (Exception e) { res.setCachesEx(new VisorExceptionWrapper(e)); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupsMetricsRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupsMetricsRebalanceTest.java index ceb98522d5a25..b81bdff154d13 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupsMetricsRebalanceTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGroupsMetricsRebalanceTest.java @@ -31,6 +31,7 @@ import org.apache.ignite.events.CacheRebalancingEvent; import org.apache.ignite.events.Event; import org.apache.ignite.events.EventType; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.PA; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgnitePredicate; @@ -159,12 +160,12 @@ public void testRebalance() throws Exception { assertTrue(rate1 > 0); assertTrue(rate2 > 0); - // rate1 has to be roughly twice more than rate2. - double ratio = ((double)rate2 / rate1) * 100; + // rate1 has to be roughly the same as rate2 + double ratio = ((double)rate2 / rate1); log.info("Ratio: " + ratio); - assertTrue(ratio > 40 && ratio < 60); + assertTrue(ratio > 0.9 && ratio < 1.1); } /** @@ -177,8 +178,6 @@ public void testRebalanceEstimateFinishTime() throws Exception { final int KEYS = 4_000_000; - IgniteCache cache1 = ig1.cache(CACHE1); - try (IgniteDataStreamer st = ig1.dataStreamer(CACHE1)) { for (int i = 0; i < KEYS; i++) st.addData(i, CACHE1 + "-" + i); @@ -211,10 +210,10 @@ public void testRebalanceEstimateFinishTime() throws Exception { CacheMetrics metrics = ig2.cache(CACHE1).localMetrics(); long startTime = metrics.getRebalancingStartTime(); + long currTime = U.currentTimeMillis(); - assertTrue(startTime > 0); - assertTrue((U.currentTimeMillis() - startTime) < 5000); - assertTrue((U.currentTimeMillis() - startTime) > 0); + assertTrue("Invalid start time [startTime=" + startTime + ", currTime=" + currTime + ']', + startTime > 0L && (currTime - startTime) >= 0L && (currTime - startTime) <= 5000L); final CountDownLatch latch = new CountDownLatch(1); @@ -223,47 +222,62 @@ public void testRebalanceEstimateFinishTime() throws Exception { // Waiting 25% keys will be rebalanced. int partKeys = KEYS / 2; - final long keysLine = (long)(partKeys - (partKeys * 0.25)); + final long keysLine = partKeys * 3L / 4L; System.out.println("Wait until keys left will be less " + keysLine); - try { - while (finishRebalanceLatch.getCount() != 0) { - CacheMetrics m = ig2.cache(CACHE1).localMetrics(); + while (true) { + CacheMetrics m = ig2.cache(CACHE1).localMetrics(); - long keyLeft = m.getKeysToRebalanceLeft(); + long keyLeft = m.getKeysToRebalanceLeft(); - if (keyLeft > 0 && keyLeft < keysLine) - latch.countDown(); + if (keyLeft > 0 && keyLeft < keysLine) { + latch.countDown(); - System.out.println("Keys left: " + m.getKeysToRebalanceLeft()); + break; + } - try { - Thread.sleep(1_000); - } - catch (InterruptedException e) { - System.out.println("Interrupt thread: " + e.getMessage()); + System.out.println("Keys left: " + m.getKeysToRebalanceLeft()); - Thread.currentThread().interrupt(); - } + try { + Thread.sleep(1_000); + } + catch (InterruptedException e) { + System.out.println("Interrupt thread: " + e.getMessage()); + + Thread.currentThread().interrupt(); } - } - finally { - latch.countDown(); } } }); assertTrue(latch.await(getTestTimeout(), TimeUnit.MILLISECONDS)); + waitForCondition(new PA() { + @Override public boolean apply() { + return ig2.cache(CACHE1).localMetrics().getEstimatedRebalancingFinishTime() != -1L; + } + }, 5_000L); + long finishTime = ig2.cache(CACHE1).localMetrics().getEstimatedRebalancingFinishTime(); - assertTrue(finishTime > 0); + assertTrue("Not a positive estimation of rebalancing finish time: " + finishTime, + finishTime > 0L); - long timePassed = U.currentTimeMillis() - startTime; - long timeLeft = finishTime - System.currentTimeMillis(); + currTime = U.currentTimeMillis(); - assertTrue(finishRebalanceLatch.await(timeLeft + 2_000, TimeUnit.SECONDS)); + long timePassed = currTime - startTime; + long timeLeft = finishTime - currTime; + + // TODO: finishRebalanceLatch gets countdown much earlier because of ForceRebalanceExchangeTask triggered by cache with delay +// assertTrue("Got timeout while waiting for rebalancing. Estimated left time: " + timeLeft, +// finishRebalanceLatch.await(timeLeft + 10_000L, TimeUnit.MILLISECONDS)); + + waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return ig2.cache(CACHE1).localMetrics().getKeysToRebalanceLeft() == 0; + } + }, timeLeft + 10_000L); System.out.println( "TimePassed:" + timePassed + @@ -275,11 +289,13 @@ public void testRebalanceEstimateFinishTime() throws Exception { System.clearProperty(IGNITE_REBALANCE_STATISTICS_TIME_INTERVAL); - System.out.println("Rebalance time:" + (U.currentTimeMillis() - startTime)); + currTime = U.currentTimeMillis(); + + log.info("Rebalance time: " + (currTime - startTime)); - long diff = finishTime - U.currentTimeMillis(); + long diff = finishTime - currTime; - assertTrue("Expected less 5000, Actual:" + diff, Math.abs(diff) < 10_000); + assertTrue("Expected less than 10000, but actual: " + diff, Math.abs(diff) < 10_000L); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java index 44a15d500ca44..fe61e35b11dd1 100644 --- a/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java +++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java @@ -473,5 +473,15 @@ private static class TestCacheMetrics implements CacheMetrics { @Override public long getCacheSize() { return 65; } + + /** {@inheritDoc} */ + @Override public long getRebalancedKeys() { + return 66; + } + + /** {@inheritDoc} */ + @Override public long getEstimatedRebalancingKeys() { + return 67; + } } } diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs index d775c05691076..e0e7301c4d233 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs @@ -654,5 +654,23 @@ public interface ICacheMetrics /// Number of clearing partitions for rebalance. /// long RebalanceClearingPartitionsLeft { get; } + + /// + /// Gets number of already rebalanced keys. + /// need to be cleared before actual rebalance start. + /// + /// + /// Number of already rebalanced keys. + /// + long RebalancedKeys { get; } + + /// + /// Gets number of estimated keys to rebalance. + /// need to be cleared before actual rebalance start. + /// + /// + /// Number of estimated keys to rebalance. + /// + long EstimatedRebalancingKeys { get; } } } \ No newline at end of file diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs index 1fdc8771f832e..be6980d7cdd97 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs @@ -247,6 +247,12 @@ internal class CacheMetricsImpl : ICacheMetrics /** */ private readonly long _rebalancingClearingPartitionsLeft; + /** */ + private readonly long _rebalancedKeys; + + /** */ + private readonly long _estimatedRebalancedKeys; + /// /// Initializes a new instance of the class. /// @@ -327,6 +333,8 @@ public CacheMetricsImpl(IBinaryRawReader reader) _rebalancingStartTime = reader.ReadLong(); _rebalancingClearingPartitionsLeft = reader.ReadLong(); _cacheSize = reader.ReadLong(); + _rebalancedKeys = reader.ReadLong(); + _estimatedRebalancedKeys = reader.ReadLong(); } /** */ @@ -550,5 +558,11 @@ public CacheMetricsImpl(IBinaryRawReader reader) /** */ public long RebalanceClearingPartitionsLeft { get { return _rebalancingClearingPartitionsLeft; } } + + /** */ + public long RebalancedKeys { get { return _rebalancedKeys; } } + + /** */ + public long EstimatedRebalancingKeys { get { return _estimatedRebalancedKeys; } } } } \ No newline at end of file From 3360c94fccba556f67f59dec5ebc4969eb6910b5 Mon Sep 17 00:00:00 2001 From: Aleksei Scherbakov Date: Tue, 19 Jun 2018 20:12:13 +0300 Subject: [PATCH 0285/1463] IGNITE-8808 Improve control.sh --tx command to show local and remote transactions. - Fixes #4209. Signed-off-by: Alexey Goncharuk --- .../internal/commandline/CommandHandler.java | 18 +- .../ignite/internal/visor/tx/VisorTxInfo.java | 41 ++- .../ignite/internal/visor/tx/VisorTxTask.java | 212 ++++++++++-- .../TestRecordingCommunicationSpi.java | 42 ++- .../ignite/util/GridCommandHandlerTest.java | 303 +++++++++++++++++- 5 files changed, 559 insertions(+), 57 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java index ed85f0c850ba9..cf0569920bde4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java @@ -1088,11 +1088,19 @@ else if (arg.getOperation() == VisorTxOperation.KILL) ", concurrency=" + info.getConcurrency() + ", timeout=" + info.getTimeout() + ", size=" + info.getSize() + - ", dhtNodes=" + F.transform(info.getPrimaryNodes(), new IgniteClosure() { - @Override public String apply(UUID id) { - return U.id8(id); - } - }) + + ", dhtNodes=" + (info.getPrimaryNodes() == null ? "N/A" : + F.transform(info.getPrimaryNodes(), new IgniteClosure() { + @Override public String apply(UUID id) { + return U.id8(id); + } + })) + + ", nearXid=" + info.getNearXid() + + ", parentNodeIds=" + (info.getMasterNodeIds() == null ? "N/A" : + F.transform(info.getMasterNodeIds(), new IgniteClosure() { + @Override public String apply(UUID id) { + return U.id8(id); + } + })) + ']'); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxInfo.java index ecf3e0d79fb4c..03de5b04a9c1a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxInfo.java @@ -33,6 +33,7 @@ import org.apache.ignite.transactions.TransactionConcurrency; import org.apache.ignite.transactions.TransactionIsolation; import org.apache.ignite.transactions.TransactionState; +import org.jetbrains.annotations.Nullable; /** */ @@ -75,6 +76,12 @@ public class VisorTxInfo extends VisorDataTransferObject { /** */ private int size; + /** */ + private IgniteUuid nearXid; + + /** */ + private Collection masterNodeIds; + /** * Default constructor. */ @@ -96,7 +103,7 @@ public VisorTxInfo() { */ public VisorTxInfo(IgniteUuid xid, long startTime, long duration, TransactionIsolation isolation, TransactionConcurrency concurrency, long timeout, String lb, Collection primaryNodes, - TransactionState state, int size) { + TransactionState state, int size, IgniteUuid nearXid, Collection masterNodeIds) { this.xid = xid; this.startTime = startTime; this.duration = duration; @@ -107,6 +114,13 @@ public VisorTxInfo(IgniteUuid xid, long startTime, long duration, TransactionIso this.primaryNodes = primaryNodes; this.state = state; this.size = size; + this.nearXid = nearXid; + this.masterNodeIds = masterNodeIds; + } + + /** {@inheritDoc} */ + @Override public byte getProtocolVersion() { + return V2; } /** */ @@ -164,9 +178,14 @@ public int getSize() { return size; } - /** {@inheritDoc} */ - @Override public byte getProtocolVersion() { - return V2; + /** */ + public @Nullable IgniteUuid getNearXid() { + return nearXid; + } + + /** */ + public @Nullable Collection getMasterNodeIds() { + return masterNodeIds; } /** {@inheritDoc} */ @@ -180,12 +199,13 @@ public int getSize() { U.writeCollection(out, primaryNodes); U.writeEnum(out, state); out.writeInt(size); + U.writeGridUuid(out, nearXid); + U.writeCollection(out, masterNodeIds); out.writeLong(startTime); } /** {@inheritDoc} */ - @Override protected void readExternalData(byte protoVer, - ObjectInput in) throws IOException, ClassNotFoundException { + @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException { xid = U.readGridUuid(in); duration = in.readLong(); isolation = TransactionIsolation.fromOrdinal(in.readByte()); @@ -195,7 +215,14 @@ public int getSize() { primaryNodes = U.readCollection(in); state = TransactionState.fromOrdinal(in.readByte()); size = in.readInt(); - startTime = protoVer >= V2 ? in.readLong() : 0L; + if (protoVer >= V2) { + nearXid = U.readGridUuid(in); + + masterNodeIds = U.readCollection(in); + + startTime = in.readLong(); + } + } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java index 25a69d1d67e06..9919b7d654458 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/tx/VisorTxTask.java @@ -21,7 +21,8 @@ import java.util.Collection; import java.util.Collections; import java.util.Comparator; -import java.util.ConcurrentModificationException; +import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.TreeMap; @@ -31,22 +32,32 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.compute.ComputeJobResult; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxMapping; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocal; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxRemote; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; import org.apache.ignite.internal.processors.cache.distributed.near.IgniteTxMappings; -import org.apache.ignite.internal.processors.cache.transactions.TransactionProxyImpl; +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.transactions.IgniteTxRemoteEx; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.task.GridInternal; +import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.visor.VisorJob; import org.apache.ignite.internal.visor.VisorMultiNodeTask; import org.apache.ignite.internal.visor.VisorTaskArgument; +import org.apache.ignite.lang.IgniteBiClosure; import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgnitePredicate; -import org.apache.ignite.lang.IgniteUuid; -import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionState; import org.jetbrains.annotations.Nullable; +import static org.apache.ignite.transactions.TransactionState.COMMITTED; +import static org.apache.ignite.transactions.TransactionState.COMMITTING; + /** * */ @@ -103,6 +114,8 @@ public class VisorTxTask extends VisorMultiNodeTask reduce0(List results) throws IgniteException { Map mapRes = new TreeMap<>(); + Map nodeMap = new HashMap<>(); + for (ComputeJobResult result : results) { VisorTxTaskResult data = result.getData(); @@ -110,6 +123,47 @@ public class VisorTxTask extends VisorMultiNodeTask infos = result.getInfos(); + + Iterator it = infos.iterator(); + + while (it.hasNext()) { + VisorTxInfo info = it.next(); + + if (!info.getXid().equals(info.getNearXid())) { + UUID nearNodeId = info.getMasterNodeIds().iterator().next(); + + // Try find id. + ClusterNode node = nodeMap.get(nearNodeId); + + if (node == null) + continue; + + VisorTxTaskResult res0 = mapRes.get(node); + + if (res0 == null) + continue; + + boolean exists = false; + + for (VisorTxInfo txInfo : res0.getInfos()) { + if (txInfo.getXid().equals(info.getNearXid())) { + exists = true; + + break; + } + } + + if (exists) + it.remove(); + } + } } return mapRes; @@ -123,7 +177,16 @@ private static class VisorTxJob extends VisorJob transactions = ignite.transactions().localActiveTransactions(); + IgniteTxManager tm = ignite.context().cache().context().tm(); + + Collection transactions = tm.activeTransactions(); List infos = new ArrayList<>(); @@ -155,50 +220,102 @@ private VisorTxJob(VisorTxTaskArg arg, boolean debug) { } } - for (Transaction transaction : transactions) { - GridNearTxLocal locTx = ((TransactionProxyImpl)transaction).tx(); - + for (IgniteInternalTx locTx : transactions) { if (arg.getXid() != null && !locTx.xid().toString().equals(arg.getXid())) continue; if (arg.getState() != null && locTx.state() != arg.getState()) continue; - long duration = U.currentTimeMillis() - transaction.startTime(); + long duration = U.currentTimeMillis() - locTx.startTime(); - if (arg.getMinDuration() != null && - duration < arg.getMinDuration()) + if (arg.getMinDuration() != null && duration < arg.getMinDuration()) continue; - if (lbMatch != null && !lbMatch.matcher(locTx.label() == null ? "null" : locTx.label()).matches()) - continue; + String lb = null; + int size = 0; + Collection mappings = null; + TxKillClosure killClo = null; - Collection mappings = new ArrayList<>(); + // This filter conditions have meaning only for near txs, so we skip dht because it will never match. + boolean skip = arg.getMinSize() != null || lbMatch != null; - int size = 0; + if (locTx instanceof GridNearTxLocal) { + GridNearTxLocal locTx0 = (GridNearTxLocal)locTx; + + lb = locTx0.label(); + + if (lbMatch != null && !lbMatch.matcher(lb == null ? "null" : lb).matches()) + continue; + + mappings = new ArrayList<>(); + + if (locTx0.mappings() != null) { + IgniteTxMappings txMappings = locTx0.mappings(); + + for (GridDistributedTxMapping mapping : + txMappings.single() ? Collections.singleton(txMappings.singleMapping()) : txMappings.mappings()) { + if (mapping == null) + continue; - if (locTx.mappings() != null) { - IgniteTxMappings txMappings = locTx.mappings(); + mappings.add(mapping.primary().id()); - for (GridDistributedTxMapping mapping : - txMappings.single() ? Collections.singleton(txMappings.singleMapping()) : txMappings.mappings()) { - if (mapping == null) - continue; + size += mapping.entries().size(); // Entries are not synchronized so no visibility guaranties for size. + } + } + + if (arg.getMinSize() != null && size < arg.getMinSize()) + continue; + + killClo = NEAR_KILL_CLOSURE; + } + else if (locTx instanceof GridDhtTxLocal) { + if (skip) + continue; + + GridDhtTxLocal locTx0 = (GridDhtTxLocal)locTx; + + Map dhtMap = U.field(locTx0, "dhtMap"); + + mappings = new ArrayList<>(); + + if (dhtMap != null) { + for (GridDistributedTxMapping mapping : dhtMap.values()) { + mappings.add(mapping.primary().id()); + + size += mapping.entries().size(); + } + } + + Map nearMap = U.field(locTx, "nearMap"); - mappings.add(mapping.primary().id()); + if (nearMap != null) { + for (GridDistributedTxMapping mapping : nearMap.values()) { + mappings.add(mapping.primary().id()); - size += mapping.entries().size(); // Entries are not synchronized so no visibility guaranties for size. + size += mapping.entries().size(); + } } + + killClo = LOCAL_KILL_CLOSURE; } + else if (locTx instanceof GridDhtTxRemote) { + if (skip) + continue; - if (arg.getMinSize() != null && size < arg.getMinSize()) - continue; + GridDhtTxRemote locTx0 = (GridDhtTxRemote)locTx; + + size = locTx0.readMap().size() + locTx.writeMap().size(); + + killClo = REMOTE_KILL_CLOSURE; + } infos.add(new VisorTxInfo(locTx.xid(), locTx.startTime(), duration, locTx.isolation(), locTx.concurrency(), - locTx.timeout(), locTx.label(), mappings, locTx.state(), size)); + locTx.timeout(), lb, mappings, locTx.state(), + size, locTx.nearXidVersion().asGridUuid(), locTx.masterNodeIds())); if (arg.getOperation() == VisorTxOperation.KILL) - locTx.rollbackAsync(); + killClo.apply(locTx, tm); if (infos.size() == limit) break; @@ -271,4 +388,43 @@ private static class TxSizeComparator implements Comparator { return Long.compare(o2.getSize(), o1.getSize()); } } + + /** Type shortcut. */ + private interface TxKillClosure extends + IgniteBiClosure> { + } + + /** Kills near or local tx. */ + private static class NearKillClosure implements TxKillClosure { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture apply(IgniteInternalTx tx, IgniteTxManager tm) { + return tx.isRollbackOnly() || tx.state() == COMMITTING || tx.state() == COMMITTED ? + new GridFinishedFuture<>() : tx.rollbackAsync(); + } + } + + /** Kills remote tx. */ + private static class RemoteKillClosure implements TxKillClosure { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public IgniteInternalFuture apply(IgniteInternalTx tx, IgniteTxManager tm) { + IgniteTxRemoteEx remote = (IgniteTxRemoteEx)tx; + + if (tx.isRollbackOnly() || tx.state() == COMMITTING || tx.state() == COMMITTED) + return new GridFinishedFuture<>(); + + if (tx.state() == TransactionState.PREPARED) + remote.doneRemote(tx.xidVersion(), + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList()); + + return tx.rollbackAsync(); + } + } } 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 5d12d9aea9ff7..b36bf16dc9fdb 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 @@ -18,6 +18,7 @@ package org.apache.ignite.internal; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -29,13 +30,16 @@ import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.managers.communication.GridIoMessage; +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.IgniteBiPredicate; import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGNITE_INSTANCE_NAME; @@ -198,6 +202,18 @@ public void waitForBlocked() throws InterruptedException { } } + /** + * @param cnt Number of messages to wait. + * + * @throws InterruptedException If interrupted. + */ + public void waitForBlocked(int cnt) throws InterruptedException { + synchronized (this) { + while (blockedMsgs.size() < cnt) + wait(); + } + } + /** * @throws InterruptedException If interrupted. */ @@ -251,26 +267,38 @@ public void blockMessages(Class cls, String nodeName) { } /** - * Stops block messages and can sends all already blocked messages. + * Stops block messages and sends all already blocked messages. */ public void stopBlock() { - stopBlock(true); + stopBlock(true, null); } /** * Stops block messages and sends all already blocked messages if sndMsgs is 'true'. * - * @param sndMsgs If {@code true} sends blocked messages. + * @param sndMsgs {@code True} to send blocked messages. */ public void stopBlock(boolean sndMsgs) { - synchronized (this) { - blockP = null; + stopBlock(sndMsgs, null); + } + /** + * Stops block messages and sends all already blocked messages if sndMsgs is 'true' optionally filtered + * by unblockPred. + * + * @param sndMsgs If {@code true} sends blocked messages. + * @param unblockPred If not null unblocks only messages allowed by predicate. + */ + public void stopBlock(boolean sndMsgs, @Nullable IgnitePredicate> unblockPred) { + synchronized (this) { blockCls.clear(); blockP = null; + Collection> msgs = + unblockPred == null ? blockedMsgs : F.view(blockedMsgs, unblockPred); + if (sndMsgs) { - for (T2 msg : blockedMsgs) { + for (T2 msg : msgs) { try { ignite.log().info("Send blocked message [node=" + msg.get1().id() + ", order=" + msg.get1().order() + @@ -284,7 +312,7 @@ public void stopBlock(boolean sndMsgs) { } } - blockedMsgs.clear(); + msgs.clear(); } } } diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java index 467e04620521e..5917d9159353e 100644 --- a/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java +++ b/modules/core/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java @@ -24,6 +24,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.TreeMap; import java.util.concurrent.CountDownLatch; @@ -31,7 +32,10 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import java.util.List; +import java.util.concurrent.atomic.LongAdder; +import javax.cache.processor.EntryProcessor; +import javax.cache.processor.EntryProcessorException; +import javax.cache.processor.MutableEntry; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteAtomicSequence; import org.apache.ignite.IgniteCache; @@ -46,26 +50,45 @@ import org.apache.ignite.configuration.WALMode; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.TestRecordingCommunicationSpi; import org.apache.ignite.internal.commandline.CommandHandler; import org.apache.ignite.internal.commandline.cache.CacheCommand; +import org.apache.ignite.internal.managers.communication.GridIoMessage; import org.apache.ignite.internal.pagemem.wal.record.DataEntry; import org.apache.ignite.internal.processors.cache.CacheObjectImpl; import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; import org.apache.ignite.internal.processors.cache.GridCacheFuture; +import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; +import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockResponse; +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.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; +import org.apache.ignite.internal.processors.cache.transactions.TransactionProxyImpl; import org.apache.ignite.internal.processors.cache.GridCacheOperation; import org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl; import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.visor.tx.VisorTxInfo; import org.apache.ignite.internal.visor.tx.VisorTxTaskResult; +import org.apache.ignite.lang.IgniteBiPredicate; import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.plugin.extensions.communication.Message; +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.TransactionRollbackException; +import org.apache.ignite.transactions.TransactionTimeoutException; import static org.apache.ignite.IgniteSystemProperties.IGNITE_ENABLE_EXPERIMENTAL_COMMAND; import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; @@ -137,10 +160,12 @@ protected void injectTestSystemOut() { @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + cfg.setCommunicationSpi(new TestRecordingCommunicationSpi()); + cfg.setConnectorConfiguration(new ConnectorConfiguration()); DataStorageConfiguration memCfg = new DataStorageConfiguration().setDefaultDataRegionConfiguration( - new DataRegionConfiguration().setMaxSize(100 * 1024 * 1024)); + new DataRegionConfiguration().setMaxSize(100L * 1024 * 1024)); cfg.setDataStorageConfiguration(memCfg); @@ -150,7 +175,7 @@ protected void injectTestSystemOut() { cfg.setConsistentId(igniteInstanceName); - cfg.setClientMode("client".equals(igniteInstanceName)); + cfg.setClientMode(igniteInstanceName.startsWith("client")); return cfg; } @@ -469,26 +494,22 @@ else if (entry.getKey().equals(node2)) { assertNotNull(res); - for (VisorTxInfo txInfo : res.getInfos()) { + for (VisorTxInfo txInfo : res.getInfos()) assertTrue(txInfo.getSize() >= minSize); - - } }, "--tx", "minSize", Integer.toString(minSize)); // test order by size. validate(h, map -> { VisorTxTaskResult res = map.get(grid(0).localNode()); - assertTrue(res.getInfos().get(0).getSize() >= res.getInfos().get(1).getSize()); - + assertTrue(res.getInfos().get(0).getSize() >= res.getInfos().get(1).getSize()); }, "--tx", "order", "SIZE"); // test order by duration. validate(h, map -> { VisorTxTaskResult res = map.get(grid(0).localNode()); - assertTrue(res.getInfos().get(0).getDuration() >= res.getInfos().get(1).getDuration()); - + assertTrue(res.getInfos().get(0).getDuration() >= res.getInfos().get(1).getDuration()); }, "--tx", "order", "DURATION"); // test order by start_time. @@ -537,6 +558,251 @@ else if (entry.getKey().equals(node2)) { checkFutures(); } + /** + * + */ + public void testKillHangingLocalTransactions() throws Exception { + Ignite ignite = startGridsMultiThreaded(2); + + ignite.cluster().active(true); + + Ignite client = startGrid("client"); + + client.getOrCreateCache(new CacheConfiguration<>(DEFAULT_CACHE_NAME). + setAtomicityMode(TRANSACTIONAL). + setWriteSynchronizationMode(FULL_SYNC). + setAffinity(new RendezvousAffinityFunction(false, 64))); + + Ignite prim = primaryNode(0L, DEFAULT_CACHE_NAME); + + // Blocks lock response to near node. + TestRecordingCommunicationSpi.spi(prim).blockMessages(GridNearLockResponse.class, client.name()); + + TestRecordingCommunicationSpi.spi(client).blockMessages(GridNearTxFinishRequest.class, prim.name()); + + GridNearTxLocal clientTx = null; + + try(Transaction tx = client.transactions().txStart(PESSIMISTIC, READ_COMMITTED, 2000, 1)) { + clientTx = ((TransactionProxyImpl)tx).tx(); + + client.cache(DEFAULT_CACHE_NAME).put(0L, 0L); + + fail(); + } + catch (Exception e) { + assertTrue(X.hasCause(e, TransactionTimeoutException.class)); + } + + assertNotNull(clientTx); + + IgniteEx primEx = (IgniteEx)prim; + + IgniteInternalTx tx0 = primEx.context().cache().context().tm().activeTransactions().iterator().next(); + + assertNotNull(tx0); + + CommandHandler h = new CommandHandler(); + + validate(h, map -> { + ClusterNode node = grid(0).cluster().localNode(); + + VisorTxTaskResult res = map.get(node); + + for (VisorTxInfo info : res.getInfos()) + assertEquals(tx0.xid(), info.getXid()); + + assertEquals(1, map.size()); + }, "--tx", "kill"); + + tx0.finishFuture().get(); + + TestRecordingCommunicationSpi.spi(prim).stopBlock(); + + TestRecordingCommunicationSpi.spi(client).stopBlock(); + + IgniteInternalFuture nearFinFut = U.field(clientTx, "finishFut"); + + nearFinFut.get(); + + checkFutures(); + } + + /** + * Simulate uncommitted backup transactions and test rolling back using utility. + */ + public void testKillHangingRemoteTransactions() throws Exception { + final int cnt = 3; + + startGridsMultiThreaded(cnt); + + Ignite[] clients = new Ignite[] { + startGrid("client1"), + startGrid("client2"), + startGrid("client3"), + startGrid("client4") + }; + + clients[0].getOrCreateCache(new CacheConfiguration<>(DEFAULT_CACHE_NAME). + setBackups(2). + setAtomicityMode(TRANSACTIONAL). + setWriteSynchronizationMode(FULL_SYNC). + setAffinity(new RendezvousAffinityFunction(false, 64))); + + for (Ignite client : clients) { + assertTrue(client.configuration().isClientMode()); + + assertNotNull(client.cache(DEFAULT_CACHE_NAME)); + } + + LongAdder progress = new LongAdder(); + + AtomicInteger idx = new AtomicInteger(); + + int tc = clients.length; + + CountDownLatch lockLatch = new CountDownLatch(1); + CountDownLatch commitLatch = new CountDownLatch(1); + + Ignite prim = primaryNode(0L, DEFAULT_CACHE_NAME); + + TestRecordingCommunicationSpi primSpi = TestRecordingCommunicationSpi.spi(prim); + + primSpi.blockMessages(new IgniteBiPredicate() { + @Override public boolean apply(ClusterNode node, Message message) { + return message instanceof GridDhtTxFinishRequest; + } + }); + + IgniteInternalFuture fut = multithreadedAsync(new Runnable() { + @Override public void run() { + int id = idx.getAndIncrement(); + + Ignite client = clients[id]; + + try (Transaction tx = client.transactions().txStart(PESSIMISTIC, READ_COMMITTED, 0, 1)) { + IgniteCache cache = client.cache(DEFAULT_CACHE_NAME); + + if (id != 0) + U.awaitQuiet(lockLatch); + + cache.invoke(0L, new IncrementClosure(), null); + + if (id == 0) { + lockLatch.countDown(); + + U.awaitQuiet(commitLatch); + + doSleep(500); // Wait until candidates will enqueue. + } + + tx.commit(); + } + catch (Exception e) { + assertTrue(X.hasCause(e, TransactionTimeoutException.class)); + } + + progress.increment(); + + } + }, tc, "invoke-thread"); + + U.awaitQuiet(lockLatch); + + commitLatch.countDown(); + + primSpi.waitForBlocked(clients.length); + + // Unblock only finish messages from clients from 2 to 4. + primSpi.stopBlock(true, new IgnitePredicate>() { + @Override public boolean apply(T2 objects) { + GridIoMessage iom = objects.get2(); + + Message m = iom.message(); + + if (m instanceof GridDhtTxFinishRequest) { + GridDhtTxFinishRequest r = (GridDhtTxFinishRequest)m; + + if (r.nearNodeId().equals(clients[0].cluster().localNode().id())) + return false; + } + + return true; + } + }); + + // Wait until queue is stable + for (Ignite ignite : G.allGrids()) { + if (ignite.configuration().isClientMode()) + continue; + + Collection txs = ((IgniteEx)ignite).context().cache().context().tm().activeTransactions(); + + GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + for (IgniteInternalTx tx : txs) + if (!tx.local()) { + IgniteTxEntry entry = tx.writeEntries().iterator().next(); + + GridCacheEntryEx cached = entry.cached(); + + Collection candidates = cached.remoteMvccSnapshot(); + + if (candidates.size() != clients.length) + return false; + } + + return true; + } + }, 10_000); + } + + CommandHandler h = new CommandHandler(); + + // Check listing. + validate(h, map -> { + for (int i = 0; i < cnt; i++) { + IgniteEx grid = grid(i); + + // Skip primary. + if (grid.localNode().id().equals(prim.cluster().localNode().id())) + continue; + + VisorTxTaskResult res = map.get(grid.localNode()); + + // Validate queue length on backups. + assertEquals(clients.length, res.getInfos().size()); + } + }, "--tx"); + + // Check kill. + validate(h, map -> { + // No-op. + }, "--tx", "kill"); + + // Wait for all remote txs to finish. + for (Ignite ignite : G.allGrids()) { + if (ignite.configuration().isClientMode()) + continue; + + Collection txs = ((IgniteEx)ignite).context().cache().context().tm().activeTransactions(); + + for (IgniteInternalTx tx : txs) + if (!tx.local()) + tx.finishFuture().get(); + } + + // Unblock finish message from client1. + primSpi.stopBlock(true); + + fut.get(); + + Long cur = (Long)clients[0].cache(DEFAULT_CACHE_NAME).get(0L); + + assertEquals(tc - 1, cur.longValue()); + + checkFutures(); + } + /** * Test baseline add items works via control.sh * @@ -1006,6 +1272,23 @@ private void checkFutures() { log.info("Waiting for future: " + fut); assertTrue("Expecting no active futures: node=" + ig.localNode().id(), futs.isEmpty()); + + Collection txs = ig.context().cache().context().tm().activeTransactions(); + + for (IgniteInternalTx tx : txs) + log.info("Waiting for tx: " + tx); + + assertTrue("Expecting no active transactions: node=" + ig.localNode().id(), txs.isEmpty()); + } + } + + /** */ + private static class IncrementClosure implements EntryProcessor { + /** {@inheritDoc} */ + @Override public Void process(MutableEntry entry, Object... arguments) throws EntryProcessorException { + entry.setValue(entry.exists() ? entry.getValue() + 1 : 0); + + return null; } } From 7bcc6b1a319f026ce477f251d05bf1e4acf20abc Mon Sep 17 00:00:00 2001 From: Alexey Goncharuk Date: Wed, 20 Jun 2018 14:05:53 +0300 Subject: [PATCH 0286/1463] IGNITE-8491 Add JMX flag: Is the node in baseline or not - fixed license header (cherry picked from commit d58f368) --- .../util/mbeans/GridMBeanBaselineTest.java | 31 ++++++++++--------- 1 file changed, 16 insertions(+), 15 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanBaselineTest.java b/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanBaselineTest.java index 390eb21386dae..31c012c8b2a49 100644 --- a/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanBaselineTest.java +++ b/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanBaselineTest.java @@ -1,19 +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. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.ignite.util.mbeans; import org.apache.ignite.Ignite; From 007626a106335a2db80529d75594f1c99c243709 Mon Sep 17 00:00:00 2001 From: Dmitriy Sorokin Date: Wed, 20 Jun 2018 16:41:46 +0300 Subject: [PATCH 0287/1463] IGNITE-8749 Exception for "no space left" situation should be propagated to FailureHandler - Fixes #4200. Signed-off-by: Ivan Rakov (cherry picked from commit 3fff8a8) Signed-off-by: Ivan Rakov --- .../wal/FileWriteAheadLogManager.java | 190 +++++++------ .../FsyncModeFileWriteAheadLogManager.java | 235 +++++++++------- .../ignite/failure/TestFailureHandler.java | 19 ++ .../wal/IgniteWalFormatFileFailoverTest.java | 258 ++++++++++++++++++ .../testsuites/IgnitePdsTestSuite2.java | 3 + 5 files changed, 513 insertions(+), 192 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFormatFileFailoverTest.java 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 9b39987edf42a..3f406629f9e49 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 @@ -605,48 +605,43 @@ private void checkWalConfiguration() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public void resumeLogging(WALPointer lastPtr) throws IgniteCheckedException { - try { - assert currHnd == null; - assert lastPtr == null || lastPtr instanceof FileWALPointer; + assert currHnd == null; + assert lastPtr == null || lastPtr instanceof FileWALPointer; - FileWALPointer filePtr = (FileWALPointer)lastPtr; + FileWALPointer filePtr = (FileWALPointer)lastPtr; - walWriter = new WALWriter(log); + walWriter = new WALWriter(log); - if (!mmap) - new IgniteThread(walWriter).start(); + if (!mmap) + new IgniteThread(walWriter).start(); - currHnd = restoreWriteHandle(filePtr); + currHnd = restoreWriteHandle(filePtr); - // For new handle write serializer version to it. - if (filePtr == null) - currHnd.writeHeader(); + // For new handle write serializer version to it. + if (filePtr == null) + currHnd.writeHeader(); - if (currHnd.serializer.version() != serializer.version()) { - if (log.isInfoEnabled()) - log.info("Record serializer version change detected, will start logging with a new WAL record " + - "serializer to a new WAL segment [curFile=" + currHnd + ", newVer=" + serializer.version() + - ", oldVer=" + currHnd.serializer.version() + ']'); + if (currHnd.serializer.version() != serializer.version()) { + if (log.isInfoEnabled()) + log.info("Record serializer version change detected, will start logging with a new WAL record " + + "serializer to a new WAL segment [curFile=" + currHnd + ", newVer=" + serializer.version() + + ", oldVer=" + currHnd.serializer.version() + ']'); - rollOver(currHnd); - } + rollOver(currHnd); + } - currHnd.resume = false; + currHnd.resume = false; - if (mode == WALMode.BACKGROUND) { - backgroundFlushSchedule = cctx.time().schedule(new Runnable() { - @Override public void run() { - doFlush(); - } - }, flushFreq, flushFreq); - } - - if (walAutoArchiveAfterInactivity > 0) - scheduleNextInactivityPeriodElapsedCheck(); - } - catch (StorageException e) { - throw new IgniteCheckedException(e); + if (mode == WALMode.BACKGROUND) { + backgroundFlushSchedule = cctx.time().schedule(new Runnable() { + @Override public void run() { + doFlush(); + } + }, flushFreq, flushFreq); } + + if (walAutoArchiveAfterInactivity > 0) + scheduleNextInactivityPeriodElapsedCheck(); } /** @@ -1131,9 +1126,9 @@ private FileWriteHandle rollOver(FileWriteHandle cur) throws StorageException, I /** * @param lastReadPtr Last read WAL file pointer. * @return Initialized file write handle. - * @throws IgniteCheckedException If failed to initialize WAL write handle. + * @throws StorageException If failed to initialize WAL write handle. */ - private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws IgniteCheckedException { + private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws StorageException { long absIdx = lastReadPtr == null ? 0 : lastReadPtr.index(); @Nullable FileArchiver archiver0 = archiver; @@ -1175,14 +1170,9 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws Ig SegmentedRingByteBuffer rbuf; if (mmap) { - try { - MappedByteBuffer buf = fileIO.map((int)maxWalSegmentSize); + MappedByteBuffer buf = fileIO.map((int)maxWalSegmentSize); - rbuf = new SegmentedRingByteBuffer(buf, metrics); - } - catch (IOException e) { - throw new IgniteCheckedException(e); - } + rbuf = new SegmentedRingByteBuffer(buf, metrics); } else rbuf = new SegmentedRingByteBuffer(dsCfg.getWalBufferSize(), maxWalSegmentSize, DIRECT, metrics); @@ -1206,13 +1196,21 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws Ig return hnd; } catch (IgniteCheckedException | IOException e) { - fileIO.close(); + try { + fileIO.close(); + } + catch (IOException suppressed) { + e.addSuppressed(suppressed); + } - throw e; + if (e instanceof StorageException) + throw (StorageException) e; + + throw e instanceof IOException ? (IOException) e : new IOException(e); } } catch (IOException e) { - throw new IgniteCheckedException("Failed to restore WAL write handle: " + curFile.getAbsolutePath(), e); + throw new StorageException("Failed to restore WAL write handle: " + curFile.getAbsolutePath(), e); } } @@ -1222,10 +1220,11 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws Ig * * @param cur Current file write handle released by WAL writer * @return Initialized file handle. - * @throws StorageException If IO exception occurred. - * @throws IgniteCheckedException If failed. + * @throws IgniteCheckedException If exception occurred. */ - private FileWriteHandle initNextWriteHandle(FileWriteHandle cur) throws StorageException, IgniteCheckedException { + private FileWriteHandle initNextWriteHandle(FileWriteHandle cur) throws IgniteCheckedException { + IgniteCheckedException error = null; + try { File nextFile = pollNextFile(cur.idx); @@ -1299,19 +1298,24 @@ private FileWriteHandle initNextWriteHandle(FileWriteHandle cur) throws StorageE return hnd; } + catch (IgniteCheckedException e) { + throw error = e; + } catch (IOException e) { - StorageException se = new StorageException("Unable to initialize WAL segment", e); - - cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, se)); - - throw se; + throw error = new StorageException("Unable to initialize WAL segment", e); + } + finally { + if (error != null) + cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, error)); } } /** * Deletes temp files, creates and prepares new; Creates first segment if necessary + * + * @throws StorageException If failed. */ - private void checkOrPrepareFiles() throws IgniteCheckedException { + private void checkOrPrepareFiles() throws StorageException { // Clean temp files. { File[] tmpFiles = walWorkDir.listFiles(WAL_SEGMENT_TEMP_FILE_FILTER); @@ -1321,7 +1325,7 @@ private void checkOrPrepareFiles() throws IgniteCheckedException { boolean deleted = tmp.delete(); if (!deleted) - throw new IgniteCheckedException("Failed to delete previously created temp file " + + throw new StorageException("Failed to delete previously created temp file " + "(make sure Ignite process has enough rights): " + tmp.getAbsolutePath()); } } @@ -1331,7 +1335,7 @@ private void checkOrPrepareFiles() throws IgniteCheckedException { if(isArchiverEnabled()) if (allFiles.length != 0 && allFiles.length > dsCfg.getWalSegments()) - throw new IgniteCheckedException("Failed to initialize wal (work directory contains " + + throw new StorageException("Failed to initialize wal (work directory contains " + "incorrect number of segments) [cur=" + allFiles.length + ", expected=" + dsCfg.getWalSegments() + ']'); // Allocate the first segment synchronously. All other segments will be allocated by archiver in background. @@ -1371,9 +1375,9 @@ public void cleanupWalDirectories() throws IgniteCheckedException { * Clears whole the file, fills with zeros for Default mode. * * @param file File to format. - * @throws IgniteCheckedException if formatting failed + * @throws StorageException if formatting failed */ - private void formatFile(File file) throws IgniteCheckedException { + private void formatFile(File file) throws StorageException { formatFile(file, dsCfg.getWalSegmentSize()); } @@ -1382,9 +1386,9 @@ private void formatFile(File file) throws IgniteCheckedException { * * @param file File to format. * @param bytesCntToFormat Count of first bytes to format. - * @throws IgniteCheckedException if formatting failed + * @throws StorageException if formatting failed */ - private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedException { + private void formatFile(File file, int bytesCntToFormat) throws StorageException { if (log.isDebugEnabled()) log.debug("Formatting file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']'); @@ -1396,7 +1400,7 @@ private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedExc int toWrite = Math.min(FILL_BUF.length, left); if (fileIO.write(FILL_BUF, 0, toWrite) < toWrite) { - final IgniteCheckedException ex = new IgniteCheckedException("Failed to extend WAL segment file: " + + final StorageException ex = new StorageException("Failed to extend WAL segment file: " + file.getName() + ". Probably disk is too busy, please check your device."); if (failureProcessor != null) @@ -1414,7 +1418,7 @@ private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedExc fileIO.clear(); } catch (IOException e) { - throw new IgniteCheckedException("Failed to format WAL segment file: " + file.getAbsolutePath(), e); + throw new StorageException("Failed to format WAL segment file: " + file.getAbsolutePath(), e); } } @@ -1422,9 +1426,9 @@ private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedExc * Creates a file atomically with temp file. * * @param file File to create. - * @throws IgniteCheckedException If failed. + * @throws StorageException If failed. */ - private void createFile(File file) throws IgniteCheckedException { + private void createFile(File file) throws StorageException { if (log.isDebugEnabled()) log.debug("Creating new file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']'); @@ -1436,7 +1440,7 @@ private void createFile(File file) throws IgniteCheckedException { Files.move(tmp.toPath(), file.toPath()); } catch (IOException e) { - throw new IgniteCheckedException("Failed to move temp file to a regular WAL segment file: " + + throw new StorageException("Failed to move temp file to a regular WAL segment file: " + file.getAbsolutePath(), e); } @@ -1449,9 +1453,9 @@ private void createFile(File file) throws IgniteCheckedException { * * @param curIdx Current absolute WAL segment index. * @return File ready for use as new WAL segment. - * @throws IgniteCheckedException If failed. + * @throws StorageException If exception occurred in the archiver thread. */ - private File pollNextFile(long curIdx) throws IgniteCheckedException { + private File pollNextFile(long curIdx) throws StorageException { FileArchiver archiver0 = archiver; if (archiver0 == null) { @@ -1527,7 +1531,7 @@ public long maxWalSegmentSize() { */ private class FileArchiver extends GridWorker { /** Exception which occurred during initial creation of files or during archiving WAL segment */ - private IgniteCheckedException cleanErr; + private StorageException cleanErr; /** * Absolute current segment index WAL Manager writes to. Guarded by this. Incremented during @@ -1599,15 +1603,17 @@ private synchronized boolean locked(long absIdx) { try { allocateRemainingFiles(); } - catch (IgniteCheckedException e) { + catch (StorageException e) { synchronized (this) { // Stop the thread and report to starter. cleanErr = e; notifyAll(); - - return; } + + cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, e)); + + return; } Throwable err = null; @@ -1691,9 +1697,9 @@ private void changeLastArchivedIndexAndNotifyWaiters(long idx) { * * @param curIdx Current absolute index that we want to increment. * @return Next index (curWalSegmIdx+1) when it is ready to be written. - * @throws IgniteCheckedException If failed (if interrupted or if exception occurred in the archiver thread). + * @throws StorageException If exception occurred in the archiver thread. */ - private long nextAbsoluteSegmentIndex(long curIdx) throws IgniteCheckedException { + private long nextAbsoluteSegmentIndex(long curIdx) throws StorageException { synchronized (this) { if (cleanErr != null) throw cleanErr; @@ -1708,6 +1714,9 @@ private long nextAbsoluteSegmentIndex(long curIdx) throws IgniteCheckedException while (curAbsWalIdx - lastAbsArchivedIdx > dsCfg.getWalSegments() && cleanErr == null) { try { wait(); + + if (cleanErr != null) + throw cleanErr; } catch (InterruptedException ignore) { interrupted.set(true); @@ -1715,9 +1724,12 @@ private long nextAbsoluteSegmentIndex(long curIdx) throws IgniteCheckedException } // Wait for formatter so that we do not open an empty file in DEFAULT mode. - while (curAbsWalIdx % dsCfg.getWalSegments() > formatted) + while (curAbsWalIdx % dsCfg.getWalSegments() > formatted && cleanErr == null) try { wait(); + + if (cleanErr != null) + throw cleanErr; } catch (InterruptedException ignore) { interrupted.set(true); @@ -1789,7 +1801,7 @@ private void releaseWorkSegment(long absIdx) { * * @param absIdx Absolute index to archive. */ - private SegmentArchiveResult archiveSegment(long absIdx) throws IgniteCheckedException { + private SegmentArchiveResult archiveSegment(long absIdx) throws StorageException { long segIdx = absIdx % dsCfg.getWalSegments(); File origFile = new File(walWorkDir, FileDescriptor.fileName(segIdx)); @@ -1818,7 +1830,7 @@ private SegmentArchiveResult archiveSegment(long absIdx) throws IgniteCheckedExc } } catch (IOException e) { - throw new IgniteCheckedException("Failed to archive WAL segment [" + + throw new StorageException("Failed to archive WAL segment [" + "srcFile=" + origFile.getAbsolutePath() + ", dstFile=" + dstTmpFile.getAbsolutePath() + ']', e); } @@ -1841,7 +1853,7 @@ private boolean checkStop() { * Background creation of all segments except first. First segment was created in main thread by {@link * FileWriteAheadLogManager#checkOrPrepareFiles()} */ - private void allocateRemainingFiles() throws IgniteCheckedException { + private void allocateRemainingFiles() throws StorageException { checkFiles( 1, true, @@ -2235,23 +2247,23 @@ private void shutdown() throws IgniteInterruptedCheckedException { * @param startWith Start with. * @param create Flag create file. * @param p Predicate Exit condition. - * @throws IgniteCheckedException if validation or create file fail. + * @throws StorageException if validation or create file fail. */ private void checkFiles( int startWith, boolean create, @Nullable IgnitePredicate p, @Nullable IgniteInClosure completionCallback - ) throws IgniteCheckedException { + ) throws StorageException { for (int i = startWith; i < dsCfg.getWalSegments() && (p == null || p.apply(i)); i++) { File checkFile = new File(walWorkDir, FileDescriptor.fileName(i)); if (checkFile.exists()) { if (checkFile.isDirectory()) - throw new IgniteCheckedException("Failed to initialize WAL log segment (a directory with " + + throw new StorageException("Failed to initialize WAL log segment (a directory with " + "the same name already exists): " + checkFile.getAbsolutePath()); else if (checkFile.length() != dsCfg.getWalSegmentSize() && mode == WALMode.FSYNC) - throw new IgniteCheckedException("Failed to initialize WAL log segment " + + throw new StorageException("Failed to initialize WAL log segment " + "(WAL segment size change is not supported in 'DEFAULT' WAL mode) " + "[filePath=" + checkFile.getAbsolutePath() + ", fileSize=" + checkFile.length() + @@ -2651,9 +2663,8 @@ public void writeHeader() { * Flush or wait for concurrent flush completion. * * @param ptr Pointer. - * @throws IgniteCheckedException If failed. */ - private void flushOrWait(FileWALPointer ptr) throws IgniteCheckedException { + private void flushOrWait(FileWALPointer ptr) { if (ptr != null) { // If requested obsolete file index, it must be already flushed by close. if (ptr.index() != idx) @@ -2665,10 +2676,8 @@ private void flushOrWait(FileWALPointer ptr) throws IgniteCheckedException { /** * @param ptr Pointer. - * @throws IgniteCheckedException If failed. - * @throws StorageException If failed. */ - private void flush(FileWALPointer ptr) throws IgniteCheckedException, StorageException { + private void flush(FileWALPointer ptr) { if (ptr == null) { // Unconditional flush. walWriter.flushAll(); @@ -2884,7 +2893,7 @@ private boolean close(boolean rollOver) throws IgniteCheckedException, StorageEx } } catch (IOException e) { - throw new IgniteCheckedException(e); + throw new StorageException("Failed to close WAL write handle [idx=" + idx + "]", e); } if (log.isDebugEnabled()) @@ -3365,28 +3374,29 @@ private void unparkWaiters(long pos) { /** * Forces all made changes to the file. */ - void force() throws IgniteCheckedException { + void force() { flushBuffer(FILE_FORCE); } /** * Closes file. */ - void close() throws IgniteCheckedException { + void close() { flushBuffer(FILE_CLOSE); } /** * Flushes all data from the buffer. */ - void flushAll() throws IgniteCheckedException { + void flushAll() { flushBuffer(UNCONDITIONAL_FLUSH); } /** * @param expPos Expected position. */ - void flushBuffer(long expPos) throws StorageException, IgniteCheckedException { + @SuppressWarnings("ForLoopReplaceableByForEach") + void flushBuffer(long expPos) { if (mmap) return; 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 49fbc73b9a48c..6e59ad3614dec 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 @@ -487,37 +487,32 @@ private void checkWalConfiguration() throws IgniteCheckedException { /** {@inheritDoc} */ @Override public void resumeLogging(WALPointer lastPtr) throws IgniteCheckedException { - try { - assert currentHnd == null; - assert lastPtr == null || lastPtr instanceof FileWALPointer; - - FileWALPointer filePtr = (FileWALPointer)lastPtr; + assert currentHnd == null; + assert lastPtr == null || lastPtr instanceof FileWALPointer; - currentHnd = restoreWriteHandle(filePtr); - - if (currentHnd.serializer.version() != serializer.version()) { - if (log.isInfoEnabled()) - log.info("Record serializer version change detected, will start logging with a new WAL record " + - "serializer to a new WAL segment [curFile=" + currentHnd + ", newVer=" + serializer.version() + - ", oldVer=" + currentHnd.serializer.version() + ']'); + FileWALPointer filePtr = (FileWALPointer)lastPtr; - rollOver(currentHnd); - } + currentHnd = restoreWriteHandle(filePtr); - if (mode == WALMode.BACKGROUND) { - backgroundFlushSchedule = cctx.time().schedule(new Runnable() { - @Override public void run() { - doFlush(); - } - }, flushFreq, flushFreq); - } + if (currentHnd.serializer.version() != serializer.version()) { + if (log.isInfoEnabled()) + log.info("Record serializer version change detected, will start logging with a new WAL record " + + "serializer to a new WAL segment [curFile=" + currentHnd + ", newVer=" + serializer.version() + + ", oldVer=" + currentHnd.serializer.version() + ']'); - if (walAutoArchiveAfterInactivity > 0) - scheduleNextInactivityPeriodElapsedCheck(); + rollOver(currentHnd); } - catch (StorageException e) { - throw new IgniteCheckedException(e); + + if (mode == WALMode.BACKGROUND) { + backgroundFlushSchedule = cctx.time().schedule(new Runnable() { + @Override public void run() { + doFlush(); + } + }, flushFreq, flushFreq); } + + if (walAutoArchiveAfterInactivity > 0) + scheduleNextInactivityPeriodElapsedCheck(); } /** @@ -1019,7 +1014,7 @@ private FileWriteHandle currentHandle() { * @param cur Handle that failed to fit the given entry. * @return Handle that will fit the entry. */ - private FileWriteHandle rollOver(FileWriteHandle cur) throws StorageException, IgniteCheckedException { + private FileWriteHandle rollOver(FileWriteHandle cur) throws IgniteCheckedException { FileWriteHandle hnd = currentHandle(); if (hnd != cur) @@ -1050,9 +1045,9 @@ private FileWriteHandle rollOver(FileWriteHandle cur) throws StorageException, I /** * @param lastReadPtr Last read WAL file pointer. * @return Initialized file write handle. - * @throws IgniteCheckedException If failed to initialize WAL write handle. + * @throws StorageException If failed to initialize WAL write handle. */ - private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws IgniteCheckedException { + private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws StorageException { long absIdx = lastReadPtr == null ? 0 : lastReadPtr.index(); long segNo = absIdx % dsCfg.getWalSegments(); @@ -1100,13 +1095,21 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws Ig return hnd; } catch (IgniteCheckedException | IOException e) { - fileIO.close(); + try { + fileIO.close(); + } + catch (IOException suppressed) { + e.addSuppressed(suppressed); + } - throw e; + if (e instanceof StorageException) + throw (StorageException) e; + + throw e instanceof IOException ? (IOException) e : new IOException(e); } } catch (IOException e) { - throw new IgniteCheckedException("Failed to restore WAL write handle: " + curFile.getAbsolutePath(), e); + throw new StorageException("Failed to restore WAL write handle: " + curFile.getAbsolutePath(), e); } } @@ -1117,10 +1120,11 @@ private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws Ig * * @param curIdx current absolute segment released by WAL writer * @return Initialized file handle. - * @throws StorageException If IO exception occurred. - * @throws IgniteCheckedException If failed. + * @throws IgniteCheckedException If exception occurred. */ - private FileWriteHandle initNextWriteHandle(long curIdx) throws StorageException, IgniteCheckedException { + private FileWriteHandle initNextWriteHandle(long curIdx) throws IgniteCheckedException { + IgniteCheckedException error = null; + try { File nextFile = pollNextFile(curIdx); @@ -1140,19 +1144,24 @@ private FileWriteHandle initNextWriteHandle(long curIdx) throws StorageException return hnd; } + catch (IgniteCheckedException e) { + throw error = e; + } catch (IOException e) { - StorageException se = new StorageException("Unable to initialize WAL segment", e); - - cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, se)); - - throw se; + throw error = new StorageException("Unable to initialize WAL segment", e); + } + finally { + if (error != null) + cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, error)); } } /** - * Deletes temp files, creates and prepares new; Creates first segment if necessary + * Deletes temp files, creates and prepares new; Creates first segment if necessary. + * + * @throws StorageException If failed. */ - private void checkOrPrepareFiles() throws IgniteCheckedException { + private void checkOrPrepareFiles() throws StorageException { // Clean temp files. { File[] tmpFiles = walWorkDir.listFiles(WAL_SEGMENT_TEMP_FILE_FILTER); @@ -1162,7 +1171,7 @@ private void checkOrPrepareFiles() throws IgniteCheckedException { boolean deleted = tmp.delete(); if (!deleted) - throw new IgniteCheckedException("Failed to delete previously created temp file " + + throw new StorageException("Failed to delete previously created temp file " + "(make sure Ignite process has enough rights): " + tmp.getAbsolutePath()); } } @@ -1171,7 +1180,7 @@ private void checkOrPrepareFiles() throws IgniteCheckedException { File[] allFiles = walWorkDir.listFiles(WAL_SEGMENT_FILE_FILTER); if (allFiles.length != 0 && allFiles.length > dsCfg.getWalSegments()) - throw new IgniteCheckedException("Failed to initialize wal (work directory contains " + + throw new StorageException("Failed to initialize wal (work directory contains " + "incorrect number of segments) [cur=" + allFiles.length + ", expected=" + dsCfg.getWalSegments() + ']'); // Allocate the first segment synchronously. All other segments will be allocated by archiver in background. @@ -1188,9 +1197,9 @@ private void checkOrPrepareFiles() throws IgniteCheckedException { * Clears whole the file, fills with zeros for Default mode. * * @param file File to format. - * @throws IgniteCheckedException if formatting failed + * @throws StorageException if formatting failed. */ - private void formatFile(File file) throws IgniteCheckedException { + private void formatFile(File file) throws StorageException { formatFile(file, dsCfg.getWalSegmentSize()); } @@ -1199,9 +1208,9 @@ private void formatFile(File file) throws IgniteCheckedException { * * @param file File to format. * @param bytesCntToFormat Count of first bytes to format. - * @throws IgniteCheckedException if formatting failed + * @throws StorageException If formatting failed. */ - private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedException { + private void formatFile(File file, int bytesCntToFormat) throws StorageException { if (log.isDebugEnabled()) log.debug("Formatting file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']'); @@ -1223,7 +1232,7 @@ private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedExc fileIO.clear(); } catch (IOException e) { - throw new IgniteCheckedException("Failed to format WAL segment file: " + file.getAbsolutePath(), e); + throw new StorageException("Failed to format WAL segment file: " + file.getAbsolutePath(), e); } } @@ -1231,9 +1240,9 @@ private void formatFile(File file, int bytesCntToFormat) throws IgniteCheckedExc * Creates a file atomically with temp file. * * @param file File to create. - * @throws IgniteCheckedException If failed. + * @throws StorageException If failed. */ - private void createFile(File file) throws IgniteCheckedException { + private void createFile(File file) throws StorageException { if (log.isDebugEnabled()) log.debug("Creating new file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']'); @@ -1245,7 +1254,7 @@ private void createFile(File file) throws IgniteCheckedException { Files.move(tmp.toPath(), file.toPath()); } catch (IOException e) { - throw new IgniteCheckedException("Failed to move temp file to a regular WAL segment file: " + + throw new StorageException("Failed to move temp file to a regular WAL segment file: " + file.getAbsolutePath(), e); } @@ -1259,9 +1268,10 @@ private void createFile(File file) throws IgniteCheckedException { * * @param curIdx Current absolute WAL segment index. * @return File ready for use as new WAL segment. - * @throws IgniteCheckedException If failed. + * @throws StorageException If exception occurred in the archiver thread. + * @throws IgniteInterruptedCheckedException If interrupted. */ - private File pollNextFile(long curIdx) throws IgniteCheckedException { + private File pollNextFile(long curIdx) throws StorageException, IgniteInterruptedCheckedException { // Signal to archiver that we are done with the segment and it can be archived. long absNextIdx = archiver.nextAbsoluteSegmentIndex(curIdx); @@ -1318,7 +1328,7 @@ private void checkNode() throws StorageException { */ private class FileArchiver extends GridWorker { /** Exception which occurred during initial creation of files or during archiving WAL segment */ - private IgniteCheckedException cleanException; + private StorageException cleanException; /** * Absolute current segment index WAL Manager writes to. Guarded by this. @@ -1426,15 +1436,17 @@ private synchronized void release(long absIdx) { try { allocateRemainingFiles(); } - catch (IgniteCheckedException e) { + catch (StorageException e) { synchronized (this) { // Stop the thread and report to starter. cleanException = e; notifyAll(); - - return; } + + cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, e)); + + return; } Throwable err = null; @@ -1515,9 +1527,10 @@ private void changeLastArchivedIndexAndWakeupCompressor(long idx) { * * @param curIdx Current absolute index that we want to increment. * @return Next index (curWalSegmIdx+1) when it is ready to be written. - * @throws IgniteCheckedException If failed (if interrupted or if exception occurred in the archiver thread). + * @throws StorageException If exception occurred in the archiver thread. + * @throws IgniteInterruptedCheckedException If interrupted. */ - private long nextAbsoluteSegmentIndex(long curIdx) throws IgniteCheckedException { + private long nextAbsoluteSegmentIndex(long curIdx) throws StorageException, IgniteInterruptedCheckedException { try { synchronized (this) { if (cleanException != null) @@ -1535,10 +1548,16 @@ private long nextAbsoluteSegmentIndex(long curIdx) throws IgniteCheckedException while ((curAbsWalIdx - lastAbsArchivedIdx > segments && cleanException == null)) wait(); + if (cleanException != null) + throw cleanException; + // Wait for formatter so that we do not open an empty file in DEFAULT mode. - while (curAbsWalIdx % dsCfg.getWalSegments() > formatted) + while (curAbsWalIdx % dsCfg.getWalSegments() > formatted && cleanException == null) wait(); + if (cleanException != null) + throw cleanException; + return curAbsWalIdx; } } @@ -1664,7 +1683,7 @@ private boolean checkStop() { * Background creation of all segments except first. First segment was created in main thread by * {@link FsyncModeFileWriteAheadLogManager#checkOrPrepareFiles()} */ - private void allocateRemainingFiles() throws IgniteCheckedException { + private void allocateRemainingFiles() throws StorageException { final FileArchiver archiver = this; checkFiles(1, @@ -2029,23 +2048,23 @@ private void shutdown() { * @param startWith Start with. * @param create Flag create file. * @param p Predicate Exit condition. - * @throws IgniteCheckedException if validation or create file fail. + * @throws StorageException if validation or create file fail. */ private void checkFiles( int startWith, boolean create, @Nullable IgnitePredicate p, @Nullable IgniteInClosure completionCallback - ) throws IgniteCheckedException { + ) throws StorageException { for (int i = startWith; i < dsCfg.getWalSegments() && (p == null || (p != null && p.apply(i))); i++) { File checkFile = new File(walWorkDir, FileDescriptor.fileName(i)); if (checkFile.exists()) { if (checkFile.isDirectory()) - throw new IgniteCheckedException("Failed to initialize WAL log segment (a directory with " + + throw new StorageException("Failed to initialize WAL log segment (a directory with " + "the same name already exists): " + checkFile.getAbsolutePath()); else if (checkFile.length() != dsCfg.getWalSegmentSize() && mode == WALMode.FSYNC) - throw new IgniteCheckedException("Failed to initialize WAL log segment " + + throw new StorageException("Failed to initialize WAL log segment " + "(WAL segment size change is not supported):" + checkFile.getAbsolutePath()); } else if (create) @@ -2410,7 +2429,7 @@ private FileWriteHandle( * * @throws IOException If fail to write serializer version. */ - public void writeSerializerVersion() throws IOException { + private void writeSerializerVersion() throws IOException { try { assert fileIO.position() == 0 : "Serializer version can be written only at the begin of file " + fileIO.position(); @@ -2448,9 +2467,8 @@ private boolean stopped(WALRecord record) { * @param rec Record to be added to record chain as new {@link #head} * @return Pointer or null if roll over to next segment is required or already started by other thread. * @throws StorageException If failed. - * @throws IgniteCheckedException If failed. */ - @Nullable private WALPointer addRecord(WALRecord rec) throws StorageException, IgniteCheckedException { + @Nullable private WALPointer addRecord(WALRecord rec) throws StorageException { assert rec.size() > 0 || rec.getClass() == FakeRecord.class; boolean flushed = false; @@ -2503,9 +2521,9 @@ private long nextPosition(WALRecord rec) { * Flush or wait for concurrent flush completion. * * @param ptr Pointer. - * @throws IgniteCheckedException If failed. + * @throws StorageException If failed. */ - private void flushOrWait(FileWALPointer ptr, boolean stop) throws IgniteCheckedException { + private void flushOrWait(FileWALPointer ptr, boolean stop) throws StorageException { long expWritten; if (ptr != null) { @@ -2549,10 +2567,9 @@ else if (stop) { /** * @param ptr Pointer. * @return {@code true} If the flush really happened. - * @throws IgniteCheckedException If failed. * @throws StorageException If failed. */ - private boolean flush(FileWALPointer ptr, boolean stop) throws IgniteCheckedException, StorageException { + private boolean flush(FileWALPointer ptr, boolean stop) throws StorageException { if (ptr == null) { // Unconditional flush. for (; ; ) { WALRecord expHead = head.get(); @@ -2594,10 +2611,9 @@ private long chainBeginPosition(WALRecord h) { /** * @param expHead Expected head of chain. If head was changed, flush is not performed in this thread - * @throws IgniteCheckedException If failed. * @throws StorageException If failed. */ - private boolean flush(WALRecord expHead, boolean stop) throws StorageException, IgniteCheckedException { + private boolean flush(WALRecord expHead, boolean stop) throws StorageException { if (expHead.previous() == null) { FakeRecord frHead = (FakeRecord)expHead; @@ -2643,7 +2659,8 @@ private boolean flush(WALRecord expHead, boolean stop) throws StorageException, return true; } catch (Throwable e) { - StorageException se = new StorageException("Unable to write", new IOException(e)); + StorageException se = e instanceof StorageException ? (StorageException) e : + new StorageException("Unable to write", new IOException(e)); cctx.kernalContext().failure().process(new FailureContext(CRITICAL_ERROR, se)); @@ -2725,8 +2742,9 @@ private FileWALPointer position() { /** * @param ptr Pointer to sync. * @throws StorageException If failed. + * @throws IgniteInterruptedCheckedException If interrupted. */ - private void fsync(FileWALPointer ptr, boolean stop) throws StorageException, IgniteCheckedException { + private void fsync(FileWALPointer ptr, boolean stop) throws StorageException, IgniteInterruptedCheckedException { lock.lock(); try { @@ -2780,10 +2798,9 @@ private void fsync(FileWALPointer ptr, boolean stop) throws StorageException, Ig /** * @return {@code true} If this thread actually closed the segment. - * @throws IgniteCheckedException If failed. * @throws StorageException If failed. */ - private boolean close(boolean rollOver) throws IgniteCheckedException, StorageException { + private boolean close(boolean rollOver) throws StorageException { if (stop.compareAndSet(false, true)) { lock.lock(); @@ -2793,43 +2810,49 @@ private boolean close(boolean rollOver) throws IgniteCheckedException, StorageEx assert stopped() : "Segment is not closed after close flush: " + head.get(); try { - RecordSerializer backwardSerializer = new RecordSerializerFactoryImpl(cctx) - .createSerializer(serializerVersion); + try { + RecordSerializer backwardSerializer = new RecordSerializerFactoryImpl(cctx) + .createSerializer(serializerVersion); - SwitchSegmentRecord segmentRecord = new SwitchSegmentRecord(); + SwitchSegmentRecord segmentRecord = new SwitchSegmentRecord(); - int switchSegmentRecSize = backwardSerializer.size(segmentRecord); + int switchSegmentRecSize = backwardSerializer.size(segmentRecord); - if (rollOver && written < (maxSegmentSize - switchSegmentRecSize)) { - final ByteBuffer buf = ByteBuffer.allocate(switchSegmentRecSize); + if (rollOver && written < (maxSegmentSize - switchSegmentRecSize)) { + final ByteBuffer buf = ByteBuffer.allocate(switchSegmentRecSize); - segmentRecord.position(new FileWALPointer(idx, (int)written, switchSegmentRecSize)); - backwardSerializer.writeRecord(segmentRecord, buf); + segmentRecord.position(new FileWALPointer(idx, (int)written, switchSegmentRecSize)); + backwardSerializer.writeRecord(segmentRecord, buf); - buf.rewind(); + buf.rewind(); - int rem = buf.remaining(); + int rem = buf.remaining(); - while (rem > 0) { - int written0 = fileIO.write(buf, written); + while (rem > 0) { + int written0 = fileIO.write(buf, written); - written += written0; + written += written0; - rem -= written0; + rem -= written0; + } } } + catch (IgniteCheckedException e) { + throw new IOException(e); + } + finally { + assert mode == WALMode.FSYNC; - // Do the final fsync. - if (mode == WALMode.FSYNC) { + // Do the final fsync. fileIO.force(); lastFsyncPos = written; - } - fileIO.close(); + fileIO.close(); + } } catch (IOException e) { - throw new IgniteCheckedException(e); + throw new StorageException("Failed to close WAL write handle [idx=" + idx + "]", e); } if (log.isDebugEnabled()) @@ -2860,9 +2883,17 @@ private void signalNextAvailable() { assert written == lastFsyncPos || mode != WALMode.FSYNC : "fsync [written=" + written + ", lastFsync=" + lastFsyncPos + ']'; - } - fileIO = null; + fileIO = null; + } + else { + try { + fileIO.close(); + } + catch (IOException e) { + U.error(log, "Failed to close WAL file [idx=" + idx + ", fileIO=" + fileIO + "]", e); + } + } nextSegment.signalAll(); } @@ -2872,9 +2903,9 @@ private void signalNextAvailable() { } /** - * @throws IgniteCheckedException If failed. + * */ - private void awaitNext() throws IgniteCheckedException { + private void awaitNext() { lock.lock(); try { @@ -2894,7 +2925,7 @@ private void awaitNext() throws IgniteCheckedException { * @throws IgniteCheckedException If failed. */ @SuppressWarnings("TooBroadScope") - private void writeBuffer(long pos, ByteBuffer buf) throws StorageException, IgniteCheckedException { + private void writeBuffer(long pos, ByteBuffer buf) throws StorageException { boolean interrupted = false; lock.lock(); diff --git a/modules/core/src/test/java/org/apache/ignite/failure/TestFailureHandler.java b/modules/core/src/test/java/org/apache/ignite/failure/TestFailureHandler.java index 1159683e6b54f..545c9ea1176d1 100644 --- a/modules/core/src/test/java/org/apache/ignite/failure/TestFailureHandler.java +++ b/modules/core/src/test/java/org/apache/ignite/failure/TestFailureHandler.java @@ -18,6 +18,7 @@ package org.apache.ignite.failure; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import org.apache.ignite.Ignite; /** @@ -33,6 +34,13 @@ public class TestFailureHandler implements FailureHandler { /** Failure context. */ volatile FailureContext failureCtx; + /** + * @param invalidate Invalidate. + */ + public TestFailureHandler(boolean invalidate) { + this(invalidate, new CountDownLatch(1)); + } + /** * @param invalidate Invalidate. * @param latch Latch. @@ -60,4 +68,15 @@ public TestFailureHandler(boolean invalidate, CountDownLatch latch) { public FailureContext failureContext() { return failureCtx; } + + /** + * @param millis Millis. + + * @return Failure context. + */ + public FailureContext awaitFailure(long millis) throws InterruptedException { + latch.await(millis, TimeUnit.MILLISECONDS); + + return failureCtx; + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFormatFileFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFormatFileFailoverTest.java new file mode 100644 index 0000000000000..d30289682cbba --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFormatFileFailoverTest.java @@ -0,0 +1,258 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.io.IOException; +import java.nio.file.OpenOption; +import java.util.Arrays; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.CacheAtomicityMode; +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.failure.FailureHandler; +import org.apache.ignite.failure.TestFailureHandler; +import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager; +import org.apache.ignite.internal.pagemem.wal.StorageException; +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.X; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static java.nio.file.StandardOpenOption.CREATE; +import static java.nio.file.StandardOpenOption.READ; +import static java.nio.file.StandardOpenOption.WRITE; + +/** + * + */ +public class IgniteWalFormatFileFailoverTest extends GridCommonAbstractTest { + /** */ + private static final String TEST_CACHE = "testCache"; + + /** */ + private static final String formatFile = "formatFile"; + + /** Fail method name reference. */ + private final AtomicReference failMtdNameRef = new AtomicReference<>(); + + /** */ + private boolean fsync; + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setCacheConfiguration(new CacheConfiguration(TEST_CACHE) + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL)); + + DataStorageConfiguration memCfg = new DataStorageConfiguration() + .setDefaultDataRegionConfiguration(new DataRegionConfiguration() + .setMaxSize(2048L * 1024 * 1024) + .setPersistenceEnabled(true)) + .setWalMode(fsync ? WALMode.FSYNC : WALMode.BACKGROUND) + .setWalBufferSize(1024 * 1024) + .setWalSegmentSize(512 * 1024) + .setFileIOFactory(new FailingFileIOFactory(failMtdNameRef)); + + cfg.setDataStorageConfiguration(memCfg); + + cfg.setFailureHandler(new TestFailureHandler(false)); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testNodeStartFailedFsync() throws Exception { + fsync = true; + + failMtdNameRef.set(formatFile); + + checkCause(GridTestUtils.assertThrows(log, () -> startGrid(0), IgniteCheckedException.class, null)); + } + + /** + * @throws Exception If failed. + */ + public void testFailureHandlerTriggeredFsync() throws Exception { + fsync = true; + + failFormatFileOnClusterActivate(); + } + + /** + * @throws Exception If failed. + */ + public void testFailureHandlerTriggered() throws Exception { + fsync = false; + + failFormatFileOnClusterActivate(); + } + + /** + * @throws Exception If failed. + */ + private void failFormatFileOnClusterActivate() throws Exception { + failMtdNameRef.set(null); + + startGrid(0); + startGrid(1); + + if (!fsync) { + setFileIOFactory(grid(0).context().cache().context().wal()); + setFileIOFactory(grid(1).context().cache().context().wal()); + } + + failMtdNameRef.set(formatFile); + + grid(0).cluster().active(true); + + checkCause(failureHandler(0).awaitFailure(2000).error()); + checkCause(failureHandler(1).awaitFailure(2000).error()); + } + + /** + * @param mtdName Method name. + */ + private static boolean isCalledFrom(String mtdName) { + return isCalledFrom(Thread.currentThread().getStackTrace(), mtdName); + } + + /** + * @param stackTrace Stack trace. + * @param mtdName Method name. + */ + private static boolean isCalledFrom(StackTraceElement[] stackTrace, String mtdName) { + return Arrays.stream(stackTrace).map(StackTraceElement::getMethodName).anyMatch(mtdName::equals); + } + + /** + * @param gridIdx Grid index. + * @return Failure handler configured for grid with given index. + */ + private TestFailureHandler failureHandler(int gridIdx) { + FailureHandler hnd = grid(gridIdx).configuration().getFailureHandler(); + + assertTrue(hnd instanceof TestFailureHandler); + + return (TestFailureHandler)hnd; + } + + /** + * @param t Throwable. + */ + private void checkCause(Throwable t) { + StorageException e = X.cause(t, StorageException.class); + + assertNotNull(e); + assertNotNull(e.getMessage()); + assertTrue(e.getMessage().contains("Failed to format WAL segment file")); + + IOException ioe = X.cause(e, IOException.class); + + assertNotNull(ioe); + assertNotNull(ioe.getMessage()); + assertTrue(ioe.getMessage().contains("No space left on device")); + + assertTrue(isCalledFrom(ioe.getStackTrace(), formatFile)); + } + + /** */ + private void setFileIOFactory(IgniteWriteAheadLogManager wal) { + if (wal instanceof FileWriteAheadLogManager) + ((FileWriteAheadLogManager)wal).setFileIOFactory(new FailingFileIOFactory(failMtdNameRef)); + else + fail(wal.getClass().toString()); + } + + /** + * Create File I/O which fails if specific method call present in stack trace. + */ + private static class FailingFileIOFactory implements FileIOFactory { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Delegate factory. */ + private final FileIOFactory delegateFactory = new RandomAccessFileIOFactory(); + + /** Fail method name reference. */ + private final AtomicReference failMtdNameRef; + + /** + * @param failMtdNameRef Fail method name reference. + */ + FailingFileIOFactory(AtomicReference failMtdNameRef) { + assertNotNull(failMtdNameRef); + + this.failMtdNameRef = failMtdNameRef; + } + + /** {@inheritDoc} */ + @Override public FileIO create(File file) throws IOException { + return create(file, CREATE, READ, WRITE); + } + + /** {@inheritDoc} */ + @Override public FileIO create(File file, OpenOption... modes) throws IOException { + final FileIO delegate = delegateFactory.create(file, modes); + + return new FileIODecorator(delegate) { + @Override public int write(byte[] buf, int off, int len) throws IOException { + conditionalFail(); + + return super.write(buf, off, len); + } + + @Override public void clear() throws IOException { + conditionalFail(); + + super.clear(); + } + + private void conditionalFail() throws IOException { + String failMtdName = failMtdNameRef.get(); + + if (failMtdName != null && isCalledFrom(failMtdName)) + throw new IOException("No space left on device"); + } + }; + } + } +} 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 27757435a7d9c..316ff92fcdab2 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 @@ -49,6 +49,7 @@ import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushFsyncWithMmapBufferSelfTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushLogOnlySelfTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushLogOnlyWithMmapBufferSelfTest; +import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFormatFileFailoverTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalHistoryReservationsTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalIteratorSwitchSegmentTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalSerializerVersionTest; @@ -146,6 +147,8 @@ public static void addRealPageStoreTests(TestSuite suite) { suite.addTestSuite(IgniteWalFlushLogOnlyWithMmapBufferSelfTest.class); + suite.addTestSuite(IgniteWalFormatFileFailoverTest.class); + // Test suite uses Standalone WAL iterator to verify PDS content. suite.addTestSuite(IgniteWalReaderTest.class); From 2d76dcd33b4caf6e50486f1f18601f2a7135d6d4 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Wed, 20 Jun 2018 20:11:06 +0300 Subject: [PATCH 0288/1463] IGNITE-8838: Fixed ODBC issue with two inserts without closing cursor. (cherry picked from commit 909b644) --- .../cpp/core-test/src/compute_test.cpp | 4 +-- .../cpp/odbc-test/src/queries_test.cpp | 35 +++++++++++++++++++ .../cpp/odbc-test/src/sql_types_test.cpp | 4 +-- .../src/sql_value_expressions_test.cpp | 2 +- modules/platforms/cpp/odbc/src/message.cpp | 2 +- .../cpp/odbc/src/query/batch_query.cpp | 6 +--- .../cpp/odbc/src/query/data_query.cpp | 6 +--- .../cpp/odbc/src/query/type_info_query.cpp | 2 +- 8 files changed, 44 insertions(+), 17 deletions(-) diff --git a/modules/platforms/cpp/core-test/src/compute_test.cpp b/modules/platforms/cpp/core-test/src/compute_test.cpp index 1fd76709c1f44..57a55ae93a635 100644 --- a/modules/platforms/cpp/core-test/src/compute_test.cpp +++ b/modules/platforms/cpp/core-test/src/compute_test.cpp @@ -513,7 +513,7 @@ BOOST_AUTO_TEST_CASE(IgniteBroadcastLocalSync) { Compute compute = node.GetCompute(); - BOOST_CHECKPOINT("Broadcasting");; + BOOST_CHECKPOINT("Broadcasting"); std::vector res = compute.Broadcast(Func2(8, 5)); BOOST_CHECK_EQUAL(res.size(), 1); @@ -524,7 +524,7 @@ BOOST_AUTO_TEST_CASE(IgniteBroadcastLocalAsync) { Compute compute = node.GetCompute(); - BOOST_CHECKPOINT("Broadcasting");; + BOOST_CHECKPOINT("Broadcasting"); Future< std::vector > res = compute.BroadcastAsync(Func2(312, 245)); BOOST_CHECK(!res.IsReady()); diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index 142e277c60d14..7561e32919810 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -2024,4 +2024,39 @@ BOOST_AUTO_TEST_CASE(TestQueryAndConnectionTimeoutBoth) InsertTestBatch(11, 20, 9); } +BOOST_AUTO_TEST_CASE(TestSeveralInsertsWithoutClosing) +{ + Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;SCHEMA=cache"); + + SQLCHAR request[] = "INSERT INTO TestType(_key, i32Field) VALUES(?, ?)"; + + SQLRETURN ret = SQLPrepare(stmt, request, SQL_NTS); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + int64_t key = 0; + ret = SQLBindParameter(stmt, 1, SQL_PARAM_INPUT, SQL_C_SBIGINT, SQL_BIGINT, 0, 0, &key, 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + int32_t data = 0; + ret = SQLBindParameter(stmt, 2, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_INTEGER, 0, 0, &data, 0, 0); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + + for (int32_t i = 0; i < 10; ++i) + { + key = i; + data = i * 10; + + ret = SQLExecute(stmt); + + if (!SQL_SUCCEEDED(ret)) + BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)); + } +} + BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/odbc-test/src/sql_types_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_types_test.cpp index 404fc6e034ccf..60d9d7ee0648d 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_types_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_types_test.cpp @@ -128,7 +128,7 @@ BOOST_AUTO_TEST_CASE(TestByteArrayParamInsert) const int8_t data[] = { 'A','B','C','D','E','F','G','H','I','J' }; std::vector paramData(data, data + sizeof(data) / sizeof(data[0])); - SQLCHAR request[] = "INSERT INTO TestType(_key, i8ArrayField) VALUES(?, ?)";; + SQLCHAR request[] = "INSERT INTO TestType(_key, i8ArrayField) VALUES(?, ?)"; ret = SQLPrepare(stmt, request, SQL_NTS); @@ -161,7 +161,7 @@ BOOST_AUTO_TEST_CASE(TestByteParamInsert) { SQLRETURN ret; - SQLCHAR request[] = "INSERT INTO TestType(_key, i8Field) VALUES(?, ?)";; + SQLCHAR request[] = "INSERT INTO TestType(_key, i8Field) VALUES(?, ?)"; ret = SQLPrepare(stmt, request, SQL_NTS); diff --git a/modules/platforms/cpp/odbc-test/src/sql_value_expressions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_value_expressions_test.cpp index a115dabcab2fb..fd1268891f95b 100644 --- a/modules/platforms/cpp/odbc-test/src/sql_value_expressions_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/sql_value_expressions_test.cpp @@ -51,7 +51,7 @@ BOOST_AUTO_TEST_CASE(TestCase) "THEN (i32Field / 2) " "ELSE (i32Field / 3) " "END " - "FROM TestType", in.i32Field / 3);; + "FROM TestType", in.i32Field / 3); } BOOST_AUTO_TEST_CASE(TestCast) diff --git a/modules/platforms/cpp/odbc/src/message.cpp b/modules/platforms/cpp/odbc/src/message.cpp index d78b28d4da588..53d429bfe4429 100644 --- a/modules/platforms/cpp/odbc/src/message.cpp +++ b/modules/platforms/cpp/odbc/src/message.cpp @@ -274,7 +274,7 @@ namespace ignite if (status == ResponseStatus::SUCCESS) ReadOnSuccess(reader, ver); else - utility::ReadString(reader, error);; + utility::ReadString(reader, error); } void Response::ReadOnSuccess(impl::binary::BinaryReaderImpl&, const ProtocolVersion&) diff --git a/modules/platforms/cpp/odbc/src/query/batch_query.cpp b/modules/platforms/cpp/odbc/src/query/batch_query.cpp index 235daf823feb0..8dada3ca82774 100644 --- a/modules/platforms/cpp/odbc/src/query/batch_query.cpp +++ b/modules/platforms/cpp/odbc/src/query/batch_query.cpp @@ -50,11 +50,7 @@ namespace ignite SqlResult::Type BatchQuery::Execute() { if (executed) - { - diag.AddStatusRecord(SqlState::SHY010_SEQUENCE_ERROR, "Query cursor is in open state already."); - - return SqlResult::AI_ERROR; - } + Close(); int32_t maxPageSize = connection.GetConfiguration().GetPageSize(); int32_t rowNum = params.GetParamSetSize(); diff --git a/modules/platforms/cpp/odbc/src/query/data_query.cpp b/modules/platforms/cpp/odbc/src/query/data_query.cpp index 8b80a29f3b81b..6974c7f33c856 100644 --- a/modules/platforms/cpp/odbc/src/query/data_query.cpp +++ b/modules/platforms/cpp/odbc/src/query/data_query.cpp @@ -52,11 +52,7 @@ namespace ignite SqlResult::Type DataQuery::Execute() { if (cursor.get()) - { - diag.AddStatusRecord(SqlState::SHY010_SEQUENCE_ERROR, "Query cursor is in open state already."); - - return SqlResult::AI_ERROR; - } + InternalClose(); return MakeRequestExecute(); } diff --git a/modules/platforms/cpp/odbc/src/query/type_info_query.cpp b/modules/platforms/cpp/odbc/src/query/type_info_query.cpp index 939458a880506..c45779e863a0d 100644 --- a/modules/platforms/cpp/odbc/src/query/type_info_query.cpp +++ b/modules/platforms/cpp/odbc/src/query/type_info_query.cpp @@ -394,7 +394,7 @@ namespace ignite bool TypeInfoQuery::DataAvailable() const { - return cursor != types.end();; + return cursor != types.end(); } int64_t TypeInfoQuery::AffectedRows() const From ec1d1afc553330dfcb232b53cab624a0d6680860 Mon Sep 17 00:00:00 2001 From: Alexey Stelmak Date: Fri, 22 Jun 2018 16:43:53 +0300 Subject: [PATCH 0289/1463] IGNITE-8831 Fix of MarshallerMappingFileStore: Incorrect locks on files. - Fixes #4224. Signed-off-by: Dmitriy Pavlov --- .../internal/MarshallerMappingFileStore.java | 85 +++++++++++-------- 1 file changed, 50 insertions(+), 35 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerMappingFileStore.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerMappingFileStore.java index 6fb1371f10ea3..a01981b88ed74 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerMappingFileStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerMappingFileStore.java @@ -45,6 +45,9 @@ * when a classname is requested but is not presented in local cache of {@link MarshallerContextImpl}. */ final class MarshallerMappingFileStore { + /** File lock timeout in milliseconds. */ + private static final int FILE_LOCK_TIMEOUT_MS = 5000; + /** */ private static final GridStripedLock fileLock = new GridStripedLock(32); @@ -92,14 +95,12 @@ void writeMapping(byte platformId, int typeId, String typeName) { File file = new File(workDir, fileName); try (FileOutputStream out = new FileOutputStream(file)) { - FileLock fileLock = fileLock(out.getChannel(), false); - - assert fileLock != null : fileName; - try (Writer writer = new OutputStreamWriter(out, StandardCharsets.UTF_8)) { - writer.write(typeName); + try (FileLock ignored = fileLock(out.getChannel(), false)) { + writer.write(typeName); - writer.flush(); + writer.flush(); + } } } catch (IOException e) { @@ -120,11 +121,10 @@ void writeMapping(byte platformId, int typeId, String typeName) { } /** - * @param platformId Platform id. - * @param typeId Type id. + * @param fileName File name. */ - String readMapping(byte platformId, int typeId) throws IgniteCheckedException { - String fileName = getFileName(platformId, typeId); + private String readMapping(String fileName) throws IgniteCheckedException { + ThreadLocalRandom rnd = null; Lock lock = fileLock(fileName); @@ -133,17 +133,30 @@ String readMapping(byte platformId, int typeId) throws IgniteCheckedException { try { File file = new File(workDir, fileName); - try (FileInputStream in = new FileInputStream(file)) { - FileLock fileLock = fileLock(in.getChannel(), true); + long time = 0; + + while (true) { + try (FileInputStream in = new FileInputStream(file)) { + try (BufferedReader reader = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8))) { + try (FileLock ignored = fileLock(in.getChannel(), true)) { + if (file.length() > 0) + return reader.readLine(); + + if (rnd == null) + rnd = ThreadLocalRandom.current(); - assert fileLock != null : fileName; + if (time == 0) + time = U.currentTimeMillis(); + else if ((U.currentTimeMillis() - time) >= FILE_LOCK_TIMEOUT_MS) + return null; - try (BufferedReader reader = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8))) { - return reader.readLine(); + U.sleep(rnd.nextLong(50)); + } + } + } + catch (IOException ignored) { + return null; } - } - catch (IOException ignored) { - return null; } } finally { @@ -151,6 +164,14 @@ String readMapping(byte platformId, int typeId) throws IgniteCheckedException { } } + /** + * @param platformId Platform id. + * @param typeId Type id. + */ + String readMapping(byte platformId, int typeId) throws IgniteCheckedException { + return readMapping(getFileName(platformId, typeId)); + } + /** * Restores all mappings available in file system to marshaller context. * This method should be used only on node startup. @@ -165,22 +186,16 @@ void restoreMappings(MarshallerContext marshCtx) throws IgniteCheckedException { int typeId = getTypeId(name); - try (FileInputStream in = new FileInputStream(file)) { - try (BufferedReader reader = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8))) { - String clsName = reader.readLine(); + String clsName = readMapping(name); - if (clsName == null) { - throw new IgniteCheckedException("Class name is null for [platformId=" + platformId + - ", typeId=" + typeId + "], marshaller mappings storage is broken. " + - "Clean up marshaller directory (/marshaller) and restart the node."); - } - - marshCtx.registerClassNameLocally(platformId, typeId, clsName); - } - } - catch (IOException e) { - throw new IgniteCheckedException("Reading marshaller mapping from file " + name + " failed.", e); + if (clsName == null) { + throw new IgniteCheckedException("Class name is null for [platformId=" + platformId + + ", typeId=" + typeId + "], marshaller mappings storage is broken. " + + "Clean up marshaller directory (/marshaller) and restart the node. File name: " + name + + ", FileSize: " + file.length()); } + + marshCtx.registerClassNameLocally(platformId, typeId, clsName); } } @@ -276,10 +291,10 @@ private static FileLock fileLock( while (true) { FileLock fileLock = ch.tryLock(0L, Long.MAX_VALUE, shared); - if (fileLock == null) - U.sleep(rnd.nextLong(50)); - else + if (fileLock != null) return fileLock; + + U.sleep(rnd.nextLong(50)); } } } From fc6fbfe1e9c09a1ef440ba221a8ccf69c441c34c Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Fri, 22 Jun 2018 17:19:48 +0300 Subject: [PATCH 0290/1463] IGNITE-8503 Fix wrong GridCacheMapEntry startVersion initialization. - Fixes #4141. Signed-off-by: Dmitriy Pavlov (cherry picked from commit 2dca70d) --- .../processors/cache/GridCacheEntryEx.java | 5 ----- .../processors/cache/GridCacheMapEntry.java | 19 ++++--------------- .../version/GridCacheVersionManager.java | 3 +++ .../cache/GridCacheTestEntryEx.java | 5 ----- 4 files changed, 7 insertions(+), 25 deletions(-) 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 6cddb2da10a06..fc374bb25578c 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 @@ -98,11 +98,6 @@ public interface GridCacheEntryEx { */ public int partition(); - /** - * @return Start version. - */ - public long startVersion(); - /** * @return Key. */ 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 767c314773055..faf44601563ec 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 @@ -61,6 +61,7 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionEx; +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.schema.SchemaIndexCacheFilter; @@ -141,10 +142,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme @GridToStringInclude protected CacheObject val; - /** Start version. */ - @GridToStringInclude - protected final long startVer; - /** Version. */ @GridToStringInclude protected GridCacheVersion ver; @@ -190,14 +187,7 @@ protected GridCacheMapEntry( this.hash = key.hashCode(); this.cctx = cctx; - ver = cctx.versions().next(); - - startVer = ver.order(); - } - - /** {@inheritDoc} */ - @Override public long startVersion() { - return startVer; + ver = GridCacheVersionManager.START_VER; } /** @@ -302,7 +292,7 @@ protected void value(@Nullable CacheObject val) { * @return {@code True} if start version. */ public boolean isStartVersion() { - return ver.nodeOrder() == cctx.localNode().order() && ver.order() == startVer; + return ver == GridCacheVersionManager.START_VER; } /** {@inheritDoc} */ @@ -2734,8 +2724,7 @@ protected final boolean hasValueUnlocked() { GridCacheVersion currentVer = row != null ? row.version() : GridCacheMapEntry.this.ver; - boolean isStartVer = currentVer.nodeOrder() == cctx.localNode().order() - && currentVer.order() == startVer; + boolean isStartVer = currentVer == GridCacheVersionManager.START_VER; if (cctx.group().persistenceEnabled()) { if (!isStartVer) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java index 269925d42671f..df8af4811b7b3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java @@ -41,6 +41,9 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter { /** */ public static final GridCacheVersion EVICT_VER = new GridCacheVersion(Integer.MAX_VALUE, 0, 0, 0); + /** */ + public static final GridCacheVersion START_VER = new GridCacheVersion(0, 0, 0, 0); + /** Timestamp used as base time for cache topology version (January 1, 2014). */ public static final long TOP_VER_BASE_TIME = 1388520000000L; 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 e40700dd15119..daf96c46146de 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 @@ -853,11 +853,6 @@ GridCacheMvccCandidate anyOwner() { return false; } - /** {@inheritDoc} */ - @Override public long startVersion() { - return 0; - } - /** {@inheritDoc} */ @Nullable @Override public CacheObject peek(boolean heap, boolean offheap, From 1ef76a1f969c598716e8ee4bd5437b9193f82e12 Mon Sep 17 00:00:00 2001 From: ezhuravl Date: Fri, 22 Jun 2018 20:38:45 +0300 Subject: [PATCH 0291/1463] IGNITE-4939 Receive event before cache initialized. - Fixes #4226. Signed-off-by: Dmitriy Pavlov (cherry picked from commit f427b9d) --- .../processors/cache/GridCacheProcessor.java | 7 + .../service/SystemCacheNotConfiguredTest.java | 172 ++++++++++++++++++ .../testsuites/IgniteKernalSelfTestSuite.java | 2 + 3 files changed, 181 insertions(+) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/service/SystemCacheNotConfiguredTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 6cecf5c8774de..d995a88774113 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 @@ -3990,6 +3990,13 @@ public IgniteCacheProxy jcache(String name) { IgniteCacheProxy cache = (IgniteCacheProxy) jCacheProxies.get(name); + if (cache == null) { + GridCacheAdapter cacheAdapter = caches.get(name); + + if (cacheAdapter != null) + cache = new IgniteCacheProxyImpl(cacheAdapter.context(), cacheAdapter, false); + } + if (cache == null) throw new IllegalArgumentException("Cache is not configured: " + name); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/SystemCacheNotConfiguredTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/SystemCacheNotConfiguredTest.java new file mode 100644 index 0000000000000..a76eb22585a64 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/SystemCacheNotConfiguredTest.java @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.service; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteServices; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.services.Service; +import org.apache.ignite.services.ServiceConfiguration; +import org.apache.ignite.services.ServiceContext; +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; + +/** + * Tests if system cache was started before deploying of service. + */ +public class SystemCacheNotConfiguredTest extends GridCommonAbstractTest { + /** */ + private final ByteArrayOutputStream errContent = new ByteArrayOutputStream(); + + /** */ + private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private final PrintStream originalErr = System.err; + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 60 * 1000; + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + TcpDiscoverySpi discoverySpi = new TcpDiscoverySpi(); + + discoverySpi.setIpFinder(ipFinder); + cfg.setDiscoverySpi(discoverySpi); + + if("server".equals(igniteInstanceName)) + cfg.setServiceConfiguration(serviceConfiguration()); + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void test() throws Exception { + captureErr(); + + new Thread(this::startServer).start(); + + Ignite client = startGrid(getConfiguration("client").setClientMode(true)); + + IgniteServices services = client.services(); + + SimpleService srvc = services.serviceProxy("service", SimpleService.class, false); + + Thread.sleep(1000); + + srvc.isWorking(); + + assertFalse(getErr().contains("Cache is not configured:")); + } + + /** + * Start server node. + */ + private void startServer() { + try { + startGrid(getConfiguration("server")); + } + catch (Exception e) { + fail(); + } + } + + /** + * @return Service configuration. + */ + private ServiceConfiguration serviceConfiguration() { + ServiceConfiguration svcCfg = new ServiceConfiguration(); + + svcCfg.setName("service"); + svcCfg.setTotalCount(1); + svcCfg.setService(new SimpleServiceImpl()); + + return svcCfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + System.setErr(originalErr); + } + + /** + * Turns on stdErr output capture. + */ + private void captureErr() { + System.setErr(new PrintStream(errContent)); + } + + /** + * Turns off stdErr capture and returns the contents that have been captured. + * + * @return String of captured stdErr. + */ + private String getErr() { + return errContent.toString().replaceAll("\r", ""); + } + + /** + * Simple service implementation for test. + */ + public static class SimpleServiceImpl implements Service, SimpleService { + /** {@inheritDoc} */ + SimpleServiceImpl() { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void cancel(ServiceContext ctx) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void init(ServiceContext ctx) throws Exception { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void execute(ServiceContext ctx) throws Exception { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void isWorking() { + // No-op. + } + } + + /** + * Simple service interface for test. + */ + public interface SimpleService { + /** */ + void isWorking(); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java index 2a6683352ad45..f2fbe046f0850 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java @@ -72,6 +72,7 @@ import org.apache.ignite.internal.processors.service.ServiceDeploymentOnActivationTest; import org.apache.ignite.internal.processors.service.ServiceDeploymentOutsideBaselineTest; import org.apache.ignite.internal.processors.service.ServicePredicateAccessCacheTest; +import org.apache.ignite.internal.processors.service.SystemCacheNotConfiguredTest; import org.apache.ignite.internal.util.GridStartupWithUndefinedIgniteHomeSelfTest; import org.apache.ignite.services.ServiceThreadPoolSelfTest; import org.apache.ignite.spi.communication.GridCacheMessageSelfTest; @@ -152,6 +153,7 @@ public static TestSuite suite(Set ignoredTests) throws Exception { suite.addTestSuite(GridServiceDeploymentCompoundFutureSelfTest.class); suite.addTestSuite(ServiceDeploymentOnActivationTest.class); suite.addTestSuite(ServiceDeploymentOutsideBaselineTest.class); + suite.addTestSuite(SystemCacheNotConfiguredTest.class); suite.addTestSuite(IgniteServiceDeploymentClassLoadingDefaultMarshallerTest.class); suite.addTestSuite(IgniteServiceDeploymentClassLoadingJdkMarshallerTest.class); From 19d14243227e8966dfff136f1a3d8e20c37a8aae Mon Sep 17 00:00:00 2001 From: Alexander Menshikov Date: Fri, 22 Jun 2018 21:34:26 +0300 Subject: [PATCH 0292/1463] IGNITE-8238 Fix for Operation can fails with unexpected RuntimeException when node is stopping. - Fixes #3993. Signed-off-by: Dmitriy Pavlov (cherry picked from commit 21cd24c) --- .../internal/processors/cache/GridCacheTtlManager.java | 9 +++++++++ .../persistence/GridCacheDatabaseSharedManager.java | 3 ++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java index d36485ab70a7f..ff6ed65bd03cd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java @@ -21,6 +21,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; @@ -214,6 +215,14 @@ public boolean expire(int amount) { catch (IgniteCheckedException e) { U.error(log, "Failed to process entry expiration: " + e, e); } + catch (IgniteException e) { + if (e.hasCause(NodeStoppingException.class)) { + if (log.isDebugEnabled()) + log.debug("Failed to expire because node is stopped: " + e); + } + else + throw e; + } return false; } 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 e2fe57260e35e..a52922de8d63d 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 @@ -1460,6 +1460,7 @@ private void prepareIndexRebuildFuture(int cacheId) { /** * Gets the checkpoint read lock. While this lock is held, checkpoint thread will not acquireSnapshotWorker memory * state. + * @throws IgniteException If failed. */ @SuppressWarnings("LockAcquiredButNotSafelyReleased") @Override public void checkpointReadLock() { @@ -1472,7 +1473,7 @@ private void prepareIndexRebuildFuture(int cacheId) { if (stopping) { checkpointLock.readLock().unlock(); - throw new RuntimeException("Failed to perform cache update: node is stopping."); + throw new IgniteException(new NodeStoppingException("Failed to perform cache update: node is stopping.")); } if (safeToUpdatePageMemories() || checkpointLock.getReadHoldCount() > 1) From 5de6869804bee5abfc429691ce32be0460641585 Mon Sep 17 00:00:00 2001 From: vd-pyatkov Date: Mon, 25 Jun 2018 17:04:15 +0300 Subject: [PATCH 0293/1463] IGNITE-8594 Make error messages in validate_indexes command report more informative - Fixes #4197. Signed-off-by: Ivan Rakov (cherry picked from commit 594df60) --- .../visor/verify/ValidateIndexesClosure.java | 15 +++++++++++++-- .../util/GridCommandHandlerIndexingTest.java | 3 +++ 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java index e01dca2c92d8f..e3aebc3121919 100644 --- a/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java +++ b/modules/indexing/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java @@ -505,12 +505,21 @@ private Map processIndex(GridCacheContex long current = 0; long processedNumber = 0; + KeyCacheObject previousKey = null; + while (!enoughIssues) { KeyCacheObject h2key = null; try { - if (!cursor.next()) - break; + try { + if (!cursor.next()) + break; + } + catch (IllegalStateException e) { + throw new IgniteCheckedException("Key is present in SQL index, but is missing in corresponding " + + "data page. Previous successfully read key: " + + CacheObjectUtils.unwrapBinaryIfNeeded(ctx.cacheObjectContext(), previousKey, true, true), e); + } GridH2Row h2Row = (GridH2Row)cursor.get(); @@ -541,6 +550,8 @@ else if (current++ % checkThrough > 0) if (cacheDataStoreRow == null) throw new IgniteCheckedException("Key is present in SQL index, but can't be found in CacheDataTree."); + + previousKey = h2key; } catch (Throwable t) { Object o = CacheObjectUtils.unwrapBinaryIfNeeded( diff --git a/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java b/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java index ca9aa5372f552..1a274e5fcc7f0 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/util/GridCommandHandlerIndexingTest.java @@ -110,6 +110,9 @@ public void testBrokenCacheDataTreeShouldFailValidation() throws Exception { "checkThrough", "10")); assertTrue(testOut.toString().contains("validate_indexes has finished with errors")); + + assertTrue(testOut.toString().contains( + "Key is present in SQL index, but is missing in corresponding data page.")); } /** From cbbde37edcf975c10f8f0cb28341c6d6926e942b Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Tue, 26 Jun 2018 12:23:33 +0300 Subject: [PATCH 0294/1463] IGNITE-8768 Fixed JVM crash caused by an in-progress partition eviction during cache stop - Fixes #4227. Signed-off-by: Alexey Goncharuk (cherry picked from commit 56975c2) --- .../processors/cache/CacheGroupContext.java | 2 + .../distributed/dht/EvictionContext.java | 28 ++ .../dht/GridDhtLocalPartition.java | 14 +- .../dht/GridDhtPartitionsEvictor.java | 326 +++++++++++++----- 4 files changed, 286 insertions(+), 84 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/EvictionContext.java 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 8a650380f2518..99f9f97ec942b 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 @@ -723,6 +723,8 @@ void stopGroup() { IgniteCheckedException err = new IgniteCheckedException("Failed to wait for topology update, cache (or node) is stopping."); + evictor.stop(); + aff.cancelFutures(err); preldr.onKernalStop(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/EvictionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/EvictionContext.java new file mode 100644 index 0000000000000..0964c3c3cc5c1 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/EvictionContext.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.distributed.dht; + +/** + * Additional context for partition eviction process. + */ +public interface EvictionContext { + /** + * @return {@code true} If eviction process should be stopped. + */ + public boolean shouldStop(); +} 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 81249b613b246..e13c952644339 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 @@ -850,14 +850,14 @@ public boolean isClearing() { } /** - * Tries to start partition clear process {@link GridDhtLocalPartition#clearAll()}). + * Tries to start partition clear process {@link GridDhtLocalPartition#clearAll(EvictionContext)}). * Only one thread is allowed to do such process concurrently. * At the end of clearing method completes {@code clearFuture}. * * @return {@code false} if clearing is not started due to existing reservations. * @throws NodeStoppingException If node is stopping. */ - public boolean tryClear() throws NodeStoppingException { + public boolean tryClear(EvictionContext evictionCtx) throws NodeStoppingException { if (clearFuture.isDone()) return true; @@ -869,7 +869,7 @@ public boolean tryClear() throws NodeStoppingException { if (addEvicting()) { try { // Attempt to evict partition entries from cache. - long clearedEntities = clearAll(); + long clearedEntities = clearAll(evictionCtx); if (log.isDebugEnabled()) log.debug("Partition is cleared [clearedEntities=" + clearedEntities + ", part=" + this + "]"); @@ -985,7 +985,7 @@ public long fullSize() { * @return Number of rows cleared from page memory. * @throws NodeStoppingException If node stopping. */ - private long clearAll() throws NodeStoppingException { + private long clearAll(EvictionContext evictionCtx) throws NodeStoppingException { GridCacheVersion clearVer = ctx.versions().next(); GridCacheObsoleteEntryExtras extras = new GridCacheObsoleteEntryExtras(clearVer); @@ -1001,6 +1001,8 @@ private long clearAll() throws NodeStoppingException { long cleared = 0; + final int stopCheckingFreq = 1000; + CacheMapHolder hld = grp.sharedGroup() ? null : singleCacheEntryMap; try { @@ -1051,6 +1053,10 @@ private long clearAll() throws NodeStoppingException { cleared++; } + + // For each 'stopCheckingFreq' cleared entities check clearing process to stop. + if (cleared % stopCheckingFreq == 0 && evictionCtx.shouldStop()) + return cleared; } catch (GridDhtInvalidPartitionException e) { assert isEmpty() && state() == EVICTED : "Invalid error [e=" + e + ", part=" + this + ']'; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsEvictor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsEvictor.java index 2a289212ca30c..72063974b6d13 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsEvictor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsEvictor.java @@ -16,21 +16,32 @@ */ package org.apache.ignite.internal.processors.cache.distributed.dht; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.function.Function; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; -import org.apache.ignite.internal.util.typedef.internal.GPC; +import org.apache.ignite.internal.util.GridConcurrentHashSet; +import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.U; /** - * Class that serves asynchronous partition eviction process. + * Class that serves asynchronous part eviction process. + * Only one partition from group can be evicted at the moment. */ public class GridDhtPartitionsEvictor { - /** Show eviction progress frequency in ms. */ - private static final int SHOW_EVICTION_PROGRESS_FREQ_MS = 2 * 60 * 1000; // 2 Minutes. + /** Default eviction progress show frequency. */ + private static final int DEFAULT_SHOW_EVICTION_PROGRESS_FREQ_MS = 2 * 60 * 1000; // 2 Minutes. + + /** Eviction progress frequency property name. */ + private static final String SHOW_EVICTION_PROGRESS_FREQ = "SHOW_EVICTION_PROGRESS_FREQ"; /** */ private final GridCacheSharedContext ctx; @@ -41,11 +52,31 @@ public class GridDhtPartitionsEvictor { /** */ private final IgniteLogger log; + /** Lock object. */ + private final Object mux = new Object(); + /** Queue contains partitions scheduled for eviction. */ - private final ConcurrentHashMap evictionQueue = new ConcurrentHashMap<>(); + private final DeduplicationQueue evictionQueue = new DeduplicationQueue<>(GridDhtLocalPartition::id); + + /** + * Flag indicates that eviction process is running at the moment. + * This is needed to schedule partition eviction if there are no currently running self-scheduling eviction tasks. + * Guarded by {@link #mux}. + */ + private boolean evictionRunning; - /** Flag indicates that eviction process is running at the moment, false in other case. */ - private final AtomicBoolean evictionRunning = new AtomicBoolean(); + /** Flag indicates that eviction process has stopped. */ + private volatile boolean stop; + + /** Future for currently running partition eviction task. */ + private volatile GridFutureAdapter evictionFut; + + /** Eviction progress frequency in ms. */ + private final long evictionProgressFreqMs = IgniteSystemProperties.getLong(SHOW_EVICTION_PROGRESS_FREQ, + DEFAULT_SHOW_EVICTION_PROGRESS_FREQ_MS); + + /** Next time of show eviction progress. */ + private long nextShowProgressTime; /** * Constructor. @@ -67,78 +98,213 @@ public GridDhtPartitionsEvictor(CacheGroupContext grp) { * @param part Partition to evict. */ public void evictPartitionAsync(GridDhtLocalPartition part) { - evictionQueue.putIfAbsent(part.id(), part); - - if (evictionRunning.compareAndSet(false, true)) { - ctx.kernalContext().closure().callLocalSafe(new GPC() { - @Override public Boolean call() { - boolean locked = true; - - long nextShowProgressTime = U.currentTimeMillis() + SHOW_EVICTION_PROGRESS_FREQ_MS; - - while (locked || !evictionQueue.isEmpty()) { - if (!locked && !evictionRunning.compareAndSet(false, true)) - return false; - - try { - for (GridDhtLocalPartition part : evictionQueue.values()) { - // Show progress of currently evicting partitions. - if (U.currentTimeMillis() >= nextShowProgressTime) { - if (log.isInfoEnabled()) - log.info("Eviction in progress [grp=" + grp.cacheOrGroupName() - + ", remainingCnt=" + evictionQueue.size() + "]"); - - nextShowProgressTime = U.currentTimeMillis() + SHOW_EVICTION_PROGRESS_FREQ_MS; - } - - try { - boolean success = part.tryClear(); - - if (success) { - evictionQueue.remove(part.id()); - - if (part.state() == GridDhtPartitionState.EVICTED && part.markForDestroy()) - part.destroy(); - } - } - catch (Throwable ex) { - if (ctx.kernalContext().isStopping()) { - LT.warn(log, ex, "Partition eviction failed (current node is stopping).", - false, - true); - - evictionQueue.clear(); - - return true; - } - else - LT.error(log, ex, "Partition eviction failed, this can cause grid hang."); - } - } - } - finally { - if (!evictionQueue.isEmpty()) { - if (ctx.kernalContext().isStopping()) { - evictionQueue.clear(); - - locked = false; - } - else - locked = true; - } - else { - boolean res = evictionRunning.compareAndSet(true, false); - - assert res; - - locked = false; - } - } - } - - return true; + if (stop) + return; + + boolean added = evictionQueue.offer(part); + + if (!added) + return; + + synchronized (mux) { + if (!evictionRunning) { + nextShowProgressTime = U.currentTimeMillis() + evictionProgressFreqMs; + + scheduleNextPartitionEviction(); + } + } + } + + /** + * Stops eviction process. + * Method awaits last offered partition eviction. + */ + public void stop() { + stop = true; + + synchronized (mux) { + // Wait for last offered partition eviction completion. + IgniteInternalFuture evictionFut0 = evictionFut; + + if (evictionFut0 != null) { + try { + evictionFut0.get(); + } + catch (IgniteCheckedException e) { + if (log.isDebugEnabled()) + log.warning("Failed to await partition eviction during stopping", e); + } + } + } + } + + /** + * Gets next partition from the queue and schedules it for eviction. + */ + private void scheduleNextPartitionEviction() { + if (stop) + return; + + synchronized (mux) { + GridDhtLocalPartition next = evictionQueue.poll(); + + if (next != null) { + showProgress(); + + evictionFut = new GridFutureAdapter<>(); + + ctx.kernalContext().closure().callLocalSafe(new PartitionEvictionTask(next, () -> stop), true); + } + else + evictionRunning = false; + } + } + + /** + * Shows progress of eviction. + */ + private void showProgress() { + if (U.currentTimeMillis() >= nextShowProgressTime) { + int size = evictionQueue.size() + 1; // Queue size plus current partition. + + if (log.isInfoEnabled()) + log.info("Eviction in progress [grp=" + grp.cacheOrGroupName() + + ", remainingPartsCnt=" + size + "]"); + + nextShowProgressTime = U.currentTimeMillis() + evictionProgressFreqMs; + } + } + + /** + * Task for self-scheduled partition eviction / clearing. + */ + private class PartitionEvictionTask implements Callable { + /** Partition to evict. */ + private final GridDhtLocalPartition part; + + /** Eviction context. */ + private final EvictionContext evictionCtx; + + /** + * @param part Partition. + * @param evictionCtx Eviction context. + */ + public PartitionEvictionTask(GridDhtLocalPartition part, EvictionContext evictionCtx) { + this.part = part; + this.evictionCtx = evictionCtx; + } + + /** {@inheritDoc} */ + @Override public Boolean call() throws Exception { + if (stop) { + evictionFut.onDone(); + + return false; + } + + try { + boolean success = part.tryClear(evictionCtx); + + if (success) { + if (part.state() == GridDhtPartitionState.EVICTED && part.markForDestroy()) + part.destroy(); } - }, /*system pool*/ true); + else // Re-offer partition if clear was unsuccessful due to partition reservation. + evictionQueue.offer(part); + + // Complete eviction future before schedule new to prevent deadlock with + // simultaneous eviction stopping and scheduling new eviction. + evictionFut.onDone(); + + scheduleNextPartitionEviction(); + + return true; + } + catch (Throwable ex) { + evictionFut.onDone(ex); + + if (ctx.kernalContext().isStopping()) { + LT.warn(log, ex, "Partition eviction failed (current node is stopping).", + false, + true); + } + else + LT.error(log, ex, "Partition eviction failed, this can cause grid hang."); + } + + return false; + } + } + + /** + * Thread-safe blocking queue with items deduplication. + * + * @param Key type of item used for deduplication. + * @param Queue item type. + */ + private static class DeduplicationQueue { + /** Queue. */ + private final Queue queue; + + /** Unique items set. */ + private final Set uniqueItems; + + /** Key mapping function. */ + private final Function keyMappingFunction; + + /** + * Constructor. + * + * @param keyExtractor Function to extract a key from a queue item. + * This key is used for deduplication if some item has offered twice. + */ + public DeduplicationQueue(Function keyExtractor) { + keyMappingFunction = keyExtractor; + queue = new LinkedBlockingQueue<>(); + uniqueItems = new GridConcurrentHashSet<>(); + } + + /** + * Offers item to the queue. + * + * @param item Item. + * @return {@code true} if item has been successfully offered to the queue, + * {@code false} if item was rejected because already exists in the queue. + */ + public boolean offer(V item) { + K key = keyMappingFunction.apply(item); + + if (uniqueItems.add(key)) { + queue.offer(item); + + return true; + } + + return false; + } + + /** + * Polls next item from queue. + * + * @return Next item or {@code null} if queue is empty. + */ + public V poll() { + V item = queue.poll(); + + if (item != null) { + K key = keyMappingFunction.apply(item); + + uniqueItems.remove(key); + } + + return item; + } + + /** + * @return Size of queue. + */ + public int size() { + return queue.size(); } } } From 10153ac0fc17bc3e02db14eb2a0a4dfead2e8f25 Mon Sep 17 00:00:00 2001 From: Ilya Lantukh Date: Wed, 27 Jun 2018 14:53:05 +0300 Subject: [PATCH 0295/1463] IGNITE-8752 Deadlock when registering binary metadata while holding topology read lock - Fixes #4237. Signed-off-by: Ivan Rakov (cherry picked from commit 929a12d) --- .../internal/UnregisteredClassException.java | 74 +++++++++++++++++++ .../ignite/internal/binary/BinaryContext.java | 20 +++-- .../internal/binary/BinaryEnumObjectImpl.java | 2 +- .../internal/binary/BinaryMarshaller.java | 2 +- .../internal/binary/BinaryReaderExImpl.java | 2 +- .../ignite/internal/binary/BinaryUtils.java | 4 +- .../internal/binary/BinaryWriterExImpl.java | 28 +++++-- .../internal/binary/GridBinaryMarshaller.java | 5 +- .../binary/builder/BinaryBuilderEnum.java | 2 +- .../builder/BinaryBuilderSerializer.java | 2 +- .../builder/BinaryEnumArrayLazyValue.java | 2 +- .../builder/BinaryObjectArrayLazyValue.java | 2 +- .../builder/BinaryObjectBuilderImpl.java | 2 +- .../client/thin/ClientBinaryMarshaller.java | 2 +- .../CacheDefaultBinaryAffinityKeyMapper.java | 2 +- .../processors/cache/CacheGroupContext.java | 10 +++ .../processors/cache/GridCacheContext.java | 2 +- .../processors/cache/GridCacheReturn.java | 5 ++ .../binary/CacheObjectBinaryProcessor.java | 3 +- .../CacheObjectBinaryProcessorImpl.java | 43 ++++++----- .../cache/binary/IgniteBinaryImpl.java | 2 +- .../dht/GridClientPartitionTopology.java | 5 ++ .../dht/GridDhtPartitionTopology.java | 4 + .../dht/GridDhtPartitionTopologyImpl.java | 5 ++ .../dht/atomic/GridDhtAtomicCache.java | 70 +++++++++++------- .../IgniteCacheObjectProcessor.java | 11 +++ .../IgniteCacheObjectProcessorImpl.java | 6 ++ .../util/StripedCompositeReadWriteLock.java | 25 ++++++- .../IgniteCacheEntryProcessorCallTest.java | 5 ++ 29 files changed, 273 insertions(+), 74 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/UnregisteredClassException.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/UnregisteredClassException.java b/modules/core/src/main/java/org/apache/ignite/internal/UnregisteredClassException.java new file mode 100644 index 0000000000000..6da7daad5941d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/UnregisteredClassException.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; + +import org.apache.ignite.IgniteException; +import org.jetbrains.annotations.Nullable; + +/** + * Exception thrown during serialization if class isn't registered and it's registration isn't allowed. + */ +public class UnregisteredClassException extends IgniteException { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private final Class cls; + + /** + * @param cls Class that isn't registered. + */ + public UnregisteredClassException(Class cls) { + this.cls = cls; + } + + /** + * @param msg Error message. + * @param cls Class that isn't registered. + */ + public UnregisteredClassException(String msg, Class cls) { + super(msg); + this.cls = cls; + } + + /** + * @param cause Exception cause. + * @param cls Class that isn't registered. + */ + public UnregisteredClassException(Throwable cause, Class cls) { + super(cause); + this.cls = cls; + } + + /** + * @param msg Error message. + * @param cause Exception cause. + * @param cls Class that isn't registered. + */ + public UnregisteredClassException(String msg, @Nullable Throwable cause, Class cls) { + super(msg, cause); + this.cls = cls; + } + + /** + * @return Class that isn't registered. + */ + public Class cls() { + return cls; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java index 233769660c9f9..01215703f4e2c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java @@ -48,6 +48,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.UnregisteredClassException; import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.binary.BinaryBasicIdMapper; import org.apache.ignite.binary.BinaryBasicNameMapper; @@ -610,17 +611,22 @@ else if (cpElement.isFile()) { /** * @param cls Class. + * @param failIfUnregistered Throw exception if class isn't registered. * @return Class descriptor. * @throws BinaryObjectException In case of error. */ - public BinaryClassDescriptor descriptorForClass(Class cls, boolean deserialize) + public BinaryClassDescriptor descriptorForClass(Class cls, boolean deserialize, boolean failIfUnregistered) throws BinaryObjectException { assert cls != null; BinaryClassDescriptor desc = descByCls.get(cls); - if (desc == null) + if (desc == null) { + if (failIfUnregistered) + throw new UnregisteredClassException(cls); + desc = registerClassDescriptor(cls, deserialize); + } else if (!desc.registered()) { if (!desc.userType()) { BinaryClassDescriptor desc0 = new BinaryClassDescriptor( @@ -652,8 +658,12 @@ else if (!desc.registered()) { return desc0; } } - else + else { + if (failIfUnregistered) + throw new UnregisteredClassException(cls); + desc = registerUserClassDescriptor(desc); + } } return desc; @@ -1176,8 +1186,8 @@ public void registerUserTypesSchema() { /** * Register "type ID to class name" mapping on all nodes to allow for mapping requests resolution form client. * Other {@link BinaryContext}'s "register" methods and method - * {@link BinaryContext#descriptorForClass(Class, boolean)} already call this functionality so use this method - * only when registering class names whose {@link Class} is unknown. + * {@link BinaryContext#descriptorForClass(Class, boolean, boolean)} already call this functionality + * so use this method only when registering class names whose {@link Class} is unknown. * * @param typeId Type ID. * @param clsName Class Name. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java index 12a0fc352b99d..275169561fd56 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java @@ -437,6 +437,6 @@ public BinaryEnumObjectImpl(BinaryContext ctx, byte[] arr) { * binary enum. */ public boolean isTypeEquals(final Class cls) { - return ctx.descriptorForClass(cls, false).typeId() == typeId(); + return ctx.descriptorForClass(cls, false, false).typeId() == typeId(); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java index dfc726e81e8d3..bfb0e1018a91c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMarshaller.java @@ -79,7 +79,7 @@ private void setBinaryContext(BinaryContext ctx, IgniteConfiguration cfg) { /** {@inheritDoc} */ @Override protected byte[] marshal0(@Nullable Object obj) throws IgniteCheckedException { - return impl.marshal(obj); + return impl.marshal(obj, false); } /** {@inheritDoc} */ 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 f88e3c3234b41..ab1f874386dd9 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 @@ -265,7 +265,7 @@ public BinaryReaderExImpl(BinaryContext ctx, if (forUnmarshal) { // Registers class by type ID, at least locally if the cache is not ready yet. - desc = ctx.descriptorForClass(BinaryUtils.doReadClass(in, ctx, ldr, typeId0), false); + desc = ctx.descriptorForClass(BinaryUtils.doReadClass(in, ctx, ldr, typeId0), false, false); typeId = desc.typeId(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java index 082cc206e3231..553d8e5b79bd9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java @@ -1625,7 +1625,7 @@ public static Class doReadClass(BinaryInputStream in, BinaryContext ctx, ClassLo } // forces registering of class by type id, at least locally - ctx.descriptorForClass(cls, true); + ctx.descriptorForClass(cls, true, false); } return cls; @@ -1655,7 +1655,7 @@ public static Class resolveClass(BinaryContext ctx, int typeId, @Nullable String } // forces registering of class by type id, at least locally - ctx.descriptorForClass(cls, true); + ctx.descriptorForClass(cls, true, false); } return cls; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java index a7f645c1f4763..3d93e7088af62 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryWriterExImpl.java @@ -82,6 +82,9 @@ public class BinaryWriterExImpl implements BinaryWriter, BinaryRawWriterEx, Obje /** */ private BinaryInternalMapper mapper; + /** */ + private boolean failIfUnregistered; + /** * @param ctx Context. */ @@ -112,6 +115,13 @@ public BinaryWriterExImpl(BinaryContext ctx, BinaryOutputStream out, BinaryWrite start = out.position(); } + /** + * @param failIfUnregistered Fail if unregistered. + */ + public void failIfUnregistered(boolean failIfUnregistered) { + this.failIfUnregistered = failIfUnregistered; + } + /** * @param typeId Type ID. */ @@ -161,7 +171,7 @@ private void marshal0(Object obj, boolean enableReplace) throws BinaryObjectExce Class cls = obj.getClass(); - BinaryClassDescriptor desc = ctx.descriptorForClass(cls, false); + BinaryClassDescriptor desc = ctx.descriptorForClass(cls, false, failIfUnregistered); if (desc == null) throw new BinaryObjectException("Object is not binary: [class=" + cls + ']'); @@ -724,7 +734,10 @@ void doWriteObjectArray(@Nullable Object[] val) throws BinaryObjectException { if (tryWriteAsHandle(val)) return; - BinaryClassDescriptor desc = ctx.descriptorForClass(val.getClass().getComponentType(), false); + BinaryClassDescriptor desc = ctx.descriptorForClass( + val.getClass().getComponentType(), + false, + failIfUnregistered); out.unsafeEnsure(1 + 4); out.unsafeWriteByte(GridBinaryMarshaller.OBJ_ARR); @@ -795,7 +808,7 @@ void doWriteEnum(@Nullable Enum val) { if (val == null) out.writeByte(GridBinaryMarshaller.NULL); else { - BinaryClassDescriptor desc = ctx.descriptorForClass(val.getDeclaringClass(), false); + BinaryClassDescriptor desc = ctx.descriptorForClass(val.getDeclaringClass(), false, failIfUnregistered); out.unsafeEnsure(1 + 4); @@ -848,7 +861,10 @@ void doWriteEnumArray(@Nullable Object[] val) { if (val == null) out.writeByte(GridBinaryMarshaller.NULL); else { - BinaryClassDescriptor desc = ctx.descriptorForClass(val.getClass().getComponentType(), false); + BinaryClassDescriptor desc = ctx.descriptorForClass( + val.getClass().getComponentType(), + false, + failIfUnregistered); out.unsafeEnsure(1 + 4); @@ -877,7 +893,7 @@ void doWriteClass(@Nullable Class val) { if (val == null) out.writeByte(GridBinaryMarshaller.NULL); else { - BinaryClassDescriptor desc = ctx.descriptorForClass(val, false); + BinaryClassDescriptor desc = ctx.descriptorForClass(val, false, failIfUnregistered); out.unsafeEnsure(1 + 4); @@ -906,7 +922,7 @@ public void doWriteProxy(Proxy proxy, Class[] intfs) { out.unsafeWriteInt(intfs.length); for (Class intf : intfs) { - BinaryClassDescriptor desc = ctx.descriptorForClass(intf, false); + BinaryClassDescriptor desc = ctx.descriptorForClass(intf, false, failIfUnregistered); if (desc.registered()) out.writeInt(desc.typeId()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java index d6c8abdcfb35a..743958932f21b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java @@ -240,14 +240,17 @@ public GridBinaryMarshaller(BinaryContext ctx) { /** * @param obj Object to marshal. + * @param failIfUnregistered Throw exception if class isn't registered. * @return Byte array. * @throws org.apache.ignite.binary.BinaryObjectException In case of error. */ - public byte[] marshal(@Nullable Object obj) throws BinaryObjectException { + public byte[] marshal(@Nullable Object obj, boolean failIfUnregistered) throws BinaryObjectException { if (obj == null) return new byte[] { NULL }; try (BinaryWriterExImpl writer = new BinaryWriterExImpl(ctx)) { + writer.failIfUnregistered(failIfUnregistered); + writer.marshal(obj); return writer.array(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderEnum.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderEnum.java index bc5eb9e030b35..3930c463528e2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderEnum.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderEnum.java @@ -63,7 +63,7 @@ public BinaryBuilderEnum(BinaryBuilderReader reader) { throw new BinaryInvalidTypeException("Failed to load the class: " + clsName, e); } - this.typeId = reader.binaryContext().descriptorForClass(cls, false).typeId(); + this.typeId = reader.binaryContext().descriptorForClass(cls, false, false).typeId(); } else { this.typeId = typeId; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java index 42f6873bf1548..5333cc4643a9f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryBuilderSerializer.java @@ -129,7 +129,7 @@ public void writeValue(BinaryWriterExImpl writer, Object val, boolean forceCol, writer.context().updateMetadata(typeId, meta); // Need register class for marshaller to be able to deserialize enum value. - writer.context().descriptorForClass(((Enum)val).getDeclaringClass(), false); + writer.context().descriptorForClass(((Enum)val).getDeclaringClass(), false, false); writer.writeByte(GridBinaryMarshaller.ENUM); writer.writeInt(typeId); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryEnumArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryEnumArrayLazyValue.java index 787ff638b995e..c0e79ec760594 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryEnumArrayLazyValue.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryEnumArrayLazyValue.java @@ -56,7 +56,7 @@ protected BinaryEnumArrayLazyValue(BinaryBuilderReader reader) { throw new BinaryInvalidTypeException("Failed to load the class: " + clsName, e); } - compTypeId = reader.binaryContext().descriptorForClass(cls, true).typeId(); + compTypeId = reader.binaryContext().descriptorForClass(cls, true, false).typeId(); } else { compTypeId = typeId; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectArrayLazyValue.java index 8962107c77ac8..d4882dc6fb462 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectArrayLazyValue.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectArrayLazyValue.java @@ -55,7 +55,7 @@ protected BinaryObjectArrayLazyValue(BinaryBuilderReader reader) { throw new BinaryInvalidTypeException("Failed to load the class: " + clsName, e); } - compTypeId = reader.binaryContext().descriptorForClass(cls, true).typeId(); + compTypeId = reader.binaryContext().descriptorForClass(cls, true, false).typeId(); } else { compTypeId = typeId; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java index b9eb3e5e0267b..3fc5dc4bee0a8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java @@ -157,7 +157,7 @@ public BinaryObjectBuilderImpl(BinaryObjectImpl obj) { throw new BinaryInvalidTypeException("Failed to load the class: " + clsNameToWrite, e); } - this.typeId = ctx.descriptorForClass(cls, false).typeId(); + this.typeId = ctx.descriptorForClass(cls, false, false).typeId(); registeredType = false; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientBinaryMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientBinaryMarshaller.java index c68b8f909e3fa..aac6873e95fad 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientBinaryMarshaller.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientBinaryMarshaller.java @@ -66,7 +66,7 @@ public T unmarshal(BinaryInputStream in) { * Serializes Java object into a byte array. */ public byte[] marshal(Object obj) { - return impl.marshal(obj); + return impl.marshal(obj, false); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheDefaultBinaryAffinityKeyMapper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheDefaultBinaryAffinityKeyMapper.java index 43506873fb427..385ed59cfd36a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheDefaultBinaryAffinityKeyMapper.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheDefaultBinaryAffinityKeyMapper.java @@ -63,7 +63,7 @@ public CacheDefaultBinaryAffinityKeyMapper(@Nullable CacheKeyConfiguration[] cac /** {@inheritDoc} */ @Override public Object affinityKey(Object key) { try { - key = proc.toBinary(key); + key = proc.toBinary(key, false); } catch (IgniteException e) { U.error(log, "Failed to marshal key to binary: " + key, e); 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 99f9f97ec942b..d2b98f2fead39 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 @@ -580,6 +580,16 @@ public GridDhtPartitionTopology topology() { return top; } + /** + * @return {@code True} if current thread holds lock on topology. + */ + public boolean isTopologyLocked() { + if (top == null) + return false; + + return top.holdsLock(); + } + /** * @return Offheap manager. */ 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 c5f4d3c9c993a..191734b02f9bc 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 @@ -1799,7 +1799,7 @@ public CacheObjectContext cacheObjectContext() { @Nullable public CacheObject toCacheObject(@Nullable Object obj) { assert validObjectForCache(obj) : obj; - return cacheObjects().toCacheObject(cacheObjCtx, obj, true); + return cacheObjects().toCacheObject(cacheObjCtx, obj, true, grp.isTopologyLocked()); } /** 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 551d70d83761d..bc859312b371e 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 @@ -31,6 +31,7 @@ import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.internal.GridDirectCollection; import org.apache.ignite.internal.GridDirectTransient; +import org.apache.ignite.internal.UnregisteredClassException; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; @@ -242,6 +243,10 @@ public synchronized void addEntryProcessResult( v = resMap; } + // This exception means that we should register class and call EntryProcessor again. + if (err != null && err instanceof UnregisteredClassException) + throw (UnregisteredClassException) err; + CacheInvokeResult res0 = err == null ? CacheInvokeResult.fromResult(res) : CacheInvokeResult.fromError(err); Object resKey = key0 != null ? key0 : diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessor.java index 14dd5cbc9a35b..c7e2e689f7c06 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessor.java @@ -140,8 +140,9 @@ public void updateMetadata(int typeId, String typeName, @Nullable String affKeyF /** * @param obj Original object. + * @param failIfUnregistered Throw exception if class isn't registered. * @return Binary object (in case binary marshaller is used). * @throws IgniteException If failed. */ - public Object marshalToBinary(Object obj) throws IgniteException; + public Object marshalToBinary(Object obj, boolean failIfUnregistered) throws IgniteException; } 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 8e5ec5c924728..69d1f917ace69 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 @@ -295,7 +295,7 @@ public void addBinaryMetadataUpdateListener(BinaryMetadataUpdatedListener lsnr) * @throws BinaryObjectException If failed. */ public byte[] marshal(@Nullable Object obj) throws BinaryObjectException { - byte[] arr = binaryMarsh.marshal(obj); + byte[] arr = binaryMarsh.marshal(obj, false); assert arr.length > 0; @@ -330,7 +330,10 @@ public Object unmarshal(long ptr, boolean forceHeap) throws BinaryObjectExceptio /** {@inheritDoc} */ @SuppressWarnings("unchecked") - @Override public Object marshalToBinary(@Nullable Object obj) throws BinaryObjectException { + @Override public Object marshalToBinary( + @Nullable Object obj, + boolean failIfUnregistered + ) throws BinaryObjectException { if (obj == null) return null; @@ -343,7 +346,7 @@ public Object unmarshal(long ptr, boolean forceHeap) throws BinaryObjectExceptio Object[] pArr = new Object[arr.length]; for (int i = 0; i < arr.length; i++) - pArr[i] = marshalToBinary(arr[i]); + pArr[i] = marshalToBinary(arr[i], failIfUnregistered); return pArr; } @@ -352,9 +355,11 @@ public Object unmarshal(long ptr, boolean forceHeap) throws BinaryObjectExceptio IgniteBiTuple tup = (IgniteBiTuple)obj; if (obj instanceof T2) - return new T2<>(marshalToBinary(tup.get1()), marshalToBinary(tup.get2())); + return new T2<>(marshalToBinary(tup.get1(), failIfUnregistered), + marshalToBinary(tup.get2(), failIfUnregistered)); - return new IgniteBiTuple<>(marshalToBinary(tup.get1()), marshalToBinary(tup.get2())); + return new IgniteBiTuple<>(marshalToBinary(tup.get1(), failIfUnregistered), + marshalToBinary(tup.get2(), failIfUnregistered)); } { @@ -364,7 +369,7 @@ public Object unmarshal(long ptr, boolean forceHeap) throws BinaryObjectExceptio Collection col = (Collection)obj; for (Object item : col) - pCol.add(marshalToBinary(item)); + pCol.add(marshalToBinary(item, failIfUnregistered)); return (pCol instanceof MutableSingletonList) ? U.convertToSingletonList(pCol) : pCol; } @@ -377,7 +382,8 @@ public Object unmarshal(long ptr, boolean forceHeap) throws BinaryObjectExceptio Map map = (Map)obj; for (Map.Entry e : map.entrySet()) - pMap.put(marshalToBinary(e.getKey()), marshalToBinary(e.getValue())); + pMap.put(marshalToBinary(e.getKey(), failIfUnregistered), + marshalToBinary(e.getValue(), failIfUnregistered)); return pMap; } @@ -386,13 +392,14 @@ public Object unmarshal(long ptr, boolean forceHeap) throws BinaryObjectExceptio if (obj instanceof Map.Entry) { Map.Entry e = (Map.Entry)obj; - return new GridMapEntry<>(marshalToBinary(e.getKey()), marshalToBinary(e.getValue())); + return new GridMapEntry<>(marshalToBinary(e.getKey(), failIfUnregistered), + marshalToBinary(e.getValue(), failIfUnregistered)); } if (binaryMarsh.mustDeserialize(obj)) return obj; // No need to go through marshal-unmarshal because result will be the same as initial object. - byte[] arr = binaryMarsh.marshal(obj); + byte[] arr = binaryMarsh.marshal(obj, failIfUnregistered); assert arr.length > 0; @@ -765,7 +772,7 @@ public BinaryContext binaryContext() { if (!ctx.binaryEnabled() || binaryMarsh == null) return super.marshal(ctx, val); - byte[] arr = binaryMarsh.marshal(val); + byte[] arr = binaryMarsh.marshal(val, false); assert arr.length > 0; @@ -801,7 +808,7 @@ else if (key.partition() == -1) return key; } - obj = toBinary(obj); + obj = toBinary(obj, false); if (obj instanceof BinaryObjectImpl) { ((BinaryObjectImpl)obj).partition(partition(ctx, cctx, obj)); @@ -814,14 +821,14 @@ else if (key.partition() == -1) /** {@inheritDoc} */ @Nullable @Override public CacheObject toCacheObject(CacheObjectContext ctx, @Nullable Object obj, - boolean userObj) { + boolean userObj, boolean failIfUnregistered) { if (!ctx.binaryEnabled()) - return super.toCacheObject(ctx, obj, userObj); + return super.toCacheObject(ctx, obj, userObj, failIfUnregistered); if (obj == null || obj instanceof CacheObject) return (CacheObject)obj; - obj = toBinary(obj); + obj = toBinary(obj, failIfUnregistered); if (obj instanceof CacheObject) return (CacheObject)obj; @@ -864,18 +871,20 @@ else if (type == BinaryObjectImpl.TYPE_BINARY_ENUM) * @return Binary object. * @throws IgniteException In case of error. */ - @Nullable public Object toBinary(@Nullable Object obj) throws IgniteException { + @Nullable public Object toBinary(@Nullable Object obj, boolean failIfUnregistered) throws IgniteException { if (obj == null) return null; if (isBinaryObject(obj)) return obj; - return marshalToBinary(obj); + return marshalToBinary(obj, failIfUnregistered); } /** {@inheritDoc} */ - @Nullable @Override public IgniteNodeValidationResult validateNode(ClusterNode rmtNode, DiscoveryDataBag.JoiningNodeDiscoveryData discoData) { + @Nullable @Override public IgniteNodeValidationResult validateNode(ClusterNode rmtNode, + DiscoveryDataBag.JoiningNodeDiscoveryData discoData + ) { IgniteNodeValidationResult res; if (getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK) || !(marsh instanceof BinaryMarshaller)) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/IgniteBinaryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/IgniteBinaryImpl.java index e88819b0d3189..71475be7b0e23 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/IgniteBinaryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/IgniteBinaryImpl.java @@ -66,7 +66,7 @@ public IgniteBinaryImpl(GridKernalContext ctx, CacheObjectBinaryProcessor proc) guard(); try { - return (T)proc.marshalToBinary(obj); + return (T)proc.marshalToBinary(obj, false); } finally { unguard(); 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 fc80bbc0916c7..54a850c124510 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 @@ -198,6 +198,11 @@ private String mapString(GridDhtPartitionMap map) { lock.readLock().unlock(); } + /** {@inheritDoc} */ + @Override public boolean holdsLock() { + return lock.isWriteLockedByCurrentThread() || lock.getReadHoldCount() > 0; + } + /** {@inheritDoc} */ @Override public void updateTopologyVersion( GridDhtTopologyFuture exchFut, 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 b77dbd6e0d719..42ef309eafed5 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 @@ -58,6 +58,10 @@ public interface GridDhtPartitionTopology { */ public void readUnlock(); + /** + * @return {@code True} if locked by current thread. + */ + public boolean holdsLock(); /** * Updates topology version. * 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 45e85f9142a86..eb93020494b85 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 @@ -234,6 +234,11 @@ private String mapString(GridDhtPartitionMap map) { lock.readLock().unlock(); } + /** {@inheritDoc} */ + @Override public boolean holdsLock() { + return lock.isWriteLockedByCurrentThread() || lock.getReadHoldCount() > 0; + } + /** {@inheritDoc} */ @Override public void updateTopologyVersion( GridDhtTopologyFuture exchFut, 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 44f2b153db3ad..dbc298d68d37a 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 @@ -37,6 +37,7 @@ import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.NodeStoppingException; +import org.apache.ignite.internal.UnregisteredClassException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.mem.IgniteOutOfMemoryException; import org.apache.ignite.internal.pagemem.wal.StorageException; @@ -62,6 +63,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheUpdateAtomicResult; 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; @@ -84,6 +86,7 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionEx; +import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.util.GridLongList; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -1708,43 +1711,56 @@ private void updateAllAsyncInternal0( Collection> deleted = null; try { - GridDhtPartitionTopology top = topology(); + while (true) { + try { + GridDhtPartitionTopology top = topology(); - top.readLock(); + top.readLock(); - try { - if (top.stopping()) { - res.addFailedKeys(req.keys(), new CacheStoppedException(name())); + try { + if (top.stopping()) { + res.addFailedKeys(req.keys(), new CacheStoppedException(name())); - completionCb.apply(req, res); + completionCb.apply(req, res); - return; - } + return; + } + + boolean remap = false; + + // Do not check topology version if topology was locked on near node by + // external transaction or explicit lock. + if (!req.topologyLocked()) { + // Can not wait for topology future since it will break + // GridNearAtomicCheckUpdateRequest processing. + remap = !top.topologyVersionFuture().exchangeDone() || + needRemap(req.topologyVersion(), top.readyTopologyVersion()); + } - boolean remap = false; + if (!remap) { + DhtAtomicUpdateResult updRes = update(node, locked, req, res); - // Do not check topology version if topology was locked on near node by - // external transaction or explicit lock. - if (!req.topologyLocked()) { - // Can not wait for topology future since it will break - // GridNearAtomicCheckUpdateRequest processing. - remap = !top.topologyVersionFuture().exchangeDone() || - needRemap(req.topologyVersion(), top.readyTopologyVersion()); + dhtFut = updRes.dhtFuture(); + deleted = updRes.deleted(); + expiry = updRes.expiryPolicy(); + } + else + // Should remap all keys. + res.remapTopologyVersion(top.lastTopologyChangeVersion()); + } + finally { + top.readUnlock(); + } + + break; } + catch (UnregisteredClassException ex) { + IgniteCacheObjectProcessor cacheObjProc = ctx.cacheObjects(); - if (!remap) { - DhtAtomicUpdateResult updRes = update(node, locked, req, res); + assert cacheObjProc instanceof CacheObjectBinaryProcessorImpl; - dhtFut = updRes.dhtFuture(); - deleted = updRes.deleted(); - expiry = updRes.expiryPolicy(); + ((CacheObjectBinaryProcessorImpl)cacheObjProc).binaryContext().descriptorForClass(ex.cls(), false, false); } - else - // Should remap all keys. - res.remapTopologyVersion(top.lastTopologyChangeVersion()); - } - finally { - top.readUnlock(); } } catch (GridCacheEntryRemovedException e) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java index dad6728573cfc..defb3ccb2ad31 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java @@ -152,6 +152,17 @@ public KeyCacheObject toCacheKeyObject(CacheObjectContext ctx, */ @Nullable public CacheObject toCacheObject(CacheObjectContext ctx, @Nullable Object obj, boolean userObj); + /** + * @param ctx Cache context. + * @param obj Object. + * @param userObj If {@code true} then given object is object provided by user and should be copied + * before stored in cache. + * @param failIfUnregistered Throw exception if class isn't registered. + * @return Cache object. + */ + @Nullable public CacheObject toCacheObject(CacheObjectContext ctx, @Nullable Object obj, boolean userObj, + boolean failIfUnregistered); + /** * @param ctx Cache context. * @param type Object type. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java index 17be90f08cc1a..7f55614e2c593 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java @@ -225,6 +225,12 @@ protected KeyCacheObject toCacheKeyObject0(CacheObjectContext ctx, @Nullable @Override public CacheObject toCacheObject(CacheObjectContext ctx, @Nullable Object obj, boolean userObj) { + return toCacheObject(ctx, obj, userObj, false); + } + + /** {@inheritDoc} */ + @Nullable @Override public CacheObject toCacheObject(CacheObjectContext ctx, @Nullable Object obj, boolean userObj, + boolean failIfUnregistered) { if (obj == null || obj instanceof CacheObject) return (CacheObject)obj; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/StripedCompositeReadWriteLock.java b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedCompositeReadWriteLock.java index 18ef06cdd5401..42ec39772ac72 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/StripedCompositeReadWriteLock.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedCompositeReadWriteLock.java @@ -63,8 +63,10 @@ public StripedCompositeReadWriteLock(int concurrencyLvl) { writeLock = new WriteLock(); } - /** {@inheritDoc} */ - @NotNull @Override public Lock readLock() { + /** + * @return Index of current thread stripe. + */ + private int curIdx() { int idx; Thread curThread = Thread.currentThread(); @@ -83,7 +85,12 @@ public StripedCompositeReadWriteLock(int concurrencyLvl) { else idx = IDX.get(); - return locks[idx % locks.length].readLock(); + return idx % locks.length; + } + + /** {@inheritDoc} */ + @NotNull @Override public Lock readLock() { + return locks[curIdx()].readLock(); } /** {@inheritDoc} */ @@ -101,6 +108,18 @@ public boolean isWriteLockedByCurrentThread() { return locks[locks.length - 1].isWriteLockedByCurrentThread(); } + /** + * Queries the number of reentrant read holds on this lock by the + * current thread. A reader thread has a hold on a lock for + * each lock action that is not matched by an unlock action. + * + * @return the number of holds on the read lock by the current thread, + * or zero if the read lock is not held by the current thread + */ + public int getReadHoldCount() { + return locks[curIdx()].getReadHoldCount(); + } + /** * Read lock. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorCallTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorCallTest.java index cd4a78faae481..8a48ac25ffd4e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorCallTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryProcessorCallTest.java @@ -161,6 +161,11 @@ private void checkEntryProcessorCallCount(CacheConfiguration int key = 0; + // Call EntryProcessor on every node to ensure that binary metadata has been registered everywhere. + for (int i = 0; i < 1_000; i++) + ignite(i % SRV_CNT).cache(ccfg.getName()) + .invoke(key++, new TestEntryProcessor(OP_UPDATE), new TestValue(Integer.MIN_VALUE)); + checkEntryProcessCall(key++, clientCache1, null, null, expCallCnt); if (ccfg.getAtomicityMode() == TRANSACTIONAL) { From db64ac7e429595c573c3655d93749f2290fcf120 Mon Sep 17 00:00:00 2001 From: Andrey Kuznetsov Date: Thu, 28 Jun 2018 18:38:22 +0300 Subject: [PATCH 0296/1463] IGNITE-8860 Returned IgniteDiscoveryThread to RingMessageWorker. - Fixes #4248. Signed-off-by: Alexey Goncharuk (cherry-picked from commit#219bc81b730ea3ba078b61f96c9dab354496c22b) --- .../apache/ignite/spi/discovery/tcp/ServerImpl.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index 170c1badf3d54..bb76895bd204c 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -110,6 +110,7 @@ import org.apache.ignite.spi.IgniteSpiThread; import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; import org.apache.ignite.spi.discovery.DiscoverySpiListener; +import org.apache.ignite.spi.discovery.IgniteDiscoveryThread; import org.apache.ignite.spi.discovery.tcp.internal.DiscoveryDataPacket; import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNodesRing; @@ -340,7 +341,7 @@ class ServerImpl extends TcpDiscoveryImpl { msgWorker = new RingMessageWorker(log); - new MessageWorkerThread(msgWorker, log).start(); + new MessageWorkerDiscoveryThread(msgWorker, log).start(); if (tcpSrvr == null) tcpSrvr = new TcpServer(log); @@ -6821,6 +6822,14 @@ private MessageWorkerThreadWithCleanup(MessageWorker worker, IgniteLogger log } } + /** */ + private class MessageWorkerDiscoveryThread extends MessageWorkerThread implements IgniteDiscoveryThread { + /** {@inheritDoc} */ + private MessageWorkerDiscoveryThread(GridWorker worker, IgniteLogger log) { + super(worker, log); + } + } + /** * Slightly modified {@link IgniteSpiThread} intended to use with message workers. */ From 1e9e6a5e3a70a634c30cf89953831817cbfa98cf Mon Sep 17 00:00:00 2001 From: Dmitriy Govorukhin Date: Thu, 28 Jun 2018 17:39:38 +0300 Subject: [PATCH 0297/1463] IGNITE-8661 WALIterator should be stopped if it fails to deserialize a record - Fixes #4155. Signed-off-by: Alexey Goncharuk (cherry picked from commit d6ab2ae) --- .../wal/IgniteWriteAheadLogManager.java | 5 + .../GridCacheDatabaseSharedManager.java | 221 +++- .../wal/AbstractWalRecordsIterator.java | 27 +- .../wal/FileWriteAheadLogManager.java | 39 +- .../FsyncModeFileWriteAheadLogManager.java | 5 + .../SingleSegmentLogicalRecordsIterator.java | 4 +- .../wal/reader/IgniteWalIteratorFactory.java | 617 ++++++++-- .../reader/StandaloneWalRecordsIterator.java | 246 ++-- .../RecordSerializerFactoryImpl.java | 39 +- .../wal/serializer/RecordV1Serializer.java | 27 +- .../wal/serializer/RecordV2Serializer.java | 3 + ...iteWalIteratorExceptionDuringReadTest.java | 150 +++ .../db/wal/reader/IgniteWalReaderTest.java | 1050 +++++++++-------- .../persistence/pagemem/NoOpWALManager.java | 5 + .../testsuites/IgnitePdsTestSuite2.java | 3 + .../development/utils/IgniteWalConverter.java | 10 +- 16 files changed, 1589 insertions(+), 862 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorExceptionDuringReadTest.java 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 fd5d53b17e1d4..2b6358b6edaa7 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 @@ -121,6 +121,11 @@ public interface IgniteWriteAheadLogManager extends GridCacheSharedManager, Igni */ public int walArchiveSegments(); + /** + * @return Last archived segment index. + */ + public long lastArchivedSegment(); + /** * 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 a52922de8d63d..48b0f34d155e4 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 @@ -131,6 +131,7 @@ 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.wal.FileWALPointer; +import org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDataIntegrityViolationException; import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32; import org.apache.ignite.internal.processors.port.GridPortRecord; import org.apache.ignite.internal.util.GridMultiCollectionWrapper; @@ -166,6 +167,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD; 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.internal.processors.cache.persistence.metastorage.MetaStorage.METASTORAGE_CACHE_ID; /** @@ -1930,39 +1932,26 @@ private WALPointer readPointer(File cpMarkerFile, ByteBuffer buf) throws IgniteC cctx.wal().allowCompressionUntil(status.startPtr); long start = U.currentTimeMillis(); - int applied = 0; - WALPointer lastRead = null; + + long lastArchivedSegment = cctx.wal().lastArchivedSegment(); + + RestoreBinaryState restoreBinaryState = new RestoreBinaryState(status, lastArchivedSegment, log); Collection ignoreGrps = metastoreOnly ? Collections.emptySet() : F.concat(false, initiallyGlobalWalDisabledGrps, initiallyLocalWalDisabledGrps); + int applied = 0; + try (WALIterator it = cctx.wal().replay(status.endPtr)) { while (it.hasNextX()) { - IgniteBiTuple tup = it.nextX(); - - WALRecord rec = tup.get2(); + WALRecord rec = restoreBinaryState.next(it); - lastRead = tup.get1(); + if (rec == null) + break; switch (rec.type()) { - case CHECKPOINT_RECORD: - CheckpointRecord cpRec = (CheckpointRecord)rec; - - // We roll memory up until we find a checkpoint start record registered in the status. - if (F.eq(cpRec.checkpointId(), status.cpStartId)) { - log.info("Found last checkpoint marker [cpId=" + cpRec.checkpointId() + - ", pos=" + tup.get1() + ']'); - - apply = false; - } - else if (!F.eq(cpRec.checkpointId(), status.cpEndId)) - U.warn(log, "Found unexpected checkpoint marker, skipping [cpId=" + cpRec.checkpointId() + - ", expCpId=" + status.cpStartId + ", pos=" + tup.get1() + ']'); - - break; - case PAGE_RECORD: - if (apply) { + if (restoreBinaryState.needApplyBinaryUpdate()) { PageSnapshot pageRec = (PageSnapshot)rec; // Here we do not require tag check because we may be applying memory changes after @@ -2045,7 +2034,7 @@ else if (!F.eq(cpRec.checkpointId(), status.cpEndId)) break; default: - if (apply && rec instanceof PageDeltaRecord) { + if (restoreBinaryState.needApplyBinaryUpdate() && rec instanceof PageDeltaRecord) { PageDeltaRecord r = (PageDeltaRecord)rec; int grpId = r.groupId(); @@ -2086,11 +2075,13 @@ else if (!F.eq(cpRec.checkpointId(), status.cpEndId)) if (metastoreOnly) return null; + WALPointer lastReadPtr = restoreBinaryState.lastReadRecordPointer(); + if (status.needRestoreMemory()) { - if (apply) + if (restoreBinaryState.needApplyBinaryUpdate()) throw new StorageException("Failed to restore memory state (checkpoint marker is present " + "on disk, but checkpoint record is missed in WAL) " + - "[cpStatus=" + status + ", lastRead=" + lastRead + "]"); + "[cpStatus=" + status + ", lastRead=" + lastReadPtr + "]"); log.info("Finished applying memory changes [changesApplied=" + applied + ", time=" + (U.currentTimeMillis() - start) + "ms]"); @@ -2101,7 +2092,7 @@ else if (!F.eq(cpRec.checkpointId(), status.cpEndId)) cpHistory.initialize(retreiveHistory()); - return lastRead == null ? null : lastRead.next(); + return lastReadPtr == null ? null : lastReadPtr.next(); } /** @@ -2210,6 +2201,10 @@ private void applyLastUpdates(CheckpointStatus status, boolean metastoreOnly) th if (!metastoreOnly) cctx.kernalContext().query().skipFieldLookup(true); + long lastArchivedSegment = cctx.wal().lastArchivedSegment(); + + RestoreLogicalState restoreLogicalState = new RestoreLogicalState(lastArchivedSegment, log); + long start = U.currentTimeMillis(); int applied = 0; @@ -2220,9 +2215,10 @@ private void applyLastUpdates(CheckpointStatus status, boolean metastoreOnly) th Map, T2> partStates = new HashMap<>(); while (it.hasNextX()) { - IgniteBiTuple next = it.nextX(); + WALRecord rec = restoreLogicalState.next(it); - WALRecord rec = next.get2(); + if (rec == null) + break; switch (rec.type()) { case DATA_RECORD: @@ -4365,4 +4361,171 @@ else if (key.startsWith(WAL_GLOBAL_KEY_PREFIX)) else return null; } + + /** + * Abstract class for create restore context. + */ + public abstract static class RestoreStateContext { + /** */ + protected final IgniteLogger log; + + /** Last archived segment. */ + protected final long lastArchivedSegment; + + /** Last read record WAL pointer. */ + protected FileWALPointer lastRead; + + /** + * @param lastArchivedSegment Last archived segment index. + * @param log Ignite logger. + */ + public RestoreStateContext(long lastArchivedSegment, IgniteLogger log) { + this.lastArchivedSegment = lastArchivedSegment; + this.log = log; + } + + /** + * Advance iterator to the next record. + * + * @param it WAL iterator. + * @return WALRecord entry. + * @throws IgniteCheckedException If CRC check fail during binary recovery state or another exception occurring. + */ + public WALRecord next(WALIterator it) throws IgniteCheckedException { + try { + IgniteBiTuple tup = it.nextX(); + + WALRecord rec = tup.get2(); + + WALPointer ptr = tup.get1(); + + lastRead = (FileWALPointer)ptr; + + rec.position(ptr); + + return rec; + } + catch (IgniteCheckedException e) { + boolean throwsCRCError = throwsCRCError(); + + if (X.hasCause(e, IgniteDataIntegrityViolationException.class)) { + if (throwsCRCError) + throw e; + else + return null; + } + + log.error("Catch error during restore state, throwsCRCError=" + throwsCRCError, e); + + throw e; + } + } + + /** + * + * @return Last read WAL record pointer. + */ + public WALPointer lastReadRecordPointer() { + return lastRead; + } + + /** + * + * @return Flag indicates need throws CRC exception or not. + */ + public boolean throwsCRCError(){ + FileWALPointer lastReadPtr = lastRead; + + return lastReadPtr != null && lastReadPtr.index() <= lastArchivedSegment; + } + } + + /** + * Restore memory context. Tracks the safety of binary recovery. + */ + public static class RestoreBinaryState extends RestoreStateContext { + /** Checkpoint status. */ + private final CheckpointStatus status; + + /** The flag indicates need to apply the binary update or no needed. */ + private boolean needApplyBinaryUpdates; + + /** + * @param status Checkpoint status. + * @param lastArchivedSegment Last archived segment index. + * @param log Ignite logger. + */ + public RestoreBinaryState(CheckpointStatus status, long lastArchivedSegment, IgniteLogger log) { + super(lastArchivedSegment, log); + + this.status = status; + needApplyBinaryUpdates = status.needRestoreMemory(); + } + + /** + * Advance iterator to the next record. + * + * @param it WAL iterator. + * @return WALRecord entry. + * @throws IgniteCheckedException If CRC check fail during binary recovery state or another exception occurring. + */ + @Override public WALRecord next(WALIterator it) throws IgniteCheckedException { + WALRecord rec = super.next(it); + + if (rec == null) + return null; + + if (rec.type() == CHECKPOINT_RECORD) { + CheckpointRecord cpRec = (CheckpointRecord)rec; + + // We roll memory up until we find a checkpoint start record registered in the status. + if (F.eq(cpRec.checkpointId(), status.cpStartId)) { + log.info("Found last checkpoint marker [cpId=" + cpRec.checkpointId() + + ", pos=" + rec.position() + ']'); + + needApplyBinaryUpdates = false; + } + else if (!F.eq(cpRec.checkpointId(), status.cpEndId)) + U.warn(log, "Found unexpected checkpoint marker, skipping [cpId=" + cpRec.checkpointId() + + ", expCpId=" + status.cpStartId + ", pos=" + rec.position() + ']'); + } + + return rec; + } + + /** + * + * @return Flag indicates need apply binary record or not. + */ + public boolean needApplyBinaryUpdate() { + return needApplyBinaryUpdates; + } + + /** + * + * @return Flag indicates need throws CRC exception or not. + */ + @Override public boolean throwsCRCError() { + log.info("Throws CRC error check, needApplyBinaryUpdates=" + needApplyBinaryUpdates + + ", lastArchivedSegment=" + lastArchivedSegment + ", lastRead=" + lastRead); + + if (needApplyBinaryUpdates) + return true; + + return super.throwsCRCError(); + } + } + + /** + * Restore logical state context. Tracks the safety of logical recovery. + */ + public static class RestoreLogicalState extends RestoreStateContext { + /** + * @param lastArchivedSegment Last archived segment index. + * @param log Ignite logger. + */ + public RestoreLogicalState(long lastArchivedSegment, IgniteLogger log) { + super(lastArchivedSegment, log); + } + } } 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 e442386c03e87..01b093399fbc1 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 @@ -91,21 +91,21 @@ public abstract class AbstractWalRecordsIterator * @param sharedCtx Shared context. * @param serializerFactory Serializer of current version to read headers. * @param ioFactory ioFactory for file IO access. - * @param bufSize buffer for reading records size. + * @param initialReadBufferSize buffer for reading records size. */ protected AbstractWalRecordsIterator( @NotNull final IgniteLogger log, @NotNull final GridCacheSharedContext sharedCtx, @NotNull final RecordSerializerFactory serializerFactory, @NotNull final FileIOFactory ioFactory, - final int bufSize + final int initialReadBufferSize ) { this.log = log; this.sharedCtx = sharedCtx; this.serializerFactory = serializerFactory; this.ioFactory = ioFactory; - buf = new ByteBufferExpander(bufSize, ByteOrder.nativeOrder()); + buf = new ByteBufferExpander(initialReadBufferSize, ByteOrder.nativeOrder()); } /** {@inheritDoc} */ @@ -225,8 +225,12 @@ private IgniteBiTuple advanceRecord( if (e instanceof WalSegmentTailReachedException) throw (WalSegmentTailReachedException)e; - if (!(e instanceof SegmentEofException)) - handleRecordException(e, actualFilePtr); + if (!(e instanceof SegmentEofException) && !(e instanceof EOFException)) { + IgniteCheckedException e0 = handleRecordException(e, actualFilePtr); + + if (e0 != null) + throw e0; + } return null; } @@ -248,12 +252,15 @@ private 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. */ - protected void 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); + + return new IgniteCheckedException(e); } /** @@ -265,8 +272,8 @@ protected void handleRecordException( */ protected AbstractReadFileHandle initReadHandle( @NotNull final AbstractFileDescriptor desc, - @Nullable final FileWALPointer start) - throws IgniteCheckedException, FileNotFoundException { + @Nullable final FileWALPointer start + ) throws IgniteCheckedException, FileNotFoundException { try { FileIO fileIO = desc.isCompressed() ? new UnzipFileIO(desc.file()) : ioFactory.create(desc.file()); 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 3f406629f9e49..4a696e47a7abc 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 @@ -94,6 +94,7 @@ 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.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; @@ -127,6 +128,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_MMAP; import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_SERIALIZER_VERSION; import static org.apache.ignite.configuration.WALMode.LOG_ONLY; +import static org.apache.ignite.events.EventType.EVT_WAL_SEGMENT_ARCHIVED; 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.SWITCH_SEGMENT_RECORD; @@ -171,7 +173,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl private static final byte[] FILL_BUF = new byte[1024 * 1024]; /** Pattern for segment file names */ - private static final Pattern WAL_NAME_PATTERN = Pattern.compile("\\d{16}\\.wal"); + public static final Pattern WAL_NAME_PATTERN = Pattern.compile("\\d{16}\\.wal"); /** */ private static final Pattern WAL_TEMP_NAME_PATTERN = Pattern.compile("\\d{16}\\.wal\\.tmp"); @@ -191,7 +193,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl }; /** */ - private static final Pattern WAL_SEGMENT_FILE_COMPACTED_PATTERN = Pattern.compile("\\d{16}\\.wal\\.zip"); + public static final Pattern WAL_SEGMENT_FILE_COMPACTED_PATTERN = Pattern.compile("\\d{16}\\.wal\\.zip"); /** WAL segment file filter, see {@link #WAL_NAME_PATTERN} */ public static final FileFilter WAL_SEGMENT_COMPACTED_OR_RAW_FILE_FILTER = new FileFilter() { @@ -951,6 +953,11 @@ private boolean segmentReservedOrLocked(long absIdx) { return res >= 0 ? res : 0; } + /** {@inheritDoc} */ + @Override public long lastArchivedSegment() { + return archivedMonitor.lastArchivedAbsoluteIndex(); + } + /** {@inheritDoc} */ @Override public boolean reserved(WALPointer ptr) { FileWALPointer fPtr = (FileWALPointer)ptr; @@ -1656,9 +1663,12 @@ private synchronized boolean locked(long absIdx) { notifyAll(); } - if (evt.isRecordable(EventType.EVT_WAL_SEGMENT_ARCHIVED)) { - evt.record(new WalSegmentArchivedEvent(cctx.discovery().localNode(), - res.getAbsIdx(), res.getDstArchiveFile())); + if (evt.isRecordable(EVT_WAL_SEGMENT_ARCHIVED)) { + evt.record(new WalSegmentArchivedEvent( + cctx.discovery().localNode(), + res.getAbsIdx(), + res.getDstArchiveFile()) + ); } } } @@ -1912,7 +1922,7 @@ private void init() { FileDescriptor[] alreadyCompressed = scan(walArchiveDir.listFiles(WAL_SEGMENT_FILE_COMPACTED_FILTER)); if (alreadyCompressed.length > 0) - lastCompressedIdx = alreadyCompressed[alreadyCompressed.length - 1].getIdx(); + lastCompressedIdx = alreadyCompressed[alreadyCompressed.length - 1].idx(); } /** @@ -2320,7 +2330,8 @@ else if (create) /** * WAL file descriptor. */ - public static class FileDescriptor implements Comparable, AbstractWalRecordsIterator.AbstractFileDescriptor { + public static class FileDescriptor implements + Comparable, AbstractFileDescriptor { /** */ protected final File file; @@ -2390,20 +2401,6 @@ public static String fileName(long segment) { 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(); - } - /** * @return True if segment is ZIP compressed. */ 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 6e59ad3614dec..5db21d25ae4a3 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 @@ -845,6 +845,11 @@ private boolean hasIndex(long absIdx) { return res >= 0 ? res : 0; } + /** {@inheritDoc} */ + @Override public long lastArchivedSegment() { + return archiver.lastArchivedAbsoluteIndex(); + } + /** {@inheritDoc} */ @Override public boolean reserved(WALPointer ptr) { FileWALPointer fPtr = (FileWALPointer)ptr; 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 36e5b0e21b3a9..f688bb4b5f783 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 @@ -86,9 +86,7 @@ public class SingleSegmentLogicalRecordsIterator extends AbstractWalRecordsItera private static RecordSerializerFactory initLogicalRecordsSerializerFactory(GridCacheSharedContext sharedCtx) throws IgniteCheckedException { - return new RecordSerializerFactoryImpl(sharedCtx) - .recordDeserializeFilter(new LogicalRecordsFilter()) - .marshalledMode(true); + return new RecordSerializerFactoryImpl(sharedCtx, new LogicalRecordsFilter()).marshalledMode(true); } /** {@inheritDoc} */ 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 0c7bbb323656f..2bfc22d79ac54 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 @@ -17,17 +17,49 @@ package org.apache.ignite.internal.processors.cache.persistence.wal.reader; +import java.io.DataInput; import java.io.File; +import java.io.IOException; +import java.io.PrintStream; +import java.nio.ByteOrder; +import java.nio.file.FileVisitResult; +import java.nio.file.Path; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.TreeSet; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.pagemem.wal.WALIterator; +import org.apache.ignite.internal.pagemem.wal.WALPointer; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +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.UnzipFileIO; +import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferExpander; +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; +import org.apache.ignite.lang.IgniteBiPredicate; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import static java.lang.System.arraycopy; +import static java.nio.file.Files.walkFileTree; +import static org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager.WAL_NAME_PATTERN; +import static org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager.WAL_SEGMENT_FILE_COMPACTED_PATTERN; +import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.HEADER_RECORD_SIZE; +import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.readPosition; + /** * Factory for creating iterator over WAL files */ @@ -35,189 +67,522 @@ public class IgniteWalIteratorFactory { /** Logger. */ private final IgniteLogger log; - /** Page size, in standalone iterator mode this value can't be taken from memory configuration. */ - private final int pageSize; - - /** - * Folder specifying location of metadata File Store. {@code null} means no specific folder is configured.
    - * This folder should be specified for converting data entries into BinaryObjects - */ - @Nullable private File binaryMetadataFileStoreDir; - /** - * Folder specifying location of marshaller mapping file store. {@code null} means no specific folder is configured. - *
    This folder should be specified for converting data entries into BinaryObjects. Providing {@code null} will - * disable unmarshall for non primitive objects, BinaryObjects will be provided + * Creates WAL files iterator factory. + * WAL iterator supports automatic converting from CacheObjects and KeyCacheObject into BinaryObjects */ - @Nullable private File marshallerMappingFileStoreDir; - - /** Keep binary. This flag disables converting of non primitive types (BinaryObjects) */ - private boolean keepBinary; - - /** Factory to provide I/O interfaces for read/write operations with files */ - private FileIOFactory ioFactory; - - /** Wal records iterator buffer size */ - private int bufSize = StandaloneWalRecordsIterator.DFLT_BUF_SIZE; + public IgniteWalIteratorFactory() { + this(ConsoleLogger.INSTANCE); + } /** * Creates WAL files iterator factory. * WAL iterator supports automatic converting from CacheObjects and KeyCacheObject into BinaryObjects * * @param log Logger. - * @param pageSize Page size which was used in Ignite Persistent Data store to read WAL from, size is validated - * according its boundaries. - * @param binaryMetadataFileStoreDir folder specifying location of metadata File Store. Should include "binary_meta" - * subfolder and consistent ID subfolder. Note Consistent ID should be already masked and should not contain special - * symbols. Providing {@code null} means no specific folder is configured.
    - * @param marshallerMappingFileStoreDir Folder specifying location of marshaller mapping file store. Should include - * "marshaller" subfolder. Providing {@code null} will disable unmarshall for non primitive objects, BinaryObjects - * will be provided - * @param keepBinary {@code true} disables complex object unmarshall into source classes */ - public IgniteWalIteratorFactory( - @NotNull final IgniteLogger log, - final int pageSize, - @Nullable final File binaryMetadataFileStoreDir, - @Nullable final File marshallerMappingFileStoreDir, - final boolean keepBinary) { + public IgniteWalIteratorFactory(@NotNull final IgniteLogger log) { this.log = log; - this.pageSize = pageSize; - this.binaryMetadataFileStoreDir = binaryMetadataFileStoreDir; - this.marshallerMappingFileStoreDir = marshallerMappingFileStoreDir; - this.keepBinary = keepBinary; - this.ioFactory = new DataStorageConfiguration().getFileIOFactory(); - new DataStorageConfiguration().setPageSize(pageSize); // just for validate } /** - * Creates WAL files iterator factory. - * WAL iterator supports automatic converting from CacheObjects and KeyCacheObject into BinaryObjects + * Creates iterator for file by file scan mode. + * This method may be used for work folder, file indexes are scanned from the file context. + * In this mode only provided WAL segments will be scanned. New WAL files created during iteration will be ignored. * - * @param log Logger. - * @param pageSize Page size which was used in Ignite Persistent Data store to read WAL from, size is validated - * according its boundaries. - * @param binaryMetadataFileStoreDir folder specifying location of metadata File Store. Should include "binary_meta" - * subfolder and consistent ID subfolder. Note Consistent ID should be already masked and should not contain special - * symbols. Providing {@code null} means no specific folder is configured.
    - * @param marshallerMappingFileStoreDir Folder specifying location of marshaller mapping file store. Should include - * "marshaller" subfolder. Providing {@code null} will disable unmarshall for non primitive objects, BinaryObjects - * will be provided + * @param filesOrDirs files to scan. A file can be the path to '.wal' file, or directory with '.wal' files. + * Order is not important, but it is significant to provide all segments without omissions. + * Path should not contain special symbols. Special symbols should be already masked. + * @return closable WAL records iterator, should be closed when non needed. + * @throws IgniteCheckedException if failed to read files + * @throws IllegalArgumentException If parameter validation failed. */ - public IgniteWalIteratorFactory( - @NotNull final IgniteLogger log, - final int pageSize, - @Nullable final File binaryMetadataFileStoreDir, - @Nullable final File marshallerMappingFileStoreDir) { - this(log, pageSize, binaryMetadataFileStoreDir, marshallerMappingFileStoreDir, false); + public WALIterator iterator( + @NotNull File... filesOrDirs + ) throws IgniteCheckedException, IllegalArgumentException { + return iterator(new IteratorParametersBuilder().filesOrDirs(filesOrDirs)); } /** - * Creates WAL files iterator factory. This constructor does not allow WAL iterators access to data entries key and value. + * Creates iterator for file by file scan mode. + * This method may be used for work folder, file indexes are scanned from the file context. + * In this mode only provided WAL segments will be scanned. New WAL files created during iteration will be ignored. * - * @param log Logger. - * @param ioFactory Custom factory for non-standard file API to be used in WAL reading. - * @param pageSize Page size which was used in Ignite Persistent Data store to read WAL from, size is validated - * according its boundaries. + * @param filesOrDirs paths to scan. A path can be direct to '.wal' file, or directory with '.wal' files. + * Order is not important, but it is significant to provide all segments without omissions. + * Path should not contain special symbols. Special symbols should be already masked. + * @return closable WAL records iterator, should be closed when non needed. + * @throws IgniteCheckedException If failed to read files. + * @throws IllegalArgumentException If parameter validation failed. */ - public IgniteWalIteratorFactory(@NotNull final IgniteLogger log, @NotNull final FileIOFactory ioFactory, int pageSize) { - this.log = log; - this.pageSize = pageSize; - this.ioFactory = ioFactory; - new DataStorageConfiguration().setPageSize(pageSize); // just for validate + public WALIterator iterator( + @NotNull String... filesOrDirs + ) throws IgniteCheckedException, IllegalArgumentException { + return iterator(new IteratorParametersBuilder().filesOrDirs(filesOrDirs)); } /** - * Creates WAL files iterator factory. This constructor does not allow WAL iterators access to data entries key and - * value. - * - * @param log Logger. - * @param pageSize Page size which was used in Ignite Persistent Data store to read WAL from, size is validated - * according its boundaries. + * @param iteratorParametersBuilder Iterator parameters builder. + * @return closable WAL records iterator, should be closed when non needed */ - public IgniteWalIteratorFactory(@NotNull final IgniteLogger log, int pageSize) { - this(log, new DataStorageConfiguration().getFileIOFactory(), pageSize); + public WALIterator iterator( + @NotNull IteratorParametersBuilder iteratorParametersBuilder + ) throws IgniteCheckedException, IllegalArgumentException { + iteratorParametersBuilder.validate(); + + return new StandaloneWalRecordsIterator(log, + prepareSharedCtx(iteratorParametersBuilder), + iteratorParametersBuilder.ioFactory, + resolveWalFiles( + iteratorParametersBuilder.filesOrDirs, + iteratorParametersBuilder + ), + iteratorParametersBuilder.filter, + iteratorParametersBuilder.keepBinary, + iteratorParametersBuilder.bufferSize + ); } /** - * Creates iterator for (archive) directory scan mode. - * Note in this mode total scanned files at end of iteration may be wider that initial files in directory. - * This mode does not support work directory scan because work directory contains unpredictable number in file name. - * Such file may broke iteration. + * Find WAL gaps, for example: + * 0 1 2 3 4 7 8 10 - WAL segment files in directory, this method will return + * List with two tuples [(4,7),(8,10)]. * - * @param walDirWithConsistentId directory with WAL files. Should already contain node consistent ID as subfolder. - * Note: 'Consistent ID'-based subfolder name (if any) should not contain special symbols. - * @return closable WAL records iterator, should be closed when non needed - * @throws IgniteCheckedException if failed to read folder + * @param filesOrDirs Paths to files or directories for scan. + * @return List of tuples, low and high index segments with gap. */ - public WALIterator iteratorArchiveDirectory( - @NotNull final File walDirWithConsistentId) throws IgniteCheckedException { - return new StandaloneWalRecordsIterator( - walDirWithConsistentId, log, prepareSharedCtx(), ioFactory, keepBinary, bufSize); + public List> hasGaps( + @NotNull String... filesOrDirs + ) throws IllegalArgumentException { + return hasGaps(new IteratorParametersBuilder().filesOrDirs(filesOrDirs)); } /** - * Creates iterator for file by file scan mode. - * This method may be used only for archive folder (not for work). - * In this mode only provided WAL segments will be scanned. New WAL files created during iteration will be ignored + * Find WAL gaps, for example: + * 0 1 2 3 4 7 8 10 - WAL segment files in directory, this method will return + * List with two tuples [(4,7),(8,10)]. * - * @param files files to scan. Order is not important, but it is significant to provide all segments without omissions. - * Parameter should contain direct file links to '.wal' files from archive directory. - * 'Consistent ID'-based subfolder name (if any) should not contain special symbols. - * Special symbols should be already masked. - * - * @return closable WAL records iterator, should be closed when non needed - * @throws IgniteCheckedException if failed to read files + * @param filesOrDirs Files or directories to scan. + * @return List of tuples, low and high index segments with gap. */ - public WALIterator iteratorArchiveFiles(@NotNull final File... files) throws IgniteCheckedException { - return new StandaloneWalRecordsIterator(log, prepareSharedCtx(), ioFactory, false, keepBinary, bufSize, files); + public List> hasGaps( + @NotNull File... filesOrDirs + ) throws IllegalArgumentException { + return hasGaps(new IteratorParametersBuilder().filesOrDirs(filesOrDirs)); } /** - * Creates iterator for file by file scan mode. - * This method may be used for work folder, file indexes are scanned from the file context. - * In this mode only provided WAL segments will be scanned. New WAL files created during iteration will be ignored. - * - * @param files files to scan. Order is not important, but it is significant to provide all segments without omissions. - * Parameter should contain direct file links to '.wal' files from work directory. - * 'Consistent ID'-based subfolder name (if any) should not contain special symbols. - * Special symbols should be already masked. + * @param iteratorParametersBuilder Iterator parameters builder. + * @return List of tuples, low and high index segments with gap. + */ + public List> hasGaps( + @NotNull IteratorParametersBuilder iteratorParametersBuilder + ) throws IllegalArgumentException { + iteratorParametersBuilder.validate(); + + List> gaps = new ArrayList<>(); + + List descriptors = resolveWalFiles( + iteratorParametersBuilder.filesOrDirs, + iteratorParametersBuilder + ); + + Iterator it = descriptors.iterator(); + + FileDescriptor prevFd = null; + + while (it.hasNext()) { + FileDescriptor nextFd = it.next(); + + if (prevFd == null) { + prevFd = nextFd; + + continue; + } + + if (prevFd.idx() + 1 != nextFd.idx()) + gaps.add(new T2<>(prevFd.idx(), nextFd.idx())); + + prevFd = nextFd; + } + + return gaps; + } + + /** + * This methods checks all provided files to be correct WAL segment. + * Header record and its position is checked. WAL position is used to determine real index. + * File index from file name is ignored. * - * @return closable WAL records iterator, should be closed when non needed - * @throws IgniteCheckedException if failed to read files + * @param iteratorParametersBuilder IteratorParametersBuilder. + * @return list of file descriptors with checked header records, having correct file index is set + */ + private List resolveWalFiles( + File[] filesOrDirs, + IteratorParametersBuilder iteratorParametersBuilder + ) { + if (filesOrDirs == null || filesOrDirs.length == 0) + return Collections.emptyList(); + + final FileIOFactory ioFactory = iteratorParametersBuilder.ioFactory; + + final TreeSet descriptors = new TreeSet<>(); + + for (File file : filesOrDirs) { + if (file.isDirectory()) { + try { + walkFileTree(file.toPath(), new SimpleFileVisitor() { + @Override + public FileVisitResult visitFile(Path path, BasicFileAttributes attrs) { + addFileDescriptor(path.toFile(), ioFactory, descriptors); + + return FileVisitResult.CONTINUE; + } + }); + } + catch (IOException e) { + U.error(log, "Failed to walk directories from root [" + file + "]. Skipping this directory.", e); + } + + continue; + } + + addFileDescriptor(file, ioFactory, descriptors); + } + + return new ArrayList<>(descriptors); + } + + /** + * @param file File. + * @param ioFactory IO factory. + * @param descriptors List of descriptors. + */ + private void addFileDescriptor(File file, FileIOFactory ioFactory, TreeSet descriptors) { + if (file.length() < HEADER_RECORD_SIZE) + return; // Filter out this segment as it is too short. + + String fileName = file.getName(); + + if (!WAL_NAME_PATTERN.matcher(fileName).matches() && + !WAL_SEGMENT_FILE_COMPACTED_PATTERN.matcher(fileName).matches()) + return; // Filter out this because it is not segment file. + + FileDescriptor desc = readFileDescriptor(file, ioFactory); + + if (desc != null) + descriptors.add(desc); + } + + /** + * @param file File to read. + * @param ioFactory IO factory. */ - public WALIterator iteratorWorkFiles(@NotNull final File... files) throws IgniteCheckedException { - return new StandaloneWalRecordsIterator(log, prepareSharedCtx(), ioFactory, true, keepBinary, bufSize, files); + private FileDescriptor readFileDescriptor(File file, FileIOFactory ioFactory) { + FileDescriptor ds = new FileDescriptor(file); + + try ( + FileIO fileIO = ds.isCompressed() ? new UnzipFileIO(file) : ioFactory.create(file); + ByteBufferExpander buf = new ByteBufferExpander(HEADER_RECORD_SIZE, ByteOrder.nativeOrder()) + ) { + final DataInput in = new FileInput(fileIO, buf); + + // Header record must be agnostic to the serializer version. + final int type = in.readUnsignedByte(); + + if (type == RecordType.STOP_ITERATION_RECORD_TYPE) { + if (log.isInfoEnabled()) + log.info("Reached logical end of the segment for file " + file); + + return null; + } + + FileWALPointer ptr = readPosition(in); + + return new FileDescriptor(file, ptr.index()); + } + catch (IOException e) { + U.warn(log, "Failed to scan index from file [" + file + "]. Skipping this file during iteration", e); + + return null; + } } /** - * @return fake shared context required for create minimal services for record reading + * @return Fake shared context required for create minimal services for record reading. */ - @NotNull private GridCacheSharedContext prepareSharedCtx() throws IgniteCheckedException { - final GridKernalContext kernalCtx = new StandaloneGridKernalContext(log, binaryMetadataFileStoreDir, marshallerMappingFileStoreDir); + @NotNull private GridCacheSharedContext prepareSharedCtx( + IteratorParametersBuilder iteratorParametersBuilder + ) throws IgniteCheckedException { + GridKernalContext kernalCtx = new StandaloneGridKernalContext(log, + iteratorParametersBuilder.binaryMetadataFileStoreDir, + iteratorParametersBuilder.marshallerMappingFileStoreDir + ); - final StandaloneIgniteCacheDatabaseSharedManager dbMgr = new StandaloneIgniteCacheDatabaseSharedManager(); + StandaloneIgniteCacheDatabaseSharedManager dbMgr = new StandaloneIgniteCacheDatabaseSharedManager(); - dbMgr.setPageSize(pageSize); + dbMgr.setPageSize(iteratorParametersBuilder.pageSize); return new GridCacheSharedContext<>( kernalCtx, null, null, null, null, null, null, dbMgr, null, null, null, null, null, - null, null, null); + null, null, null + ); } /** - * @param ioFactory New factory to provide I/O interfaces for read/write operations with files + * Wal iterator parameter builder. */ - public void ioFactory(FileIOFactory ioFactory) { - this.ioFactory = ioFactory; + public static class IteratorParametersBuilder { + /** */ + private File[] filesOrDirs; + + /** */ + private int pageSize = DataStorageConfiguration.DFLT_PAGE_SIZE; + + /** Wal records iterator buffer size. */ + private int bufferSize = StandaloneWalRecordsIterator.DFLT_BUF_SIZE; + + /** Keep binary. This flag disables converting of non primitive types (BinaryObjects). */ + private boolean keepBinary; + + /** Factory to provide I/O interfaces for read/write operations with files. */ + private FileIOFactory ioFactory = new DataStorageConfiguration().getFileIOFactory(); + + /** + * Folder specifying location of metadata File Store. {@code null} means no specific folder is configured.
    + * This folder should be specified for converting data entries into BinaryObjects + */ + @Nullable private File binaryMetadataFileStoreDir; + + /** + * Folder specifying location of marshaller mapping file store. {@code null} means no specific folder is configured. + *
    This folder should be specified for converting data entries into BinaryObjects. Providing {@code null} will + * disable unmarshall for non primitive objects, BinaryObjects will be provided + */ + @Nullable private File marshallerMappingFileStoreDir; + + /** */ + @Nullable private IgniteBiPredicate filter; + + /** + * @param filesOrDirs Paths to files or directories. + * @return IteratorParametersBuilder Self reference. + */ + public IteratorParametersBuilder filesOrDirs(String... filesOrDirs) { + File[] filesOrDirs0 = new File[filesOrDirs.length]; + + for (int i = 0; i < filesOrDirs.length; i++) { + filesOrDirs0[i] = new File(filesOrDirs[i]); + } + + return filesOrDirs(filesOrDirs0); + } + + /** + * @param filesOrDirs Files or directories. + * @return IteratorParametersBuilder Self reference. + */ + public IteratorParametersBuilder filesOrDirs(File... filesOrDirs) { + if (this.filesOrDirs == null) + this.filesOrDirs = filesOrDirs; + else + this.filesOrDirs = merge(this.filesOrDirs, filesOrDirs); + + return this; + } + + /** + * @param pageSize Page size. + * @return IteratorParametersBuilder Self reference. + */ + public IteratorParametersBuilder pageSize(int pageSize) { + this.pageSize = pageSize; + + return this; + } + + /** + * @param bufferSize Initial size of buffer for reading segments. + * @return IteratorParametersBuilder Self reference. + */ + public IteratorParametersBuilder bufferSize(int bufferSize) { + this.bufferSize = bufferSize; + + return this; + } + + /** + * @return IteratorParametersBuilder Self reference. + */ + public IteratorParametersBuilder keepBinary(boolean keepBinary) { + this.keepBinary = keepBinary; + + return this; + } + + /** + * @param ioFactory Custom IO factory for reading files. + * @return IteratorParametersBuilder Self reference. + */ + public IteratorParametersBuilder ioFactory(FileIOFactory ioFactory) { + this.ioFactory = ioFactory; + + return this; + } + + /** + * @param binaryMetadataFileStoreDir Path to the binary metadata. + * @return IteratorParametersBuilder Self reference. + */ + public IteratorParametersBuilder binaryMetadataFileStoreDir(File binaryMetadataFileStoreDir) { + this.binaryMetadataFileStoreDir = binaryMetadataFileStoreDir; + + return this; + } + + /** + * @param marshallerMappingFileStoreDir Path to the marshaller mapping. + * @return IteratorParametersBuilder Self reference. + */ + public IteratorParametersBuilder marshallerMappingFileStoreDir(File marshallerMappingFileStoreDir) { + this.marshallerMappingFileStoreDir = marshallerMappingFileStoreDir; + + return this; + } + + /** + * @param filter Record filter for skip records during iteration. + * @return IteratorParametersBuilder Self reference. + */ + public IteratorParametersBuilder filter(IgniteBiPredicate filter) { + this.filter = filter; + + return this; + } + + /** + * Copy current state of builder to new instance. + * + * @return IteratorParametersBuilder Self reference. + */ + public IteratorParametersBuilder copy() { + return new IteratorParametersBuilder() + .filesOrDirs(filesOrDirs) + .pageSize(pageSize) + .bufferSize(bufferSize) + .keepBinary(keepBinary) + .ioFactory(ioFactory) + .binaryMetadataFileStoreDir(binaryMetadataFileStoreDir) + .marshallerMappingFileStoreDir(marshallerMappingFileStoreDir) + .filter(filter); + } + + /** + * @throws IllegalArgumentException If validation failed. + */ + public void validate() throws IllegalArgumentException { + A.ensure(pageSize >= 1024 && pageSize <= 16 * 1024, "Page size must be between 1kB and 16kB."); + A.ensure(U.isPow2(pageSize), "Page size must be a power of 2."); + + A.ensure(bufferSize >= pageSize * 2, "Buffer to small."); + } + + /** + * Merge file arrays. + * + * @param f1 Files array one. + * @param f2 Files array two. + * @return Merged arrays from one and two arrays. + */ + private File[] merge(File[] f1, File[] f2) { + File[] merged = new File[f1.length + f2.length]; + + arraycopy(f1, 0, merged, 0, f1.length); + arraycopy(f2, 0, merged, f1.length, f2.length); + + return merged; + } } /** - * @param bufSize New wal records iterator buffer size + * */ - public void bufferSize(int bufSize) { - this.bufSize = bufSize; + private static class ConsoleLogger implements IgniteLogger { + /** */ + private static final ConsoleLogger INSTANCE = new ConsoleLogger(); + + /** */ + private static final PrintStream OUT = System.out; + + /** */ + private static final PrintStream ERR = System.err; + + /** */ + private ConsoleLogger() { + + } + + /** {@inheritDoc} */ + @Override public IgniteLogger getLogger(Object ctgr) { + return this; + } + + /** {@inheritDoc} */ + @Override public void trace(String msg) { + + } + + /** {@inheritDoc} */ + @Override public void debug(String msg) { + + } + + /** {@inheritDoc} */ + @Override public void info(String msg) { + OUT.println(msg); + } + + /** {@inheritDoc} */ + @Override public void warning(String msg, @Nullable Throwable e) { + OUT.println(msg); + + if (e != null) + e.printStackTrace(OUT); + } + + /** {@inheritDoc} */ + @Override public void error(String msg, @Nullable Throwable e) { + ERR.println(msg); + + if (e != null) + e.printStackTrace(ERR); + } + + /** {@inheritDoc} */ + @Override public boolean isTraceEnabled() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean isDebugEnabled() { + return false; + } + + /** {@inheritDoc} */ + @Override public boolean isInfoEnabled() { + return true; + } + + /** {@inheritDoc} */ + @Override public boolean isQuiet() { + return false; + } + + /** {@inheritDoc} */ + @Override public String fileName() { + return "SYSTEM.OUT"; + } } } 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 712517bbbf90c..9df446836198d 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 @@ -17,44 +17,41 @@ package org.apache.ignite.internal.processors.cache.persistence.wal.reader; -import java.io.DataInput; import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; -import java.nio.ByteOrder; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.List; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.pagemem.wal.WALPointer; import org.apache.ignite.internal.pagemem.wal.record.DataEntry; import org.apache.ignite.internal.pagemem.wal.record.DataRecord; import org.apache.ignite.internal.pagemem.wal.record.LazyDataEntry; import org.apache.ignite.internal.pagemem.wal.record.UnwrapDataEntry; import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType; import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.CacheObjectContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.KeyCacheObject; 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.UnzipFileIO; import org.apache.ignite.internal.processors.cache.persistence.wal.AbstractWalRecordsIterator; -import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferExpander; 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; +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.serializer.RecordSerializer; import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactoryImpl; -import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer; import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor; -import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiPredicate; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.HEADER_RECORD_SIZE; +import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.readSegmentHeader; /** * WAL reader iterator, for creation in standalone WAL reader tool @@ -66,78 +63,52 @@ class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator { /** */ private static final long serialVersionUID = 0L; - - /** - * WAL files directory. Should already contain 'consistent ID' as subfolder. - * null value means file-by-file iteration mode - */ - @Nullable - private File walFilesDir; - /** * File descriptors remained to scan. * null value means directory scan mode */ @Nullable - private List walFileDescriptors; + private final List walFileDescriptors; + + /** */ + private int curIdx = -1; /** Keep binary. This flag disables converting of non primitive types (BinaryObjects) */ private boolean keepBinary; - /** - * Creates iterator in directory scan mode - * @param walFilesDir Wal files directory. Should already contain node consistent ID as subfolder - * @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 - * @param bufSize Buffer size. - */ - StandaloneWalRecordsIterator( - @NotNull File walFilesDir, - @NotNull IgniteLogger log, - @NotNull GridCacheSharedContext sharedCtx, - @NotNull FileIOFactory ioFactory, - boolean keepBinary, - int bufSize - ) throws IgniteCheckedException { - super(log, - sharedCtx, - new RecordSerializerFactoryImpl(sharedCtx), - ioFactory, - bufSize); - this.keepBinary = keepBinary; - init(walFilesDir, false, null); - advance(); - } - /** * 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 workDir Work directory is scanned, false - archive * @param keepBinary Keep binary. This flag disables converting of non primitive types * (BinaryObjects will be used instead) * @param walFiles Wal files. */ StandaloneWalRecordsIterator( - @NotNull IgniteLogger log, - @NotNull GridCacheSharedContext sharedCtx, - @NotNull FileIOFactory ioFactory, - boolean workDir, - boolean keepBinary, - int bufSize, - @NotNull File... walFiles) throws IgniteCheckedException { - super(log, + @NotNull IgniteLogger log, + @NotNull GridCacheSharedContext sharedCtx, + @NotNull FileIOFactory ioFactory, + @NotNull List walFiles, + IgniteBiPredicate readTypeFilter, + boolean keepBinary, + int initialReadBufferSize + ) throws IgniteCheckedException { + super( + log, sharedCtx, - new RecordSerializerFactoryImpl(sharedCtx), + new RecordSerializerFactoryImpl(sharedCtx, readTypeFilter), ioFactory, - bufSize); + initialReadBufferSize + ); this.keepBinary = keepBinary; - init(null, workDir, walFiles); + + walFileDescriptors = walFiles; + + init(walFiles); + advance(); } @@ -145,119 +116,42 @@ 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. * - * @param walFilesDir directory for directory scan mode - * @param workDir work directory, only for file-by-file mode * @param walFiles files for file-by-file iteration mode */ - private void init( - @Nullable final File walFilesDir, - final boolean workDir, - @Nullable final File[] walFiles) throws IgniteCheckedException { - if (walFilesDir != null) { - FileWriteAheadLogManager.FileDescriptor[] descs = FileWriteAheadLogManager.loadFileDescriptors(walFilesDir); - curWalSegmIdx = !F.isEmpty(descs) ? descs[0].getIdx() : 0; - this.walFilesDir = walFilesDir; - } - else { + private void init(List walFiles) { + if (walFiles == null || walFiles.isEmpty()) + return; - if (workDir) - walFileDescriptors = scanIndexesFromFileHeaders(walFiles); - else - walFileDescriptors = new ArrayList<>(Arrays.asList(FileWriteAheadLogManager.scan(walFiles))); - - curWalSegmIdx = !walFileDescriptors.isEmpty() ? walFileDescriptors.get(0).getIdx() : 0; - } - curWalSegmIdx--; + curWalSegmIdx = walFiles.get(curIdx + 1).idx() - 1; if (log.isDebugEnabled()) log.debug("Initialized WAL cursor [curWalSegmIdx=" + curWalSegmIdx + ']'); } - /** - * This methods checks all provided files to be correct WAL segment. - * Header record and its position is checked. WAL position is used to determine real index. - * File index from file name is ignored. - * - * @param allFiles files to scan. - * @return list of file descriptors with checked header records, having correct file index is set - */ - private List scanIndexesFromFileHeaders( - @Nullable final File[] allFiles) { - if (allFiles == null || allFiles.length == 0) - return Collections.emptyList(); - - final List resultingDescs = new ArrayList<>(); - - for (File file : allFiles) { - if (file.length() < HEADER_RECORD_SIZE) - continue; //filter out this segment as it is too short - - FileWALPointer ptr; - - try ( - FileIO fileIO = ioFactory.create(file); - ByteBufferExpander buf = new ByteBufferExpander(HEADER_RECORD_SIZE, ByteOrder.nativeOrder()) - ) { - final DataInput in = new FileInput(fileIO, buf); - - // Header record must be agnostic to the serializer version. - final int type = in.readUnsignedByte(); - - if (type == WALRecord.RecordType.STOP_ITERATION_RECORD_TYPE) { - if (log.isInfoEnabled()) - log.info("Reached logical end of the segment for file " + file); - - continue; //filter out this segment - } - ptr = RecordV1Serializer.readPosition(in); - } - catch (IOException e) { - U.warn(log, "Failed to scan index from file [" + file + "]. Skipping this file during iteration", e); - - continue; //filter out this segment - } - - resultingDescs.add(new FileWriteAheadLogManager.FileDescriptor(file, ptr.index())); - } - Collections.sort(resultingDescs); - - return resultingDescs; - } - /** {@inheritDoc} */ @Override protected AbstractReadFileHandle advanceSegment( - @Nullable final AbstractReadFileHandle curWalSegment) throws IgniteCheckedException { + @Nullable final AbstractReadFileHandle curWalSegment + ) throws IgniteCheckedException { if (curWalSegment != null) curWalSegment.close(); curWalSegmIdx++; - // curHandle.workDir is false - final FileWriteAheadLogManager.FileDescriptor fd; - - if (walFilesDir != null) { - File segmentFile = new File(walFilesDir, - FileWriteAheadLogManager.FileDescriptor.fileName(curWalSegmIdx)); - if (!segmentFile.exists()) - segmentFile = new File(walFilesDir, - FileWriteAheadLogManager.FileDescriptor.fileName(curWalSegmIdx) + ".zip"); + curIdx++; - fd = new FileWriteAheadLogManager.FileDescriptor(segmentFile); - } - else { - if (walFileDescriptors.isEmpty()) - return null; //no files to read, stop iteration + if (curIdx >= walFileDescriptors.size()) + return null; - fd = walFileDescriptors.remove(0); - } + FileDescriptor fd = walFileDescriptors.get(curIdx); if (log.isDebugEnabled()) - log.debug("Reading next file [absIdx=" + curWalSegmIdx + ", file=" + fd.getAbsolutePath() + ']'); + log.debug("Reading next file [absIdx=" + curWalSegmIdx + ", file=" + fd.file().getAbsolutePath() + ']'); assert fd != null; curRec = null; + try { return initReadHandle(fd, null); } @@ -269,12 +163,43 @@ private List scanIndexesFromFileHeaders } } + /** {@inheritDoc} */ + @Override protected AbstractReadFileHandle initReadHandle( + @NotNull AbstractFileDescriptor desc, + @Nullable FileWALPointer start + ) throws IgniteCheckedException, FileNotFoundException { + + AbstractFileDescriptor fd = desc; + + while (true) { + try { + FileIO fileIO = fd.isCompressed() ? new UnzipFileIO(fd.file()) : ioFactory.create(fd.file()); + + readSegmentHeader(fileIO, curWalSegmIdx); + + break; + } + catch (IOException | IgniteCheckedException e) { + log.error("Failed to init segment curWalSegmIdx=" + curWalSegmIdx + ", curIdx=" + curIdx, e); + + curIdx++; + + if (curIdx >= walFileDescriptors.size()) + return null; + + fd = walFileDescriptors.get(curIdx); + } + } + + return super.initReadHandle(fd, start); + } + /** {@inheritDoc} */ @NotNull @Override protected WALRecord postProcessRecord(@NotNull final WALRecord rec) { - final GridKernalContext kernalCtx = sharedCtx.kernalContext(); - final IgniteCacheObjectProcessor processor = kernalCtx.cacheObjects(); + GridKernalContext kernalCtx = sharedCtx.kernalContext(); + IgniteCacheObjectProcessor processor = kernalCtx.cacheObjects(); - if (processor != null && rec.type() == WALRecord.RecordType.DATA_RECORD) { + if (processor != null && rec.type() == RecordType.DATA_RECORD) { try { return postProcessDataRecord((DataRecord)rec, kernalCtx, processor); } @@ -296,11 +221,12 @@ private List scanIndexesFromFileHeaders * @throws IgniteCheckedException if failed. */ @NotNull private WALRecord postProcessDataRecord( - @NotNull final DataRecord dataRec, - final GridKernalContext kernalCtx, - final IgniteCacheObjectProcessor processor) throws IgniteCheckedException { - final CacheObjectContext fakeCacheObjCtx = new CacheObjectContext(kernalCtx, - null, null, false, false, false); + @NotNull DataRecord dataRec, + GridKernalContext kernalCtx, + IgniteCacheObjectProcessor processor + ) throws IgniteCheckedException { + final CacheObjectContext fakeCacheObjCtx = new CacheObjectContext( + kernalCtx, null, null, false, false, false); final List entries = dataRec.writeEntries(); final List postProcessedEntries = new ArrayList<>(entries.size()); @@ -327,8 +253,7 @@ private List scanIndexesFromFileHeaders * @return post precessed entry * @throws IgniteCheckedException if failed */ - @NotNull - private DataEntry postProcessDataEntry( + @NotNull private DataEntry postProcessDataEntry( final IgniteCacheObjectProcessor processor, final CacheObjectContext fakeCacheObjCtx, final DataEntry dataEntry) throws IgniteCheckedException { @@ -383,8 +308,9 @@ private DataEntry postProcessDataEntry( } /** {@inheritDoc} */ - @Override protected AbstractReadFileHandle createReadFileHandle(FileIO fileIO, long idx, - RecordSerializer ser, FileInput in) { - return new FileWriteAheadLogManager.ReadFileHandle(fileIO, idx, ser, in); + @Override protected AbstractReadFileHandle createReadFileHandle( + FileIO fileIO, long idx, RecordSerializer ser, FileInput in + ) { + return new ReadFileHandle(fileIO, idx, ser, in); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java index 468392a2631bb..2e2e2f8cbe942 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordSerializerFactoryImpl.java @@ -31,7 +31,7 @@ public class RecordSerializerFactoryImpl implements RecordSerializerFactory { private GridCacheSharedContext cctx; /** Write pointer. */ - private boolean writePointer; + private boolean needWritePointer; /** Read record filter. */ private IgniteBiPredicate recordDeserializeFilter; @@ -49,7 +49,17 @@ public class RecordSerializerFactoryImpl implements RecordSerializerFactory { * @param cctx Cctx. */ public RecordSerializerFactoryImpl(GridCacheSharedContext cctx) { + this(cctx, null); + } + /** + * @param cctx Cctx. + */ + public RecordSerializerFactoryImpl( + GridCacheSharedContext cctx, + IgniteBiPredicate readTypeFilter + ) { this.cctx = cctx; + this.recordDeserializeFilter = readTypeFilter; } /** {@inheritDoc} */ @@ -59,14 +69,24 @@ public RecordSerializerFactoryImpl(GridCacheSharedContext cctx) { switch (ver) { case 1: - return new RecordV1Serializer(new RecordDataV1Serializer(cctx), - writePointer, marshalledMode, skipPositionCheck, recordDeserializeFilter); + return new RecordV1Serializer( + new RecordDataV1Serializer(cctx), + needWritePointer, + marshalledMode, + skipPositionCheck, + recordDeserializeFilter); case 2: - RecordDataV2Serializer dataV2Serializer = new RecordDataV2Serializer(new RecordDataV1Serializer(cctx)); + RecordDataV2Serializer dataV2Serializer = new RecordDataV2Serializer( + new RecordDataV1Serializer(cctx)); - return new RecordV2Serializer(dataV2Serializer, - writePointer, marshalledMode, skipPositionCheck, recordDeserializeFilter); + return new RecordV2Serializer( + dataV2Serializer, + needWritePointer, + marshalledMode, + skipPositionCheck, + recordDeserializeFilter + ); default: throw new IgniteCheckedException("Failed to create a serializer with the given version " + @@ -78,12 +98,12 @@ public RecordSerializerFactoryImpl(GridCacheSharedContext cctx) { * @return Write pointer. */ public boolean writePointer() { - return writePointer; + return needWritePointer; } /** {@inheritDoc} */ @Override public RecordSerializerFactoryImpl writePointer(boolean writePointer) { - this.writePointer = writePointer; + this.needWritePointer = writePointer; return this; } @@ -97,7 +117,8 @@ public IgniteBiPredicate recordDeserializeFilt /** {@inheritDoc} */ @Override public RecordSerializerFactoryImpl recordDeserializeFilter( - IgniteBiPredicate readTypeFilter) { + IgniteBiPredicate readTypeFilter + ) { this.recordDeserializeFilter = readTypeFilter; return this; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java index caa096294e88b..ca484ce112031 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java @@ -134,6 +134,9 @@ public class RecordV1Serializer implements RecordSerializer { throw new SegmentEofException("WAL segment rollover detected (will end iteration) [expPtr=" + expPtr + ", readPtr=" + ptr + ']', null); + if (recType == null) + throw new IOException("Unknown record type: " + recType); + final WALRecord rec = dataSerializer.readRecord(recType, in); rec.position(ptr); @@ -341,12 +344,7 @@ static RecordType readRecordType(DataInput in) throws IgniteCheckedException, IO if (type == WALRecord.RecordType.STOP_ITERATION_RECORD_TYPE) throw new SegmentEofException("Reached logical end of the segment", null); - RecordType recType = RecordType.fromOrdinal(type - 1); - - if (recType == null) - throw new IOException("Unknown record type: " + type); - - return recType; + return RecordType.fromOrdinal(type - 1); } /** @@ -359,7 +357,11 @@ static RecordType readRecordType(DataInput in) throws IgniteCheckedException, IO * @throws EOFException In case of end of file. * @throws IgniteCheckedException If it's unable to read record. */ - static WALRecord readWithCrc(FileInput in0, WALPointer expPtr, RecordIO reader) throws EOFException, IgniteCheckedException { + static WALRecord readWithCrc( + FileInput in0, + WALPointer expPtr, + RecordIO reader + ) throws EOFException, IgniteCheckedException { long startPos = -1; try (FileInput.Crc32CheckingFileInput in = in0.startRead(skipCrc)) { @@ -377,7 +379,16 @@ static WALRecord readWithCrc(FileInput in0, WALPointer expPtr, RecordIO reader) throw e; } catch (Exception e) { - throw new IgniteCheckedException("Failed to read WAL record at position: " + startPos, e); + long size = -1; + + try { + size = in0.io().size(); + } + catch (IOException ignore) { + // No-op. It just for information. Fail calculate file size. + } + + throw new IgniteCheckedException("Failed to read WAL record at position: " + startPos + " size: " + size, e); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java index 2b81210e6fd0a..2c65ebe1d2c96 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java @@ -115,6 +115,9 @@ public class RecordV2Serializer implements RecordSerializer { FileWALPointer ptr = readPositionAndCheckPoint(in, expPtr, skipPositionCheck); + if (recType == null) + throw new IOException("Unknown record type: " + recType); + if (recordFilter != null && !recordFilter.apply(recType, ptr)) { int toSkip = ptr.length() - REC_TYPE_SIZE - FILE_WAL_POINTER_SIZE - CRC_SIZE; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorExceptionDuringReadTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorExceptionDuringReadTest.java new file mode 100644 index 0000000000000..ccd889aaaaf42 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorExceptionDuringReadTest.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.IgniteCache; +import org.apache.ignite.IgniteException; +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.pagemem.wal.WALIterator; +import org.apache.ignite.internal.pagemem.wal.WALPointer; +import org.apache.ignite.internal.pagemem.wal.record.WALRecord; +import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer; +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.X; +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.junit.Assert; + +/** + * + */ +public class IgniteWalIteratorExceptionDuringReadTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final int WAL_SEGMENT_SIZE = 1024 * 1024 * 20; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER)); + + cfg.setDataStorageConfiguration( + new DataStorageConfiguration() + .setWalSegmentSize(WAL_SEGMENT_SIZE) + .setWalMode(WALMode.LOG_ONLY) + .setDefaultDataRegionConfiguration( + new DataRegionConfiguration() + .setPersistenceEnabled(true) + ) + ); + + cfg.setCacheConfiguration(new CacheConfiguration(DEFAULT_CACHE_NAME)); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + cleanPersistenceDir(); + } + + /** + * @throws Exception If failed. + */ + public void test() throws Exception { + IgniteEx ig = (IgniteEx)startGrid(); + + ig.cluster().active(true); + + IgniteCache cache = ig.cache(DEFAULT_CACHE_NAME); + + for (int i = 0; i < 20 * 4; i++) + cache.put(i, new byte[1024 * 1024]); + + ig.cluster().active(false); + + IgniteWalIteratorFactory iteratorFactory = new IgniteWalIteratorFactory(log); + + FileWALPointer failOnPtr = new FileWALPointer(3, 1024 * 1024 * 5, 0); + + String failMessage = "test fail message"; + + IteratorParametersBuilder builder = new IteratorParametersBuilder() + .filesOrDirs(U.defaultWorkDirectory()) + .filter((r, ptr) -> { + FileWALPointer ptr0 = (FileWALPointer)ptr; + + if (ptr0.compareTo(failOnPtr) >= 0) + throw new TestRuntimeException(failMessage); + + return true; + }); + + try (WALIterator it = iteratorFactory.iterator(builder)) { + FileWALPointer ptr = null; + + boolean failed = false; + + while (it.hasNext()) { + try { + IgniteBiTuple tup = it.next(); + + ptr = (FileWALPointer)tup.get1(); + } + catch (IgniteException e) { + Assert.assertNotNull(ptr); + Assert.assertEquals(failOnPtr.index(), ptr.index()); + Assert.assertTrue(ptr.compareTo(failOnPtr) < 0); + + failed = X.hasCause(e, TestRuntimeException.class); + + break; + } + } + + assertTrue(failed); + } + } + + /** + * + */ + private static class TestRuntimeException extends IgniteException { + /** + * @param msg Exception message. + */ + private TestRuntimeException(String msg) { + super(msg); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java index 28f049aaef5c9..a6183a9073b9a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java @@ -23,7 +23,6 @@ import java.io.ObjectInput; import java.io.ObjectOutput; import java.io.Serializable; -import java.util.Arrays; import java.util.Collection; import java.util.EnumMap; import java.util.HashMap; @@ -48,11 +47,10 @@ import org.apache.ignite.cache.CacheRebalanceMode; 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.DataRegionConfiguration; import org.apache.ignite.configuration.WALMode; -import org.apache.ignite.events.Event; import org.apache.ignite.events.EventType; import org.apache.ignite.events.WalSegmentArchivedEvent; import org.apache.ignite.internal.pagemem.wal.WALIterator; @@ -66,24 +64,27 @@ import org.apache.ignite.internal.processors.cache.CacheObject; import org.apache.ignite.internal.processors.cache.GridCacheOperation; import org.apache.ignite.internal.processors.cache.KeyCacheObject; -import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager; 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.processors.cache.version.GridCacheVersion; -import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiInClosure; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteInClosure; -import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.logger.NullLogger; -import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.apache.ignite.transactions.Transaction; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.junit.Assert; +import static java.util.Arrays.fill; import static org.apache.ignite.events.EventType.EVT_WAL_SEGMENT_ARCHIVED; +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD; +import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.TX_RECORD; 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.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; @@ -93,6 +94,9 @@ * Test suite for WAL segments reader and event generator. */ public class IgniteWalReaderTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + /** Wal segments count */ private static final int WAL_SEGMENTS = 10; @@ -103,10 +107,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest { private static final String CACHE_ADDL_NAME = "cache1"; /** Dump records to logger. Should be false for non local run. */ - private static final boolean dumpRecords = false; - - /** Page size to set. */ - public static final int PAGE_SIZE = 4 * 1024; + private static final boolean DUMP_RECORDS = true; /** * Field for transferring setting from test to getConfig method. @@ -125,9 +126,11 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest { /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - final IgniteConfiguration cfg = super.getConfiguration(gridName); + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER)); - final CacheConfiguration ccfg = new CacheConfiguration<>(CACHE_NAME); + CacheConfiguration ccfg = new CacheConfiguration<>(CACHE_NAME); ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); ccfg.setRebalanceMode(CacheRebalanceMode.SYNC); @@ -140,8 +143,9 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest { DataStorageConfiguration dsCfg = new DataStorageConfiguration() .setDefaultDataRegionConfiguration( - new DataRegionConfiguration().setMaxSize(1024 * 1024 * 1024).setPersistenceEnabled(true)) - .setPageSize(PAGE_SIZE) + new DataRegionConfiguration() + .setMaxSize(1024 * 1024 * 1024) + .setPersistenceEnabled(true)) .setWalHistorySize(1) .setWalSegmentSize(1024 * 1024) .setWalSegments(WAL_SEGMENTS) @@ -150,12 +154,12 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest { if (archiveIncompleteSegmentAfterInactivityMs > 0) dsCfg.setWalAutoArchiveAfterInactivity(archiveIncompleteSegmentAfterInactivityMs); - final String workDir = U.defaultWorkDirectory(); - final File db = U.resolveWorkDirectory(workDir, DFLT_STORE_DIR, false); - final File wal = new File(db, "wal"); + String workDir = U.defaultWorkDirectory(); + File db = U.resolveWorkDirectory(workDir, DFLT_STORE_DIR, false); + File wal = new File(db, "wal"); if(setWalAndArchiveToSameValue) { - final String walAbsPath = wal.getAbsolutePath(); + String walAbsPath = wal.getAbsolutePath(); dsCfg.setWalPath(walAbsPath); dsCfg.setWalArchivePath(walAbsPath); @@ -173,85 +177,69 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest { @Override protected void beforeTest() throws Exception { stopAllGrids(); - deleteWorkFiles(); + cleanPersistenceDir(); } /** {@inheritDoc} */ @Override protected void afterTest() throws Exception { stopAllGrids(); + cleanPersistenceDir(); + if (clearProperties) System.clearProperty(IgniteSystemProperties.IGNITE_WAL_LOG_TX_RECORDS); } - /** - * @throws IgniteCheckedException If failed. - */ - private void deleteWorkFiles() throws Exception { - cleanPersistenceDir(); - } - /** * @throws Exception if failed. */ public void testFillWalAndReadRecords() throws Exception { setWalAndArchiveToSameValue = false; - final int cacheObjectsToWrite = 10000; - final Ignite ignite0 = startGrid("node0"); + Ignite ignite0 = startGrid(); - ignite0.active(true); + ignite0.cluster().active(true); - final Serializable consistentId = (Serializable)ignite0.cluster().localNode().consistentId(); - final String subfolderName = genNewStyleSubfolderName(0, (UUID)consistentId); + Serializable consistentId = (Serializable)ignite0.cluster().localNode().consistentId(); - putDummyRecords(ignite0, cacheObjectsToWrite); + String subfolderName = genNewStyleSubfolderName(0, (UUID)consistentId); - stopGrid("node0"); + int cacheObjectsToWrite = 10_000; - final String workDir = U.defaultWorkDirectory(); - final File db = U.resolveWorkDirectory(workDir, DFLT_STORE_DIR, false); - final File wal = new File(db, "wal"); - final File walArchive = setWalAndArchiveToSameValue ? wal : new File(wal, "archive"); + putDummyRecords(ignite0, cacheObjectsToWrite); - int[] checkKeyIterArr = new int[cacheObjectsToWrite]; + stopGrid(); - final File walArchiveDirWithConsistentId = new File(walArchive, subfolderName); - final File walWorkDirWithConsistentId = new File(wal, subfolderName); - final IgniteWalIteratorFactory factory = createWalIteratorFactory(workDir, subfolderName); + String workDir = U.defaultWorkDirectory(); - //Check iteratorArchiveDirectory and iteratorArchiveFiles are same. - final int cntArchiveDir = iterateAndCount(factory.iteratorArchiveDirectory(walArchiveDirWithConsistentId)); + File db = U.resolveWorkDirectory(workDir, DFLT_STORE_DIR, false); - log.info("Total records loaded using directory : " + cntArchiveDir); + IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log); - final int cntArchiveFileByFile = iterateAndCount(factory.iteratorArchiveFiles( - walArchiveDirWithConsistentId.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER))); + IteratorParametersBuilder params = + createIteratorParametersBuilder(workDir, subfolderName) + .filesOrDirs(db); - log.info("Total records loaded using archive directory (file-by-file): " + cntArchiveFileByFile); + // Check iteratorArchiveDirectory and iteratorArchiveFiles are same. + int cntArchiveDir = iterateAndCount(factory.iterator(params)); - assertTrue(cntArchiveDir == cntArchiveFileByFile); + log.info("Total records loaded using directory : " + cntArchiveDir); - //Check iteratorArchiveFiles + iteratorWorkFiles iterate over all entries. - Arrays.fill(checkKeyIterArr, 0); + assertTrue(cntArchiveDir > 0); - iterateAndCountDataRecord(factory.iteratorArchiveFiles( - walArchiveDirWithConsistentId.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER)), new IgniteBiInClosure() { - @Override public void apply(Object o, Object o2) { - checkKeyIterArr[(Integer)o]++; - } - }, null); + // Check iteratorArchiveFiles + iteratorWorkFiles iterate over all entries. + int[] checkKeyIterArr = new int[cacheObjectsToWrite]; - final File[] workFiles = walWorkDirWithConsistentId.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER); + fill(checkKeyIterArr, 0); - iterateAndCountDataRecord(factory.iteratorWorkFiles(workFiles), new IgniteBiInClosure() { - @Override public void apply(Object o, Object o2) { - checkKeyIterArr[(Integer) o]++; - } - }, null).size(); + iterateAndCountDataRecord( + factory.iterator(params), + (o1, o2) -> checkKeyIterArr[(Integer)o1]++, + null + ); - for (int i =0 ; i< cacheObjectsToWrite; i++) - assertTrue("Iterator didn't find key="+ i, checkKeyIterArr[i] > 0); + for (int i = 0; i < cacheObjectsToWrite; i++) + assertTrue("Iterator didn't find key=" + i, checkKeyIterArr[i] > 0); } /** @@ -262,35 +250,29 @@ public void testFillWalAndReadRecords() throws Exception { * @throws IgniteCheckedException if failed to iterate. */ private int iterateAndCount(WALIterator walIter) throws IgniteCheckedException { - return iterateAndCount(walIter, true); - } - - /** - * Iterates on records and closes iterator. - * - * @param walIter iterator to count, will be closed. - * @param touchEntries access data within entries. - * @return count of records. - * @throws IgniteCheckedException if failed to iterate. - */ - private int iterateAndCount(WALIterator walIter, boolean touchEntries) throws IgniteCheckedException { int cnt = 0; try (WALIterator it = walIter) { while (it.hasNextX()) { - final IgniteBiTuple next = it.nextX(); - final WALRecord walRecord = next.get2(); - if (touchEntries && walRecord.type() == WALRecord.RecordType.DATA_RECORD) { - final DataRecord record = (DataRecord)walRecord; + IgniteBiTuple tup = it.nextX(); + + WALRecord walRecord = tup.get2(); + + if (walRecord.type() == DATA_RECORD) { + DataRecord record = (DataRecord)walRecord; + for (DataEntry entry : record.writeEntries()) { - final KeyCacheObject key = entry.key(); - final CacheObject val = entry.value(); - if (dumpRecords) + KeyCacheObject key = entry.key(); + CacheObject val = entry.value(); + + if (DUMP_RECORDS) log.info("Op: " + entry.op() + ", Key: " + key + ", Value: " + val); } } - if (dumpRecords) + + if (DUMP_RECORDS) log.info("Record: " + walRecord); + cnt++; } } @@ -303,128 +285,148 @@ private int iterateAndCount(WALIterator walIter, boolean touchEntries) throws Ig * @throws Exception if failed. */ public void testArchiveCompletedEventFired() throws Exception { - final AtomicBoolean evtRecorded = new AtomicBoolean(); + AtomicBoolean evtRecorded = new AtomicBoolean(); - final Ignite ignite = startGrid("node0"); + Ignite ignite = startGrid(); - ignite.active(true); + ignite.cluster().active(true); final IgniteEvents evts = ignite.events(); if (!evts.isEnabled(EVT_WAL_SEGMENT_ARCHIVED)) - assertTrue("nothing to test", false); + fail("nothing to test"); - evts.localListen(new IgnitePredicate() { - @Override public boolean apply(Event e) { - WalSegmentArchivedEvent archComplEvt = (WalSegmentArchivedEvent)e; - long idx = archComplEvt.getAbsWalSegmentIdx(); - log.info("Finished archive for segment [" + idx + ", " + - archComplEvt.getArchiveFile() + "]: [" + e + "]"); + evts.localListen(e -> { + WalSegmentArchivedEvent archComplEvt = (WalSegmentArchivedEvent)e; - evtRecorded.set(true); - return true; - } + long idx = archComplEvt.getAbsWalSegmentIdx(); + + log.info("Finished archive for segment [" + + idx + ", " + archComplEvt.getArchiveFile() + "]: [" + e + "]"); + + evtRecorded.set(true); + + return true; }, EVT_WAL_SEGMENT_ARCHIVED); putDummyRecords(ignite, 500); - stopGrid("node0"); + stopGrid(); + assertTrue(evtRecorded.get()); } /** - * Puts provided number of records to fill WAL. + * Tests time out based WAL segment archiving. * - * @param ignite ignite instance. - * @param recordsToWrite count. + * @throws Exception if failure occurs. */ - private void putDummyRecords(Ignite ignite, int recordsToWrite) { - IgniteCache cache0 = ignite.cache(CACHE_NAME); + public void testArchiveIncompleteSegmentAfterInactivity() throws Exception { + AtomicBoolean waitingForEvt = new AtomicBoolean(); - for (int i = 0; i < recordsToWrite; i++) - cache0.put(i, new IndexedObject(i)); - } + CountDownLatch archiveSegmentForInactivity = new CountDownLatch(1); - /** - * Puts provided number of records to fill WAL. - * - * @param ignite ignite instance. - * @param recordsToWrite count. - */ - private void putAllDummyRecords(Ignite ignite, int recordsToWrite) { - IgniteCache cache0 = ignite.cache(CACHE_NAME); + archiveIncompleteSegmentAfterInactivityMs = 1000; - Map values = new HashMap<>(); + Ignite ignite = startGrid(); - for (int i = 0; i < recordsToWrite; i++) - values.put(i, new IndexedObject(i)); + ignite.cluster().active(true); - cache0.putAll(values); - } + IgniteEvents evts = ignite.events(); - /** - * Puts provided number of records to fill WAL under transactions. - * - * @param ignite ignite instance. - * @param recordsToWrite count. - * @param txCnt transactions to run. If number is less then records count, txCnt records will be written. - */ - private IgniteCache txPutDummyRecords(Ignite ignite, int recordsToWrite, int txCnt) { - IgniteCache cache0 = ignite.cache(CACHE_NAME); - int keysPerTx = recordsToWrite / txCnt; - if (keysPerTx == 0) - keysPerTx = 1; - for (int t = 0; t < txCnt; t++) { - try (Transaction tx = ignite.transactions().txStart()) { - for (int i = t * keysPerTx; i < (t + 1) * keysPerTx; i++) - cache0.put(i, new IndexedObject(i)); + evts.localListen(e -> { + WalSegmentArchivedEvent archComplEvt = (WalSegmentArchivedEvent)e; - tx.commit(); - } - } - return cache0; + long idx = archComplEvt.getAbsWalSegmentIdx(); + + log.info("Finished archive for segment [" + idx + ", " + + archComplEvt.getArchiveFile() + "]: [" + e + "]"); + + if (waitingForEvt.get()) + archiveSegmentForInactivity.countDown(); + + return true; + }, EVT_WAL_SEGMENT_ARCHIVED); + + putDummyRecords(ignite, 100); + + waitingForEvt.set(true); // Flag for skipping regular log() and rollOver(). + + log.info("Wait for archiving segment for inactive grid started"); + + boolean recordedAfterSleep = archiveSegmentForInactivity.await( + archiveIncompleteSegmentAfterInactivityMs + 1001, TimeUnit.MILLISECONDS); + + stopGrid(); + + assertTrue(recordedAfterSleep); } /** - * Tests time out based WAL segment archiving. + * Tests archive completed event is fired. * - * @throws Exception if failure occurs. + * @throws Exception if failed. */ - public void testArchiveIncompleteSegmentAfterInactivity() throws Exception { - final AtomicBoolean waitingForEvt = new AtomicBoolean(); - final CountDownLatch archiveSegmentForInactivity = new CountDownLatch(1); + public void testFillWalForExactSegmentsCount() throws Exception { + customWalMode = WALMode.FSYNC; - archiveIncompleteSegmentAfterInactivityMs = 1000; + CountDownLatch reqSegments = new CountDownLatch(15); - final Ignite ignite = startGrid("node0"); + Ignite ignite = startGrid(); - ignite.active(true); + ignite.cluster().active(true); final IgniteEvents evts = ignite.events(); - evts.localListen(new IgnitePredicate() { - @Override public boolean apply(Event e) { - WalSegmentArchivedEvent archComplEvt = (WalSegmentArchivedEvent)e; - long idx = archComplEvt.getAbsWalSegmentIdx(); - log.info("Finished archive for segment [" + idx + ", " + - archComplEvt.getArchiveFile() + "]: [" + e + "]"); + if (!evts.isEnabled(EVT_WAL_SEGMENT_ARCHIVED)) + fail("nothing to test"); - if (waitingForEvt.get()) - archiveSegmentForInactivity.countDown(); - return true; - } + evts.localListen(e -> { + WalSegmentArchivedEvent archComplEvt = (WalSegmentArchivedEvent)e; + + long idx = archComplEvt.getAbsWalSegmentIdx(); + + log.info("Finished archive for segment [" + idx + ", " + + archComplEvt.getArchiveFile() + "]: [" + e + "]"); + + reqSegments.countDown(); + + return true; }, EVT_WAL_SEGMENT_ARCHIVED); - putDummyRecords(ignite, 100); - waitingForEvt.set(true); //flag for skipping regular log() and rollOver() + int totalEntries = 0; - log.info("Wait for archiving segment for inactive grid started"); + while (reqSegments.getCount() > 0) { + int write = 500; - boolean recordedAfterSleep = - archiveSegmentForInactivity.await(archiveIncompleteSegmentAfterInactivityMs + 1001, TimeUnit.MILLISECONDS); + putAllDummyRecords(ignite, write); - stopGrid("node0"); - assertTrue(recordedAfterSleep); + totalEntries += write; + + Assert.assertTrue("Too much entries generated, but segments was not become available", + totalEntries < 10000); + } + + String subfolderName = genDbSubfolderName(ignite, 0); + + stopGrid(); + + String workDir = U.defaultWorkDirectory(); + + IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log); + + IteratorParametersBuilder iteratorParametersBuilder = createIteratorParametersBuilder(workDir, subfolderName); + + iteratorParametersBuilder.filesOrDirs(workDir); + + scanIterateAndCount( + factory, + iteratorParametersBuilder, + totalEntries, + 0, + null, + null + ); } /** @@ -450,51 +452,50 @@ private boolean remove(Map m, Object key, Object val) { * @throws Exception if failed. */ public void testTxFillWalAndExtractDataRecords() throws Exception { - final int cntEntries = 1000; - final int txCnt = 100; + Ignite ignite0 = startGrid(); - final Ignite ignite0 = startGrid("node0"); + ignite0.cluster().active(true); - ignite0.active(true); + int cntEntries = 1000; + int txCnt = 100; - final IgniteCache entries = txPutDummyRecords(ignite0, cntEntries, txCnt); + IgniteCache entries = txPutDummyRecords(ignite0, cntEntries, txCnt); - final Map ctrlMap = new HashMap<>(); for (Cache.Entry next : entries) - ctrlMap.put(next.getKey(), next.getValue()); + Map ctrlMap = new HashMap<>(); + for (Cache.Entry next : entries) + ctrlMap.put(next.getKey(), next.getValue()); - final String subfolderName = genDbSubfolderName(ignite0, 0); - stopGrid("node0"); + String subfolderName = genDbSubfolderName(ignite0, 0); - final String workDir = U.defaultWorkDirectory(); - final File binaryMeta = U.resolveWorkDirectory(workDir, "binary_meta", false); - final File binaryMetaWithConsId = new File(binaryMeta, subfolderName); - final File marshallerMapping = U.resolveWorkDirectory(workDir, "marshaller", false); + stopGrid(); - final IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log, - PAGE_SIZE, - binaryMetaWithConsId, - marshallerMapping); + String workDir = U.defaultWorkDirectory(); - final IgniteBiInClosure objConsumer = new IgniteBiInClosure() { - @Override public void apply(Object key, Object val) { - boolean rmv = remove(ctrlMap, key, val); - if (!rmv) - log.error("Unable to remove Key and value from control Map K:[" + key + "] V: [" + val + "]"); + IteratorParametersBuilder params = createIteratorParametersBuilder(workDir,subfolderName); - if (val instanceof IndexedObject) { - IndexedObject indexedObj = (IndexedObject)val; + params.filesOrDirs(workDir); - assertEquals(indexedObj.iVal, indexedObj.jVal); - assertEquals(indexedObj.iVal, key); + IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log); - for (byte datum : indexedObj.getData()) - assertTrue(datum >= 'A' && datum <= 'A' + 10); - } + IgniteBiInClosure objConsumer = (key, val) -> { + boolean rmv = remove(ctrlMap, key, val); + + if (!rmv) + log.error("Unable to remove Key and value from control Map K:[" + key + "] V: [" + val + "]"); + + if (val instanceof IndexedObject) { + IndexedObject indexedObj = (IndexedObject)val; + + assertEquals(indexedObj.iVal, indexedObj.jVal); + assertEquals(indexedObj.iVal, key); + + for (byte datum : indexedObj.getData()) + assertTrue(datum >= 'A' && datum <= 'A' + 10); } }; - scanIterateAndCount(factory, workDir, subfolderName, cntEntries, txCnt, objConsumer, null); + scanIterateAndCount(factory, params, cntEntries, txCnt, objConsumer, null); assertTrue(" Control Map is not empty after reading entries " + ctrlMap, ctrlMap.isEmpty()); } @@ -514,8 +515,6 @@ public void testTxFillWalAndExtractDataRecords() throws Exception { * Scan WAL and WAL archive for logical records and its entries. * * @param factory WAL iterator factory. - * @param workDir Ignite work directory. - * @param subfolderName DB subfolder name based on consistent ID. * @param minCntEntries minimum expected entries count to find. * @param minTxCnt minimum expected transaction count to find. * @param objConsumer object handler, called for each object found in logical data records. @@ -523,67 +522,43 @@ public void testTxFillWalAndExtractDataRecords() throws Exception { * @throws IgniteCheckedException if failed. */ private void scanIterateAndCount( - final IgniteWalIteratorFactory factory, - final String workDir, - final String subfolderName, - final int minCntEntries, - final int minTxCnt, - @Nullable final IgniteBiInClosure objConsumer, - @Nullable final IgniteInClosure dataRecordHnd) throws IgniteCheckedException { - - final File db = U.resolveWorkDirectory(workDir, DFLT_STORE_DIR, false); - final File wal = new File(db, "wal"); - final File walArchive = new File(wal, "archive"); - - final File walArchiveDirWithConsistentId = new File(walArchive, subfolderName); - - final File[] files = walArchiveDirWithConsistentId.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER); - A.notNull(files, "Can't iterate over files [" + walArchiveDirWithConsistentId + "] Directory is N/A"); - final WALIterator iter = factory.iteratorArchiveFiles(files); + IgniteWalIteratorFactory factory, + IteratorParametersBuilder itParamBuilder, + int minCntEntries, + int minTxCnt, + @Nullable IgniteBiInClosure objConsumer, + @Nullable IgniteInClosure dataRecordHnd + ) throws IgniteCheckedException { + WALIterator iter = factory.iterator(itParamBuilder); - final Map cntArch = iterateAndCountDataRecord(iter, objConsumer, dataRecordHnd); + Map cntArch = iterateAndCountDataRecord(iter, objConsumer, dataRecordHnd); int txCntObservedArch = cntArch.size(); - if (cntArch.containsKey(null)) - txCntObservedArch -= 1; // exclude non transactional updates - final int entriesArch = valuesSum(cntArch.values()); - log.info("Total tx found loaded using archive directory (file-by-file): " + txCntObservedArch); + if (cntArch.containsKey(null)) + txCntObservedArch -= 1; // Exclude non transactional updates. - final File walWorkDirWithNodeSubDir = new File(wal, subfolderName); - final File[] workFiles = walWorkDirWithNodeSubDir.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER); + int entries = valuesSum(cntArch.values()); - final WALIterator tuples = factory.iteratorWorkFiles(workFiles); - final Map cntWork = iterateAndCountDataRecord(tuples, objConsumer, dataRecordHnd); - int txCntObservedWork = cntWork.size(); - if (cntWork.containsKey(null)) - txCntObservedWork -= 1; // exclude non transactional updates + log.info("Total tx found loaded using archive directory (file-by-file): " + txCntObservedArch); - final int entriesWork = valuesSum(cntWork.values()); - log.info("Archive directory: Tx found " + txCntObservedWork + " entries " + entriesWork); + assertTrue("txCntObservedArch=" + txCntObservedArch + " >= minTxCnt=" + minTxCnt, + txCntObservedArch >= minTxCnt); - assertTrue("entriesArch=" + entriesArch + " + entriesWork=" + entriesWork - + " >= minCntEntries=" + minCntEntries, - entriesArch + entriesWork >= minCntEntries); - assertTrue("txCntObservedWork=" + txCntObservedWork + " + txCntObservedArch=" + txCntObservedArch - + " >= minTxCnt=" + minTxCnt, - txCntObservedWork + txCntObservedArch >= minTxCnt); + assertTrue("entries=" + entries + " >= minCntEntries=" + minCntEntries, + entries >= minCntEntries); } /** * @throws Exception if failed. */ public void testFillWalWithDifferentTypes() throws Exception { - int cntEntries; + Ignite ig = startGrid(); - final Map ctrlMap = new HashMap<>(); - final Map ctrlMapForBinaryObjects = new HashMap<>(); - final Collection ctrlStringsToSearch = new HashSet<>(); - final Collection ctrlStringsForBinaryObjSearch = new HashSet<>(); - final Ignite ignite0 = startGrid("node0"); - ignite0.active(true); + ig.cluster().active(true); + + IgniteCache addlCache = ig.getOrCreateCache(CACHE_ADDL_NAME); - final IgniteCache addlCache = ignite0.getOrCreateCache(CACHE_ADDL_NAME); addlCache.put("1", "2"); addlCache.put(1, 2); addlCache.put(1L, 2L); @@ -597,200 +572,173 @@ public void testFillWalWithDifferentTypes() throws Exception { addlCache.put(new TestExternalizable(42), "Externalizable_As_Key"); addlCache.put(292, new IndexedObject(292)); - final String search1 = "SomeUnexpectedStringValueAsKeyToSearch"; + String search1 = "SomeUnexpectedStringValueAsKeyToSearch"; + + Collection ctrlStringsToSearch = new HashSet<>(); + ctrlStringsToSearch.add(search1); + + Collection ctrlStringsForBinaryObjSearch = new HashSet<>(); + ctrlStringsForBinaryObjSearch.add(search1); + addlCache.put(search1, "SearchKey"); String search2 = "SomeTestStringContainerToBePrintedLongLine"; - final TestStringContainerToBePrinted val = new TestStringContainerToBePrinted(search2); + + TestStringContainerToBePrinted val = new TestStringContainerToBePrinted(search2); + ctrlStringsToSearch.add(val.toString()); //will validate original toString() was called ctrlStringsForBinaryObjSearch.add(search2); + addlCache.put("SearchValue", val); String search3 = "SomeTestStringContainerToBePrintedLongLine2"; - final TestStringContainerToBePrinted key = new TestStringContainerToBePrinted(search3); + + TestStringContainerToBePrinted key = new TestStringContainerToBePrinted(search3); ctrlStringsToSearch.add(key.toString()); //will validate original toString() was called ctrlStringsForBinaryObjSearch.add(search3); //validate only string itself + addlCache.put(key, "SearchKey"); - cntEntries = addlCache.size(); - for (Cache.Entry next : addlCache) - ctrlMap.put(next.getKey(), next.getValue()); + int cntEntries = addlCache.size(); - for (Cache.Entry next : addlCache) - ctrlMapForBinaryObjects.put(next.getKey(), next.getValue()); + Map ctrlMap = new HashMap<>(); + for (Cache.Entry next : addlCache) + ctrlMap.put(next.getKey(), next.getValue()); - final String subfolderName = genDbSubfolderName(ignite0, 0); + Map ctrlMapForBinaryObjects = new HashMap<>(); - stopGrid("node0"); + for (Cache.Entry next : addlCache) + ctrlMapForBinaryObjects.put(next.getKey(), next.getValue()); - final String workDir = U.defaultWorkDirectory(); + String subfolderName = genDbSubfolderName(ig, 0); - final File binaryMeta = U.resolveWorkDirectory(workDir, "binary_meta", false); - final File binaryMetaWithNodeSubfolder = new File(binaryMeta, subfolderName); - final File marshallerMapping = U.resolveWorkDirectory(workDir, "marshaller", false); + // Wait async allocation wal segment file by archiver. + Thread.sleep(1000); - final IgniteWalIteratorFactory factory = createWalIteratorFactory(workDir, subfolderName); - final IgniteBiInClosure objConsumer = new IgniteBiInClosure() { - @Override public void apply(Object key, Object val) { - log.info("K: [" + key + ", " + - (key != null ? key.getClass().getName() : "?") + "]" + - " V: [" + val + ", " + - (val != null ? val.getClass().getName() : "?") + "]"); - boolean rmv = remove(ctrlMap, key, val); - if (!rmv) { - String msg = "Unable to remove pair from control map " + "K: [" + key + "] V: [" + val + "]"; - log.error(msg); - } - assertFalse(val instanceof BinaryObject); + stopGrid("node0", false); + + String workDir = U.defaultWorkDirectory(); + + IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log); + + IteratorParametersBuilder params0 = createIteratorParametersBuilder(workDir, subfolderName); + + params0.filesOrDirs(workDir); + + IgniteBiInClosure objConsumer = (key12, val1) -> { + log.info("K: [" + key12 + ", " + + (key12 != null ? key12.getClass().getName() : "?") + "]" + + " V: [" + val1 + ", " + + (val1 != null ? val1.getClass().getName() : "?") + "]"); + boolean rmv = remove(ctrlMap, key12, val1); + if (!rmv) { + String msg = "Unable to remove pair from control map " + "K: [" + key12 + "] V: [" + val1 + "]"; + log.error(msg); } + assertFalse(val1 instanceof BinaryObject); }; - final IgniteInClosure toStrChecker = new IgniteInClosure() { - @Override public void apply(DataRecord record) { - String strRepresentation = record.toString(); - for (Iterator iter = ctrlStringsToSearch.iterator(); iter.hasNext(); ) { - final String next = iter.next(); - if (strRepresentation.contains(next)) { - iter.remove(); - break; - } + IgniteInClosure toStrChecker = record -> { + String strRepresentation = record.toString(); + + for (Iterator iter = ctrlStringsToSearch.iterator(); iter.hasNext(); ) { + final String next = iter.next(); + if (strRepresentation.contains(next)) { + iter.remove(); + break; } } }; - scanIterateAndCount(factory, workDir, subfolderName, cntEntries, 0, objConsumer, toStrChecker); + + scanIterateAndCount(factory, params0, cntEntries, 0, objConsumer, toStrChecker); assertTrue(" Control Map is not empty after reading entries: " + ctrlMap, ctrlMap.isEmpty()); assertTrue(" Control Map for strings in entries is not empty after" + " reading records: " + ctrlStringsToSearch, ctrlStringsToSearch.isEmpty()); - //Validate same WAL log with flag binary objects only - final IgniteWalIteratorFactory keepBinFactory = new IgniteWalIteratorFactory(log, PAGE_SIZE, - binaryMetaWithNodeSubfolder, - marshallerMapping, - true); - final IgniteBiInClosure binObjConsumer = new IgniteBiInClosure() { - @Override public void apply(Object key, Object val) { - log.info("K(KeepBinary): [" + key + ", " + - (key != null ? key.getClass().getName() : "?") + "]" + - " V(KeepBinary): [" + val + ", " + - (val != null ? val.getClass().getName() : "?") + "]"); - boolean rmv = remove(ctrlMapForBinaryObjects, key, val); - if (!rmv) { - if (key instanceof BinaryObject) { - BinaryObject keyBinObj = (BinaryObject)key; - String binaryObjTypeName = keyBinObj.type().typeName(); - if (Objects.equals(TestStringContainerToBePrinted.class.getName(), binaryObjTypeName)) { - String data = keyBinObj.field("data"); - rmv = ctrlMapForBinaryObjects.remove(new TestStringContainerToBePrinted(data)) != null; - } - else if (Objects.equals(TestSerializable.class.getName(), binaryObjTypeName)) { - Integer iVal = keyBinObj.field("iVal"); - rmv = ctrlMapForBinaryObjects.remove(new TestSerializable(iVal)) != null; - } - else if (Objects.equals(TestEnum.class.getName(), binaryObjTypeName)) { - TestEnum key1 = TestEnum.values()[keyBinObj.enumOrdinal()]; - rmv = ctrlMapForBinaryObjects.remove(key1) != null; - } + IgniteBiInClosure binObjConsumer = (key13, val12) -> { + log.info("K(KeepBinary): [" + key13 + ", " + + (key13 != null ? key13.getClass().getName() : "?") + "]" + + " V(KeepBinary): [" + val12 + ", " + + (val12 != null ? val12.getClass().getName() : "?") + "]"); + + boolean rmv = remove(ctrlMapForBinaryObjects, key13, val12); + + if (!rmv) { + if (key13 instanceof BinaryObject) { + BinaryObject keyBinObj = (BinaryObject)key13; + String binaryObjTypeName = keyBinObj.type().typeName(); + + if (Objects.equals(TestStringContainerToBePrinted.class.getName(), binaryObjTypeName)) { + String data = keyBinObj.field("data"); + rmv = ctrlMapForBinaryObjects.remove(new TestStringContainerToBePrinted(data)) != null; } - else if (val instanceof BinaryObject) { - //don't compare BO values, just remove by key - rmv = ctrlMapForBinaryObjects.remove(key) != null; + else if (Objects.equals(TestSerializable.class.getName(), binaryObjTypeName)) { + Integer iVal = keyBinObj.field("iVal"); + rmv = ctrlMapForBinaryObjects.remove(new TestSerializable(iVal)) != null; } - } - if (!rmv) - log.error("Unable to remove pair from control map " + "K: [" + key + "] V: [" + val + "]"); - - if (val instanceof BinaryObject) { - BinaryObject binaryObj = (BinaryObject)val; - String binaryObjTypeName = binaryObj.type().typeName(); - if (Objects.equals(IndexedObject.class.getName(), binaryObjTypeName)) { - assertEquals(binaryObj.field("iVal").toString(), - binaryObj.field("jVal").toString()); - - byte data[] = binaryObj.field("data"); - for (byte datum : data) - assertTrue(datum >= 'A' && datum <= 'A' + 10); + else if (Objects.equals(TestEnum.class.getName(), binaryObjTypeName)) { + TestEnum key1 = TestEnum.values()[keyBinObj.enumOrdinal()]; + rmv = ctrlMapForBinaryObjects.remove(key1) != null; } } + else if (val12 instanceof BinaryObject) { + //don't compare BO values, just remove by key + rmv = ctrlMapForBinaryObjects.remove(key13) != null; + } } - }; + if (!rmv) + log.error("Unable to remove pair from control map " + "K: [" + key13 + "] V: [" + val12 + "]"); - final IgniteInClosure binObjToStrChecker = new IgniteInClosure() { - @Override public void apply(DataRecord record) { - String strRepresentation = record.toString(); + if (val12 instanceof BinaryObject) { + BinaryObject binaryObj = (BinaryObject)val12; + String binaryObjTypeName = binaryObj.type().typeName(); - for (Iterator iter = ctrlStringsForBinaryObjSearch.iterator(); iter.hasNext(); ) { - final String next = iter.next(); + if (Objects.equals(IndexedObject.class.getName(), binaryObjTypeName)) { + assertEquals( + binaryObj.field("iVal").toString(), + binaryObj.field("jVal").toString() + ); - if (strRepresentation.contains(next)) { - iter.remove(); + byte data[] = binaryObj.field("data"); - break; - } + for (byte datum : data) + assertTrue(datum >= 'A' && datum <= 'A' + 10); } } }; - scanIterateAndCount(keepBinFactory, workDir, subfolderName, cntEntries, 0, binObjConsumer, binObjToStrChecker); - - assertTrue(" Control Map is not empty after reading entries: " + ctrlMapForBinaryObjects, - ctrlMapForBinaryObjects.isEmpty()); - assertTrue(" Control Map for strings in entries is not empty after" + - " reading records: " + ctrlStringsForBinaryObjSearch, - ctrlStringsForBinaryObjSearch.isEmpty()); - - } - - /** - * Tests archive completed event is fired. - * - * @throws Exception if failed. - */ - public void testFillWalForExactSegmentsCount() throws Exception { - customWalMode = WALMode.FSYNC; - - final CountDownLatch reqSegments = new CountDownLatch(15); - final Ignite ignite = startGrid("node0"); - - ignite.active(true); + IgniteInClosure binObjToStrChecker = record -> { + String strRepresentation = record.toString(); - final IgniteEvents evts = ignite.events(); - - if (!evts.isEnabled(EVT_WAL_SEGMENT_ARCHIVED)) - assertTrue("nothing to test", false); + for (Iterator iter = ctrlStringsForBinaryObjSearch.iterator(); iter.hasNext(); ) { + final String next = iter.next(); - evts.localListen(new IgnitePredicate() { - @Override public boolean apply(Event e) { - WalSegmentArchivedEvent archComplEvt = (WalSegmentArchivedEvent)e; - long idx = archComplEvt.getAbsWalSegmentIdx(); - log.info("Finished archive for segment [" + idx + ", " + - archComplEvt.getArchiveFile() + "]: [" + e + "]"); + if (strRepresentation.contains(next)) { + iter.remove(); - reqSegments.countDown(); - return true; + break; + } } - }, EVT_WAL_SEGMENT_ARCHIVED); + }; + IteratorParametersBuilder params1 = createIteratorParametersBuilder(workDir, subfolderName); - int totalEntries = 0; - while (reqSegments.getCount() > 0) { - final int write = 500; - putAllDummyRecords(ignite, write); - totalEntries += write; - Assert.assertTrue("Too much entries generated, but segments was not become available", - totalEntries < 10000); - } - final String subfolderName = genDbSubfolderName(ignite, 0); + params1.filesOrDirs(workDir).keepBinary(true); - stopGrid("node0"); + //Validate same WAL log with flag binary objects only + IgniteWalIteratorFactory keepBinFactory = new IgniteWalIteratorFactory(log); - final String workDir = U.defaultWorkDirectory(); - final IgniteWalIteratorFactory factory = createWalIteratorFactory(workDir, subfolderName); + scanIterateAndCount(keepBinFactory, params1, cntEntries, 0, binObjConsumer, binObjToStrChecker); - scanIterateAndCount(factory, workDir, subfolderName, totalEntries, 0, null, null); + assertTrue(" Control Map is not empty after reading entries: " + + ctrlMapForBinaryObjects, ctrlMapForBinaryObjects.isEmpty()); + + assertTrue(" Control Map for strings in entries is not empty after" + + " reading records: " + ctrlStringsForBinaryObjSearch, ctrlStringsForBinaryObjSearch.isEmpty()); } /** @@ -801,19 +749,32 @@ public void testFillWalForExactSegmentsCount() throws Exception { public void testReadEmptyWal() throws Exception { customWalMode = WALMode.FSYNC; - final Ignite ignite = startGrid("node0"); + Ignite ignite = startGrid(); - ignite.active(true); - ignite.active(false); + ignite.cluster().active(true); + + ignite.cluster().active(false); final String subfolderName = genDbSubfolderName(ignite, 0); - stopGrid("node0"); + stopGrid(); + + String workDir = U.defaultWorkDirectory(); + + IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log); - final String workDir = U.defaultWorkDirectory(); - final IgniteWalIteratorFactory factory = createWalIteratorFactory(workDir, subfolderName); + IteratorParametersBuilder iteratorParametersBuilder = + createIteratorParametersBuilder(workDir, subfolderName) + .filesOrDirs(workDir); - scanIterateAndCount(factory, workDir, subfolderName, 0, 0, null, null); + scanIterateAndCount( + factory, + iteratorParametersBuilder, + 0, + 0, + null, + null + ); } /** @@ -872,54 +833,72 @@ public void testRemoveOperationPresentedForDataEntryForAtomic() throws Exception * @param mode Cache Atomicity Mode. */ private void runRemoveOperationTest(CacheAtomicityMode mode) throws Exception { - final Ignite ignite = startGrid("node0"); + Ignite ignite = startGrid(); + + ignite.cluster().active(true); - ignite.active(true); createCache2(ignite, mode); - ignite.active(false); - final String subfolderName = genDbSubfolderName(ignite, 0); + ignite.cluster().active(false); + + String subfolderName = genDbSubfolderName(ignite, 0); + + stopGrid(); + + String workDir = U.defaultWorkDirectory(); + + IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log); - stopGrid("node0"); + IteratorParametersBuilder params = createIteratorParametersBuilder(workDir, subfolderName); - final String workDir = U.defaultWorkDirectory(); - final IgniteWalIteratorFactory factory = createWalIteratorFactory(workDir, subfolderName); + params.filesOrDirs(workDir); - final StringBuilder builder = new StringBuilder(); - final Map operationsFound = new EnumMap<>(GridCacheOperation.class); + StringBuilder sb = new StringBuilder(); - scanIterateAndCount(factory, workDir, subfolderName, 0, 0, null, new IgniteInClosure() { - @Override public void apply(DataRecord dataRecord) { + Map operationsFound = new EnumMap<>(GridCacheOperation.class); + + scanIterateAndCount( + factory, + params, + 0, + 0, + null, + dataRecord -> { final List entries = dataRecord.writeEntries(); - builder.append("{"); + sb.append("{"); + for (DataEntry entry : entries) { - final GridCacheOperation op = entry.op(); - final Integer cnt = operationsFound.get(op); + GridCacheOperation op = entry.op(); + Integer cnt = operationsFound.get(op); operationsFound.put(op, cnt == null ? 1 : (cnt + 1)); if (entry instanceof UnwrapDataEntry) { - final UnwrapDataEntry entry1 = (UnwrapDataEntry)entry; + UnwrapDataEntry entry1 = (UnwrapDataEntry)entry; - builder.append(entry1.op()).append(" for ").append(entry1.unwrappedKey()); - final GridCacheVersion ver = entry.nearXidVersion(); + sb.append(entry1.op()) + .append(" for ") + .append(entry1.unwrappedKey()); - builder.append(", "); + GridCacheVersion ver = entry.nearXidVersion(); + + sb.append(", "); if (ver != null) - builder.append("tx=").append(ver).append(", "); + sb.append("tx=") + .append(ver) + .append(", "); } } - builder.append("}\n"); - } - }); + sb.append("}\n"); + }); final Integer deletesFound = operationsFound.get(DELETE); if (log.isInfoEnabled()) - log.info(builder.toString()); + log.info(sb.toString()); assertTrue("Delete operations should be found in log: " + operationsFound, deletesFound != null && deletesFound > 0); @@ -927,109 +906,144 @@ private void runRemoveOperationTest(CacheAtomicityMode mode) throws Exception { /** * Tests transaction generation and WAL for putAll cache operation. + * * @throws Exception if failed. */ public void testPutAllTxIntoTwoNodes() throws Exception { - final Ignite ignite = startGrid("node0"); - final Ignite ignite1 = startGrid(1); + Ignite ignite = startGrid("node0"); + Ignite ignite1 = startGrid(1); + + ignite.cluster().active(true); - ignite.active(true); + Map map = new TreeMap<>(); - final Map map = new TreeMap<>(); + int cntEntries = 1000; - final int cntEntries = 1000; for (int i = 0; i < cntEntries; i++) map.put(i, new IndexedObject(i)); ignite.cache(CACHE_NAME).putAll(map); - ignite.active(false); + ignite.cluster().active(false); - final String subfolderName = genDbSubfolderName(ignite, 0); - final String subfolderName1 = genDbSubfolderName(ignite1, 1); + String subfolderName1 = genDbSubfolderName(ignite, 0); + String subfolderName2 = genDbSubfolderName(ignite1, 1); stopAllGrids(); - final String workDir = U.defaultWorkDirectory(); - final IgniteWalIteratorFactory factory = createWalIteratorFactory(workDir, subfolderName); + String workDir = U.defaultWorkDirectory(); - final StringBuilder builder = new StringBuilder(); - final Map operationsFound = new EnumMap<>(GridCacheOperation.class); + IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log); - final IgniteInClosure drHnd = new IgniteInClosure() { - @Override public void apply(DataRecord dataRecord) { - final List entries = dataRecord.writeEntries(); + StringBuilder sb = new StringBuilder(); - builder.append("{"); - for (DataEntry entry : entries) { - final GridCacheOperation op = entry.op(); - final Integer cnt = operationsFound.get(op); + Map operationsFound = new EnumMap<>(GridCacheOperation.class); - operationsFound.put(op, cnt == null ? 1 : (cnt + 1)); + IgniteInClosure drHnd = dataRecord -> { + List entries = dataRecord.writeEntries(); - if (entry instanceof UnwrapDataEntry) { - final UnwrapDataEntry entry1 = (UnwrapDataEntry)entry; + sb.append("{"); - builder.append(entry1.op()).append(" for ").append(entry1.unwrappedKey()); - final GridCacheVersion ver = entry.nearXidVersion(); + for (DataEntry entry : entries) { + GridCacheOperation op = entry.op(); + Integer cnt = operationsFound.get(op); - builder.append(", "); + operationsFound.put(op, cnt == null ? 1 : (cnt + 1)); - if (ver != null) - builder.append("tx=").append(ver).append(", "); - } - } + if (entry instanceof UnwrapDataEntry) { + final UnwrapDataEntry entry1 = (UnwrapDataEntry)entry; + + sb.append(entry1.op()).append(" for ").append(entry1.unwrappedKey()); + final GridCacheVersion ver = entry.nearXidVersion(); + + sb.append(", "); - builder.append("}\n"); + if (ver != null) + sb.append("tx=").append(ver).append(", "); + } } + + sb.append("}\n"); }; - scanIterateAndCount(factory, workDir, subfolderName, 1, 1, null, drHnd); - scanIterateAndCount(factory, workDir, subfolderName1, 1, 1, null, drHnd); - final Integer createsFound = operationsFound.get(CREATE); + scanIterateAndCount( + factory, + createIteratorParametersBuilder(workDir, subfolderName1) + .filesOrDirs( + workDir + "/db/wal/" + subfolderName1, + workDir + "/db/wal/archive/" + subfolderName1 + ), + 1, + 1, + null, drHnd + ); + + scanIterateAndCount( + factory, + createIteratorParametersBuilder(workDir, subfolderName2) + .filesOrDirs( + workDir + "/db/wal/" + subfolderName2, + workDir + "/db/wal/archive/" + subfolderName2 + ), + 1, + 1, + null, + drHnd + ); + + Integer createsFound = operationsFound.get(CREATE); if (log.isInfoEnabled()) - log.info(builder.toString()); + log.info(sb.toString()); assertTrue("Create operations should be found in log: " + operationsFound, createsFound != null && createsFound > 0); assertTrue("Create operations count should be at least " + cntEntries + " in log: " + operationsFound, createsFound != null && createsFound >= cntEntries); - } /** * Tests transaction generation and WAL for putAll cache operation. + * * @throws Exception if failed. */ public void testTxRecordsReadWoBinaryMeta() throws Exception { clearProperties = true; + System.setProperty(IgniteSystemProperties.IGNITE_WAL_LOG_TX_RECORDS, "true"); - final Ignite ignite = startGrid("node0"); - ignite.active(true); + Ignite ignite = startGrid("node0"); - final Map map = new TreeMap<>(); + ignite.cluster().active(true); + + Map map = new TreeMap<>(); for (int i = 0; i < 1000; i++) map.put(i, new IndexedObject(i)); ignite.cache(CACHE_NAME).putAll(map); - ignite.active(false); + ignite.cluster().active(false); + + String workDir = U.defaultWorkDirectory(); + + String subfolderName = genDbSubfolderName(ignite, 0); - final String workDir = U.defaultWorkDirectory(); - final String subfolderName = genDbSubfolderName(ignite, 0); stopAllGrids(); - IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(new NullLogger(), - PAGE_SIZE, - null, - null, - false); + IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(new NullLogger()); + + IteratorParametersBuilder params = createIteratorParametersBuilder(workDir, subfolderName); - scanIterateAndCount(factory, workDir, subfolderName, 1000, 1, null, null); + scanIterateAndCount( + factory, + params.filesOrDirs(workDir), + 1000, + 1, + null, + null + ); } /** @@ -1038,19 +1052,17 @@ public void testTxRecordsReadWoBinaryMeta() throws Exception { * @return WAL iterator factory. * @throws IgniteCheckedException If failed. */ - @NotNull private IgniteWalIteratorFactory createWalIteratorFactory( - final String workDir, - final String subfolderName + @NotNull private IteratorParametersBuilder createIteratorParametersBuilder( + String workDir, + String subfolderName ) throws IgniteCheckedException { - final File binaryMeta = U.resolveWorkDirectory(workDir, "binary_meta", false); - final File binaryMetaWithConsId = new File(binaryMeta, subfolderName); - final File marshallerMapping = U.resolveWorkDirectory(workDir, "marshaller", false); - - return new IgniteWalIteratorFactory(log, - PAGE_SIZE, - binaryMetaWithConsId, - marshallerMapping, - false); + File binaryMeta = U.resolveWorkDirectory(workDir, "binary_meta", false); + File binaryMetaWithConsId = new File(binaryMeta, subfolderName); + File marshallerMapping = U.resolveWorkDirectory(workDir, "marshaller", false); + + return new IteratorParametersBuilder() + .binaryMetadataFileStoreDir(binaryMetaWithConsId) + .marshallerMappingFileStoreDir(marshallerMapping); } /** @@ -1074,28 +1086,33 @@ private int valuesSum(Iterable values) { * @throws IgniteCheckedException if failure. */ private Map iterateAndCountDataRecord( - final WALIterator walIter, - @Nullable final IgniteBiInClosure cacheObjHnd, - @Nullable final IgniteInClosure dataRecordHnd) throws IgniteCheckedException { + WALIterator walIter, + @Nullable IgniteBiInClosure cacheObjHnd, + @Nullable IgniteInClosure dataRecordHnd + ) throws IgniteCheckedException { - final Map entriesUnderTxFound = new HashMap<>(); + Map entriesUnderTxFound = new HashMap<>(); try (WALIterator stIt = walIter) { while (stIt.hasNextX()) { - final IgniteBiTuple next = stIt.nextX(); - final WALRecord walRecord = next.get2(); + IgniteBiTuple tup = stIt.nextX(); - if (walRecord.type() == WALRecord.RecordType.DATA_RECORD && walRecord instanceof DataRecord) { - final DataRecord dataRecord = (DataRecord)walRecord; + WALRecord walRecord = tup.get2(); + + if (walRecord.type() == DATA_RECORD && walRecord instanceof DataRecord) { + DataRecord dataRecord = (DataRecord)walRecord; if (dataRecordHnd != null) dataRecordHnd.apply(dataRecord); - final List entries = dataRecord.writeEntries(); + + List entries = dataRecord.writeEntries(); for (DataEntry entry : entries) { - final GridCacheVersion globalTxId = entry.nearXidVersion(); + GridCacheVersion globalTxId = entry.nearXidVersion(); + Object unwrappedKeyObj; Object unwrappedValObj; + if (entry instanceof UnwrapDataEntry) { UnwrapDataEntry unwrapDataEntry = (UnwrapDataEntry)entry; unwrappedKeyObj = unwrapDataEntry.unwrappedKey(); @@ -1116,7 +1133,7 @@ else if (entry instanceof LazyDataEntry) { unwrappedKeyObj = key instanceof BinaryObject ? key : key.value(null, false); } - if (dumpRecords) + if (DUMP_RECORDS) log.info("//Entry operation " + entry.op() + "; cache Id" + entry.cacheId() + "; " + "under transaction: " + globalTxId + //; entry " + entry + @@ -1126,23 +1143,78 @@ else if (entry instanceof LazyDataEntry) { if (cacheObjHnd != null && (unwrappedKeyObj != null || unwrappedValObj != null)) cacheObjHnd.apply(unwrappedKeyObj, unwrappedValObj); - final Integer entriesUnderTx = entriesUnderTxFound.get(globalTxId); + Integer entriesUnderTx = entriesUnderTxFound.get(globalTxId); + entriesUnderTxFound.put(globalTxId, entriesUnderTx == null ? 1 : entriesUnderTx + 1); } } - else if (walRecord.type() == WALRecord.RecordType.TX_RECORD && walRecord instanceof TxRecord) { - final TxRecord txRecord = (TxRecord)walRecord; - final GridCacheVersion globalTxId = txRecord.nearXidVersion(); + else if (walRecord.type() == TX_RECORD && walRecord instanceof TxRecord) { + TxRecord txRecord = (TxRecord)walRecord; + GridCacheVersion globalTxId = txRecord.nearXidVersion(); - if (dumpRecords) + if (DUMP_RECORDS) log.info("//Tx Record, state: " + txRecord.state() + "; nearTxVersion" + globalTxId); } } } + return entriesUnderTxFound; } + /** + * Puts provided number of records to fill WAL. + * + * @param ignite ignite instance. + * @param recordsToWrite count. + */ + private void putDummyRecords(Ignite ignite, int recordsToWrite) { + IgniteCache cache0 = ignite.cache(CACHE_NAME); + + for (int i = 0; i < recordsToWrite; i++) + cache0.put(i, new IndexedObject(i)); + } + + /** + * Puts provided number of records to fill WAL. + * + * @param ignite ignite instance. + * @param recordsToWrite count. + */ + private void putAllDummyRecords(Ignite ignite, int recordsToWrite) { + IgniteCache cache0 = ignite.cache(CACHE_NAME); + + Map values = new HashMap<>(); + + for (int i = 0; i < recordsToWrite; i++) + values.put(i, new IndexedObject(i)); + + cache0.putAll(values); + } + + /** + * Puts provided number of records to fill WAL under transactions. + * + * @param ignite ignite instance. + * @param recordsToWrite count. + * @param txCnt transactions to run. If number is less then records count, txCnt records will be written. + */ + private IgniteCache txPutDummyRecords(Ignite ignite, int recordsToWrite, int txCnt) { + IgniteCache cache0 = ignite.cache(CACHE_NAME); + int keysPerTx = recordsToWrite / txCnt; + if (keysPerTx == 0) + keysPerTx = 1; + for (int t = 0; t < txCnt; t++) { + try (Transaction tx = ignite.transactions().txStart()) { + for (int i = t * keysPerTx; i < (t + 1) * keysPerTx; i++) + cache0.put(i, new IndexedObject(i)); + + tx.commit(); + } + } + return cache0; + } + /** Enum for cover binaryObject enum save/load. */ enum TestEnum { /** */A, /** */B, /** */C @@ -1257,7 +1329,7 @@ static class TestStringContainerToBePrinted { * * @param data value to be searched in to String */ - public TestStringContainerToBePrinted(String data) { + TestStringContainerToBePrinted(String data) { this.data = data; } @@ -1297,7 +1369,7 @@ private static class Organization { * @param key Key. * @param name Name. */ - public Organization(int key, String name) { + Organization(int key, String name) { this.key = key; this.name = name; } 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 0188445463d32..0a240ea6ae36f 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 @@ -150,4 +150,9 @@ public class NoOpWALManager implements IgniteWriteAheadLogManager { @Override public int walArchiveSegments() { return 0; } + + /** {@inheritDoc} */ + @Override public long lastArchivedSegment() { + return -1L; + } } 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 316ff92fcdab2..7e126b9261083 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 @@ -51,6 +51,7 @@ import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushLogOnlyWithMmapBufferSelfTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFormatFileFailoverTest; 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.IgniteWalSerializerVersionTest; import org.apache.ignite.internal.processors.cache.persistence.db.wal.WalCompactionTest; @@ -170,5 +171,7 @@ public static void addRealPageStoreTests(TestSuite suite) { suite.addTestSuite(LocalWalModeChangeDuringRebalancingSelfTest.class); suite.addTestSuite(IgniteWalIteratorSwitchSegmentTest.class); + + suite.addTestSuite(IgniteWalIteratorExceptionDuringReadTest.class); } } 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 2da1aa30fadcb..eee193ab34965 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 @@ -54,11 +54,7 @@ public static void main(String[] args) throws Exception { boolean printRecords = IgniteSystemProperties.getBoolean("PRINT_RECORDS", false); boolean printStat = IgniteSystemProperties.getBoolean("PRINT_STAT", true); - final IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(new NullLogger(), - Integer.parseInt(args[0]), - null, - null, - false); + final IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(new NullLogger()); final File walWorkDirWithConsistentId = new File(args[1]); @@ -69,7 +65,7 @@ public static void main(String[] args) throws Exception { @Nullable final WalStat stat = printStat ? new WalStat() : null; - try (WALIterator stIt = factory.iteratorWorkFiles(workFiles)) { + try (WALIterator stIt = factory.iterator(workFiles)) { while (stIt.hasNextX()) { IgniteBiTuple next = stIt.nextX(); @@ -87,7 +83,7 @@ public static void main(String[] args) throws Exception { if (args.length >= 3) { final File walArchiveDirWithConsistentId = new File(args[2]); - try (WALIterator stIt = factory.iteratorArchiveDirectory(walArchiveDirWithConsistentId)) { + try (WALIterator stIt = factory.iterator(walArchiveDirWithConsistentId)) { while (stIt.hasNextX()) { IgniteBiTuple next = stIt.nextX(); From 10632434b3af95fdadf67f4678f6e3efa0af84bd Mon Sep 17 00:00:00 2001 From: Sergey Chugunov Date: Thu, 28 Jun 2018 17:29:20 +0300 Subject: [PATCH 0298/1463] IGNITE-8857 HashMap is returned instead of filtering wrapper. - Fixes #4258. Signed-off-by: Dmitriy Pavlov (cherry-picked from commit #e1a3398b509a16f0db3b60751d0ef52e1b763a87) --- .../zk/internal/ZookeeperClusterNode.java | 4 ++-- .../internal/ZookeeperDiscoverySpiTest.java | 22 +++++++++++++++++++ 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java index 2fe3052fa9d23..02e412326f3a9 100644 --- a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java @@ -243,11 +243,11 @@ public void setCacheMetrics(Map cacheMetrics) { /** {@inheritDoc} */ @Override public Map attributes() { // Even though discovery SPI removes this attribute after authentication, keep this check for safety. - return F.view(attrs, new IgnitePredicate() { + return new HashMap<>(F.view(attrs, new IgnitePredicate() { @Override public boolean apply(String s) { return !IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS.equals(s); } - }); + })); } /** {@inheritDoc} */ 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 1585870053b87..92706bf98220d 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 @@ -470,6 +470,28 @@ private void checkInternalStructuresCleanup() throws Exception { } } + /** + * Verifies that node attributes returned through public API are presented in standard form. + * + * It means there is no exotic classes that may unnecessary capture other classes from the context. + * + * For more information about the problem refer to + * IGNITE-8857. + */ + public void testNodeAttributesNotReferencingZookeeperClusterNode() throws Exception { + userAttrs = new HashMap<>(); + userAttrs.put("testAttr", "testAttr"); + + try { + IgniteEx ignite = startGrid(0); + + assertTrue(ignite.cluster().localNode().attributes() instanceof HashMap); + } + finally { + userAttrs = null; + } + } + /** * @throws Exception If failed. */ From bd0b0db7d16ad5d5677b43ea111afe099633f1d9 Mon Sep 17 00:00:00 2001 From: Sergey Chugunov Date: Mon, 2 Jul 2018 14:36:25 +0300 Subject: [PATCH 0299/1463] IGNITE-8857 new IgnitePredicate filtering credential attribute introduced, HashMap was removed - Fixes #4272. Signed-off-by: Alexey Goncharuk (cherry-picked from commit #a67b08cc0e76dfd1ee3810972bfe6781698550d4) --- ...ecurityCredentialsAttrFilterPredicate.java | 39 +++++++++++++++++++ .../zk/internal/ZookeeperClusterNode.java | 8 +--- .../internal/ZookeeperDiscoverySpiTest.java | 14 ++++++- 3 files changed, 54 insertions(+), 7 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/SecurityCredentialsAttrFilterPredicate.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/SecurityCredentialsAttrFilterPredicate.java b/modules/core/src/main/java/org/apache/ignite/internal/SecurityCredentialsAttrFilterPredicate.java new file mode 100644 index 0000000000000..2f774bdcf4d32 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/SecurityCredentialsAttrFilterPredicate.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.internal; + +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgnitePredicate; + +/** + * Predicate to filter out security credentials attribute by its name. + */ +public class SecurityCredentialsAttrFilterPredicate implements IgnitePredicate { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public boolean apply(String s) { + return !IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS.equals(s); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(SecurityCredentialsAttrFilterPredicate.class, this); + } +} diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java index 02e412326f3a9..1c2a589d934dc 100644 --- a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperClusterNode.java @@ -31,11 +31,11 @@ import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.internal.ClusterMetricsSnapshot; import org.apache.ignite.internal.IgniteNodeAttributes; +import org.apache.ignite.internal.SecurityCredentialsAttrFilterPredicate; import org.apache.ignite.internal.managers.discovery.IgniteClusterNode; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.spi.discovery.DiscoveryMetricsProvider; import org.jetbrains.annotations.Nullable; @@ -243,11 +243,7 @@ public void setCacheMetrics(Map cacheMetrics) { /** {@inheritDoc} */ @Override public Map attributes() { // Even though discovery SPI removes this attribute after authentication, keep this check for safety. - return new HashMap<>(F.view(attrs, new IgnitePredicate() { - @Override public boolean apply(String s) { - return !IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS.equals(s); - } - })); + return F.view(attrs, new SecurityCredentialsAttrFilterPredicate()); } /** {@inheritDoc} */ 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 92706bf98220d..fbacebfd5dff7 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 @@ -74,6 +74,7 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.IgnitionEx; +import org.apache.ignite.internal.SecurityCredentialsAttrFilterPredicate; import org.apache.ignite.internal.TestRecordingCommunicationSpi; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.managers.discovery.CustomEventListener; @@ -91,6 +92,7 @@ import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl; import org.apache.ignite.internal.util.lang.GridAbsPredicate; import org.apache.ignite.internal.util.lang.IgniteInClosure2X; +import org.apache.ignite.internal.util.lang.gridfunc.PredicateMapView; import org.apache.ignite.internal.util.nio.GridCommunicationClient; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.T3; @@ -485,7 +487,17 @@ public void testNodeAttributesNotReferencingZookeeperClusterNode() throws Except try { IgniteEx ignite = startGrid(0); - assertTrue(ignite.cluster().localNode().attributes() instanceof HashMap); + Map attrs = ignite.cluster().localNode().attributes(); + + assertTrue(attrs instanceof PredicateMapView); + + IgnitePredicate[] preds = GridTestUtils.getFieldValue(attrs, "preds"); + + assertNotNull(preds); + + assertTrue(preds.length == 1); + + assertTrue(preds[0] instanceof SecurityCredentialsAttrFilterPredicate); } finally { userAttrs = null; From aeb66e55d736f827e9cf5271304dd9b242f1bc92 Mon Sep 17 00:00:00 2001 From: Aleksey Plekhanov Date: Mon, 2 Jul 2018 14:47:13 +0300 Subject: [PATCH 0300/1463] IGNITE-8203 Handle ClosedByInterruptionException in FilePageStore - Fixes #4211. --- .../cache/persistence/file/FilePageStore.java | 341 ++++++++++++----- .../IgnitePdsTaskCancelingTest.java | 352 ++++++++++++++++++ .../testsuites/IgnitePdsTestSuite2.java | 3 + 3 files changed, 610 insertions(+), 86 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsTaskCancelingTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java index f614032c58b15..2d9e37f3b50a9 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java @@ -21,6 +21,8 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.nio.channels.ClosedByInterruptException; +import java.nio.channels.ClosedChannelException; import java.nio.file.Files; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReadWriteLock; @@ -67,7 +69,7 @@ public class FilePageStore implements PageStore { private final FileIOFactory ioFactory; /** I/O interface for read/write operations with file */ - private FileIO fileIO; + private volatile FileIO fileIO; /** */ private final AtomicLong allocated; @@ -159,74 +161,77 @@ public ByteBuffer header(byte type, int pageSize) { * @return Next available position in the file to store a data. * @throws IOException If initialization is failed. */ - private long initFile() throws IOException { - ByteBuffer hdr = header(type, dbCfg.getPageSize()); + private long initFile(FileIO fileIO) throws IOException { + try { + ByteBuffer hdr = header(type, dbCfg.getPageSize()); - while (hdr.remaining() > 0) - fileIO.write(hdr); + while (hdr.remaining() > 0) + fileIO.write(hdr); + + //there is 'super' page in every file + return headerSize() + dbCfg.getPageSize(); + } + catch (ClosedByInterruptException e) { + // If thread was interrupted written header can be inconsistent. + Files.delete(cfgFile.toPath()); - //there is 'super' page in every file - return headerSize() + dbCfg.getPageSize(); + throw e; + } } /** * Checks that file store has correct header and size. * * @return Next available position in the file to store a data. - * @throws PersistentStorageIOException If check is failed. + * @throws IOException If check is failed. */ - private long checkFile() throws PersistentStorageIOException { - try { - ByteBuffer hdr = ByteBuffer.allocate(headerSize()).order(ByteOrder.LITTLE_ENDIAN); + private long checkFile(FileIO fileIO) throws IOException { + ByteBuffer hdr = ByteBuffer.allocate(headerSize()).order(ByteOrder.LITTLE_ENDIAN); - while (hdr.remaining() > 0) - fileIO.read(hdr); + while (hdr.remaining() > 0) + fileIO.read(hdr); - hdr.rewind(); + hdr.rewind(); - long signature = hdr.getLong(); + long signature = hdr.getLong(); - if (SIGNATURE != signature) - throw new IOException("Failed to verify store file (invalid file signature)" + - " [expectedSignature=" + U.hexLong(SIGNATURE) + - ", actualSignature=" + U.hexLong(signature) + ']'); + if (SIGNATURE != signature) + throw new IOException("Failed to verify store file (invalid file signature)" + + " [expectedSignature=" + U.hexLong(SIGNATURE) + + ", actualSignature=" + U.hexLong(signature) + ']'); - int ver = hdr.getInt(); + int ver = hdr.getInt(); - if (version() != ver) - throw new IOException("Failed to verify store file (invalid file version)" + - " [expectedVersion=" + version() + - ", fileVersion=" + ver + "]"); + if (version() != ver) + throw new IOException("Failed to verify store file (invalid file version)" + + " [expectedVersion=" + version() + + ", fileVersion=" + ver + "]"); - byte type = hdr.get(); + byte type = hdr.get(); - if (this.type != type) - throw new IOException("Failed to verify store file (invalid file type)" + - " [expectedFileType=" + this.type + - ", actualFileType=" + type + "]"); + if (this.type != type) + throw new IOException("Failed to verify store file (invalid file type)" + + " [expectedFileType=" + this.type + + ", actualFileType=" + type + "]"); - int pageSize = hdr.getInt(); + int pageSize = hdr.getInt(); - if (dbCfg.getPageSize() != pageSize) - throw new IOException("Failed to verify store file (invalid page size)" + - " [expectedPageSize=" + dbCfg.getPageSize() + - ", filePageSize=" + pageSize + "]"); + if (dbCfg.getPageSize() != pageSize) + throw new IOException("Failed to verify store file (invalid page size)" + + " [expectedPageSize=" + dbCfg.getPageSize() + + ", filePageSize=" + pageSize + "]"); - long fileSize = cfgFile.length(); + long fileSize = cfgFile.length(); - if (fileSize == headerSize()) // Every file has a special meta page. - fileSize = pageSize + headerSize(); + if (fileSize == headerSize()) // Every file has a special meta page. + fileSize = pageSize + headerSize(); - if ((fileSize - headerSize()) % pageSize != 0) - throw new IOException("Failed to verify store file (invalid file size)" + - " [fileSize=" + U.hexLong(fileSize) + - ", pageSize=" + U.hexLong(pageSize) + ']'); + if ((fileSize - headerSize()) % pageSize != 0) + throw new IOException("Failed to verify store file (invalid file size)" + + " [fileSize=" + U.hexLong(fileSize) + + ", pageSize=" + U.hexLong(pageSize) + ']'); - return fileSize; - } - catch (IOException e) { - throw new PersistentStorageIOException("File check failed", e); - } + return fileSize; } /** @@ -244,6 +249,8 @@ public void stop(boolean cleanFile) throws PersistentStorageIOException { fileIO.close(); + fileIO = null; + if (cleanFile) Files.delete(cfgFile.toPath()); } @@ -273,6 +280,8 @@ public void truncate(int tag) throws PersistentStorageIOException { fileIO.close(); + fileIO = null; + Files.delete(cfgFile.toPath()); } catch (IOException e) { @@ -348,7 +357,7 @@ public void finishRecover() throws PersistentStorageIOException { int len = pageSize; do { - int n = fileIO.read(pageBuf, off); + int n = readWithFailover(pageBuf, off); // If page was not written yet, nothing to read. if (n < 0) { @@ -403,7 +412,7 @@ public void finishRecover() throws PersistentStorageIOException { long off = 0; do { - int n = fileIO.read(buf, off); + int n = readWithFailover(buf, off); // If page was not written yet, nothing to read. if (n < 0) @@ -433,10 +442,28 @@ private void init() throws PersistentStorageIOException { PersistentStorageIOException err = null; + long newSize; + try { - this.fileIO = fileIO = ioFactory.create(cfgFile, CREATE, READ, WRITE); + boolean interrupted = false; + + while (true) { + try { + this.fileIO = fileIO = ioFactory.create(cfgFile, CREATE, READ, WRITE); + + newSize = cfgFile.length() == 0 ? initFile(fileIO) : checkFile(fileIO); - long newSize = cfgFile.length() == 0 ? initFile() : checkFile(); + if (interrupted) + Thread.currentThread().interrupt(); + + break; + } + catch (ClosedByInterruptException e) { + interrupted = true; + + Thread.interrupted(); + } + } assert allocated.get() == 0; @@ -469,59 +496,158 @@ private void init() throws PersistentStorageIOException { } } - /** {@inheritDoc} */ - @Override public void write(long pageId, ByteBuffer pageBuf, int tag, boolean calculateCrc) throws IgniteCheckedException { - init(); + /** + * Reinit page store after file channel was closed by thread interruption. + * + * @param fileIO Old fileIO. + */ + private void reinit(FileIO fileIO) throws IOException { + if (!inited) + return; - lock.readLock().lock(); + if (fileIO != this.fileIO) + return; + + lock.writeLock().lock(); try { - if (tag < this.tag) + if (fileIO != this.fileIO) return; - long off = pageOffset(pageId); + try { + boolean interrupted = false; - assert (off >= 0 && off + headerSize() <= allocated.get() ) || recover : - "off=" + U.hexLong(off) + ", allocated=" + U.hexLong(allocated.get()) + ", pageId=" + U.hexLong(pageId); + while (true) { + try { + fileIO = null; - assert pageBuf.capacity() == pageSize; - assert pageBuf.position() == 0; - assert pageBuf.order() == ByteOrder.nativeOrder() : "Page buffer order " + pageBuf.order() - + " should be same with " + ByteOrder.nativeOrder(); - assert PageIO.getType(pageBuf) != 0 : "Invalid state. Type is 0! pageId = " + U.hexLong(pageId); - assert PageIO.getVersion(pageBuf) != 0 : "Invalid state. Version is 0! pageId = " + U.hexLong(pageId); + fileIO = ioFactory.create(cfgFile, CREATE, READ, WRITE); + + checkFile(fileIO); - if (calculateCrc && !skipCrc) { - assert PageIO.getCrc(pageBuf) == 0 : U.hexLong(pageId); + this.fileIO = fileIO; - PageIO.setCrc(pageBuf, calcCrc32(pageBuf, pageSize)); + if (interrupted) + Thread.currentThread().interrupt(); + + break; + } + catch (ClosedByInterruptException e) { + interrupted = true; + + Thread.interrupted(); + } + } } + catch (IOException e) { + try { + if (fileIO != null) + fileIO.close(); + } + catch (IOException e0) { + e.addSuppressed(e0); + } - // Check whether crc was calculated somewhere above the stack if it is forcibly skipped. - assert skipCrc || PageIO.getCrc(pageBuf) != 0 || calcCrc32(pageBuf, pageSize) == 0 : - "CRC hasn't been calculated, crc=0"; + throw e; + } + } + finally { + lock.writeLock().unlock(); + } + } - assert pageBuf.position() == 0 : pageBuf.position(); + /** {@inheritDoc} */ + @Override public void write(long pageId, ByteBuffer pageBuf, int tag, boolean calculateCrc) throws IgniteCheckedException { + init(); - int len = pageSize; + boolean interrupted = false; - do { - int n = fileIO.write(pageBuf, off); + while (true) { + FileIO fileIO = this.fileIO; - off += n; + try { + lock.readLock().lock(); - len -= n; + try { + if (tag < this.tag) + return; + + long off = pageOffset(pageId); + + assert (off >= 0 && off + headerSize() <= allocated.get()) || recover : + "off=" + U.hexLong(off) + ", allocated=" + U.hexLong(allocated.get()) + ", pageId=" + U.hexLong(pageId); + + assert pageBuf.capacity() == pageSize; + assert pageBuf.position() == 0; + assert pageBuf.order() == ByteOrder.nativeOrder() : "Page buffer order " + pageBuf.order() + + " should be same with " + ByteOrder.nativeOrder(); + assert PageIO.getType(pageBuf) != 0 : "Invalid state. Type is 0! pageId = " + U.hexLong(pageId); + assert PageIO.getVersion(pageBuf) != 0 : "Invalid state. Version is 0! pageId = " + U.hexLong(pageId); + + if (calculateCrc && !skipCrc) { + assert PageIO.getCrc(pageBuf) == 0 : U.hexLong(pageId); + + PageIO.setCrc(pageBuf, calcCrc32(pageBuf, pageSize)); + } + + // Check whether crc was calculated somewhere above the stack if it is forcibly skipped. + assert skipCrc || PageIO.getCrc(pageBuf) != 0 || calcCrc32(pageBuf, pageSize) == 0 : + "CRC hasn't been calculated, crc=0"; + + assert pageBuf.position() == 0 : pageBuf.position(); + + int len = pageSize; + + if (fileIO == null) + throw new IOException("FileIO has stopped"); + + do { + int n = fileIO.write(pageBuf, off); + + off += n; + + len -= n; + } + while (len > 0); + + PageIO.setCrc(pageBuf, 0); + + if (interrupted) + Thread.currentThread().interrupt(); + + return; + } + finally { + lock.readLock().unlock(); + } } - while (len > 0); + catch (IOException e) { + if (e instanceof ClosedChannelException) { + try { + if (e instanceof ClosedByInterruptException) { + interrupted = true; - PageIO.setCrc(pageBuf, 0); - } - catch (IOException e) { - throw new PersistentStorageIOException("Failed to write the page to the file store [pageId=" + pageId + - ", file=" + cfgFile.getAbsolutePath() + ']', e); - } - finally { - lock.readLock().unlock(); + Thread.interrupted(); + } + + reinit(fileIO); + + pageBuf.position(0); + + PageIO.setCrc(pageBuf, 0); + + continue; + } + catch (IOException e0) { + e0.addSuppressed(e); + + e = e0; + } + } + + throw new PersistentStorageIOException("Failed to write the page to the file store [pageId=" + pageId + + ", file=" + cfgFile.getAbsolutePath() + ']', e); + } } } @@ -552,7 +678,10 @@ private static int calcCrc32(ByteBuffer pageBuf, int pageSize) { try { init(); - fileIO.force(); + FileIO fileIO = this.fileIO; + + if (fileIO != null) + fileIO.force(); } catch (IOException e) { throw new PersistentStorageIOException("Sync error", e); @@ -603,4 +732,44 @@ private long allocPage() { return (int)((allocated.get() - headerSize()) / pageSize); } + + /** + * @param destBuf Destination buffer. + * @param position Position. + * @return Number of read bytes. + */ + private int readWithFailover(ByteBuffer destBuf, long position) throws IOException { + boolean interrupted = false; + + int bufPos = destBuf.position(); + + while (true) { + FileIO fileIO = this.fileIO; + + if (fileIO == null) + throw new IOException("FileIO has stopped"); + + try { + assert destBuf.remaining() > 0; + + int bytesRead = fileIO.read(destBuf, position); + + if (interrupted) + Thread.currentThread().interrupt(); + + return bytesRead; + } + catch (ClosedChannelException e) { + destBuf.position(bufPos); + + if (e instanceof ClosedByInterruptException) { + interrupted = true; + + Thread.interrupted(); + } + + reinit(fileIO); + } + } + } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsTaskCancelingTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsTaskCancelingTest.java new file mode 100644 index 0000000000000..d42b7885add49 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsTaskCancelingTest.java @@ -0,0 +1,352 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.persistence; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.OpenOption; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Random; +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.LockSupport; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteCountDownLatch; +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.failure.FailureContext; +import org.apache.ignite.failure.FailureHandler; +import org.apache.ignite.internal.pagemem.PageIdUtils; +import org.apache.ignite.internal.pagemem.PageMemory; +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.FilePageStore; +import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; +import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO; +import org.apache.ignite.internal.util.GridUnsafe; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteRunnable; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static java.nio.file.StandardOpenOption.CREATE; +import static java.nio.file.StandardOpenOption.READ; +import static java.nio.file.StandardOpenOption.WRITE; + +/** + * Test handle of task canceling with PDS enabled. + */ +public class IgnitePdsTaskCancelingTest extends GridCommonAbstractTest { + /** Slow file IO enabled. */ + private static final AtomicBoolean slowFileIoEnabled = new AtomicBoolean(false); + + /** Node failure occurs. */ + private static final AtomicBoolean failure = new AtomicBoolean(false); + + /** Number of executing tasks. */ + private static final int NUM_TASKS = 16; + + /** Page size. */ + private static final int PAGE_SIZE = 2048; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setFailureHandler(new FailureHandler() { + @Override public boolean onFailure(Ignite ignite, FailureContext failureCtx) { + failure.set(true); + + return true; + } + }); + + cfg.setCacheConfiguration(new CacheConfiguration().setName(DEFAULT_CACHE_NAME).setAffinity( + new RendezvousAffinityFunction(false, NUM_TASKS / 2) + )); + + cfg.setDataStorageConfiguration(getDataStorageConfiguration()); + + return cfg; + } + + /** + * Default data storage configuration. + */ + private DataStorageConfiguration getDataStorageConfiguration() { + DataStorageConfiguration dbCfg = new DataStorageConfiguration(); + + dbCfg.setPageSize(PAGE_SIZE); + + dbCfg.setFileIOFactory(new SlowIOFactory()); + + dbCfg.setDefaultDataRegionConfiguration(new DataRegionConfiguration() + .setMaxSize(100 * 1024 * 1024) + .setPersistenceEnabled(true)); + + return dbCfg; + } + + /** + * Checks that tasks canceling does not lead to node failure. + */ + public void testFailNodesOnCanceledTask() throws Exception { + cleanPersistenceDir(); + + failure.set(false); + + try { + Ignite ig0 = startGrids(4); + + ig0.cluster().active(true); + + Collection cancelFutures = new ArrayList<>(NUM_TASKS); + + IgniteCountDownLatch latch = ig0.countDownLatch("latch", NUM_TASKS, false, true); + + for (int i = 0; i < NUM_TASKS; i++) { + final Integer key = i; + + cancelFutures.add(ig0.compute().affinityRunAsync(DEFAULT_CACHE_NAME, key, + new IgniteRunnable() { + @IgniteInstanceResource + Ignite ig; + + @Override public void run() { + latch.countDown(); + + latch.await(); + + ig.cache(DEFAULT_CACHE_NAME).put(key, new byte[1024]); + } + })); + } + + slowFileIoEnabled.set(true); + + latch.await(); + + for (IgniteFuture future: cancelFutures) + future.cancel(); + + slowFileIoEnabled.set(false); + + for (int i = 0; i < NUM_TASKS; i++) { + final Integer key = i; + + ig0.compute().affinityRun(DEFAULT_CACHE_NAME, key, + new IgniteRunnable() { + @IgniteInstanceResource + Ignite ig; + + @Override public void run() { + ig.cache(DEFAULT_CACHE_NAME).put(key, new byte[1024]); + } + }); + } + + assertFalse(GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return failure.get(); + } + }, 5_000L)); + } + finally { + stopAllGrids(); + + cleanPersistenceDir(); + } + } + + /** + * Test FilePageStore with multiple interrupted threads. + */ + public void testFilePageStoreInterruptThreads() throws Exception { + failure.set(false); + + FileIOFactory factory = new RandomAccessFileIOFactory(); + + File file = new File(U.defaultWorkDirectory(), "file.bin"); + + file.deleteOnExit(); + + DataStorageConfiguration dbCfg = getDataStorageConfiguration(); + + FilePageStore pageStore = new FilePageStore(PageMemory.FLAG_DATA, file, factory, dbCfg, + new AllocatedPageTracker() { + @Override public void updateTotalAllocatedPages(long delta) { + // No-op. + } + }); + + int pageSize = dbCfg.getPageSize(); + + PageIO pageIO = PageIO.getPageIO(PageIO.T_DATA, 1); + + long ptr = GridUnsafe.allocateMemory(NUM_TASKS * pageSize); + + try { + List threadList = new ArrayList<>(NUM_TASKS); + + AtomicBoolean stopThreads = new AtomicBoolean(false); + + for (int i = 0; i < NUM_TASKS; i++) { + long pageId = PageIdUtils.pageId(0, PageMemory.FLAG_DATA, (int)pageStore.allocatePage()); + + long pageAdr = ptr + i * pageSize; + + pageIO.initNewPage(pageAdr, pageId, pageSize); + + ByteBuffer buf = GridUnsafe.wrapPointer(pageAdr, pageSize); + + pageStore.write(pageId, buf, 0, true); + + threadList.add(new Thread(new Runnable() { + @Override public void run() { + Random random = new Random(); + + while (!stopThreads.get()) { + buf.position(0); + + try { + if (random.nextBoolean()) { + log.info(">>> Read page " + U.hexLong(pageId)); + + pageStore.read(pageId, buf, false); + } + else { + log.info(">>> Write page " + U.hexLong(pageId)); + + pageStore.write(pageId, buf, 0, true); + } + + Thread.interrupted(); + } + catch (Exception e) { + log.error("Error while reading/writing page", e); + + failure.set(true); + } + } + } + })); + } + + for (Thread thread : threadList) + thread.start(); + + for (int i = 0; i < 10; i++) { + for (Thread thread : threadList) { + doSleep(10L); + + log.info("Interrupting " + thread.getName()); + + thread.interrupt(); + } + } + + stopThreads.set(true); + + for (Thread thread : threadList) + thread.join(); + + assertFalse(failure.get()); + } + finally { + GridUnsafe.freeMemory(ptr); + } + } + + /** + * Decorated FileIOFactory with slow IO operations. + */ + private static class SlowIOFactory implements FileIOFactory { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private final FileIOFactory delegateFactory = new RandomAccessFileIOFactory(); + + /** {@inheritDoc} */ + @Override public FileIO create(File file) throws IOException { + return create(file, CREATE, READ, WRITE); + } + + /** {@inheritDoc} */ + @Override public FileIO create(File file, OpenOption... openOption) throws IOException { + final FileIO delegate = delegateFactory.create(file, openOption); + + final boolean slow = file.getName().contains(".bin"); + + return new FileIODecorator(delegate) { + @Override public int write(ByteBuffer srcBuf) throws IOException { + parkForAWhile(); + + return super.write(srcBuf); + } + + @Override public int write(ByteBuffer srcBuf, long position) throws IOException { + parkForAWhile(); + + return super.write(srcBuf, position); + } + + @Override public int write(byte[] buf, int off, int len) throws IOException { + parkForAWhile(); + + return super.write(buf, off, len); + } + + @Override public int read(ByteBuffer destBuf) throws IOException { + parkForAWhile(); + + return super.read(destBuf); + } + + @Override public int read(ByteBuffer destBuf, long position) throws IOException { + parkForAWhile(); + + return super.read(destBuf, position); + } + + @Override public int read(byte[] buf, int off, int len) throws IOException { + parkForAWhile(); + + return super.read(buf, off, len); + } + + private void parkForAWhile() { + if(slowFileIoEnabled.get() && slow) + LockSupport.parkNanos(1_000_000_000L); + } + }; + } + } +} 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 7e126b9261083..e7609a36f8198 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,6 +27,7 @@ import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsPageSizesTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsPartitionFilesDestroyTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsRecoveryAfterFileCorruptionTest; +import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsTaskCancelingTest; import org.apache.ignite.internal.processors.cache.persistence.IgnitePersistentStoreDataStructuresTest; import org.apache.ignite.internal.processors.cache.persistence.LocalWalModeChangeDuringRebalancingSelfTest; import org.apache.ignite.internal.processors.cache.persistence.baseline.IgniteAbsentEvictionNodeOutOfBaselineTest; @@ -82,6 +83,8 @@ public static TestSuite suite() { suite.addTestSuite(ClientAffinityAssignmentWithBaselineTest.class); suite.addTestSuite(IgniteAbsentEvictionNodeOutOfBaselineTest.class); + suite.addTestSuite(IgnitePdsTaskCancelingTest.class); + return suite; } From b797cc9fc7518b612277931baa355ef732b627ec Mon Sep 17 00:00:00 2001 From: Dmitriy Sorokin Date: Wed, 4 Jul 2018 13:16:21 +0300 Subject: [PATCH 0301/1463] IGNITE-8910 PagesList.takeEmptyPage may fail with AssertionError: type = 1 - Fixes #4294. Signed-off-by: Ivan Rakov (cherry picked from commit e6f44ad) --- .../cache/persistence/freelist/PagesList.java | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java index 78dc91f512c57..f8400ca7430fa 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java @@ -469,8 +469,12 @@ private boolean updateTail(int bucket, long oldTailId, long newTailId) { else newTails = null; // Drop the bucket completely. - if (casBucket(bucket, tails, newTails)) + if (casBucket(bucket, tails, newTails)) { + // Reset tailId for invalidation of locking when stripe was taken concurrently. + tails[idx].tailId = 0L; + return true; + } } else { // It is safe to assign new tail since we do it only when write lock on tail is held. @@ -622,6 +626,11 @@ protected final void put( return; final long tailId = stripe.tailId; + + // Stripe was removed from bucket concurrently. + if (tailId == 0L) + continue; + final long tailPage = acquirePage(tailId); try { @@ -1035,6 +1044,11 @@ protected final long takeEmptyPage(int bucket, @Nullable IOVersions initIoVers) return 0L; final long tailId = stripe.tailId; + + // Stripe was removed from bucket concurrently. + if (tailId == 0L) + continue; + final long tailPage = acquirePage(tailId); try { From 7e966ae8b05b59547be8287ba966305d7fb6084c Mon Sep 17 00:00:00 2001 From: Alexey Stelmak Date: Wed, 4 Jul 2018 17:15:28 +0300 Subject: [PATCH 0302/1463] IGNITE-8780 File I/O operations must be retried if buffer hasn't read/written completely Signed-off-by: Andrey Gura --- .../GridCacheDatabaseSharedManager.java | 19 +- .../persistence/file/AbstractFileIO.java | 146 +++++++++ .../cache/persistence/file/AsyncFileIO.java | 2 +- .../cache/persistence/file/FileIO.java | 76 +++++ .../persistence/file/FileIODecorator.java | 2 +- .../cache/persistence/file/FilePageStore.java | 20 +- .../file/FileVersionCheckingFactory.java | 3 +- .../persistence/file/RandomAccessFileIO.java | 2 +- .../cache/persistence/file/UnzipFileIO.java | 2 +- .../cache/persistence/wal/FileWALPointer.java | 3 + .../wal/FileWriteAheadLogManager.java | 46 +-- .../FsyncModeFileWriteAheadLogManager.java | 34 +- ...itePdsRecoveryAfterFileCorruptionTest.java | 2 +- ...alModeChangeDuringRebalancingSelfTest.java | 3 +- .../db/wal/crc/IgniteDataIntegrityTests.java | 6 +- .../file/AlignedBuffersDirectFileIO.java | 2 +- .../persistence/file/IgniteFileIOTest.java | 304 ++++++++++++++++++ 17 files changed, 572 insertions(+), 100 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AbstractFileIO.java create mode 100644 modules/direct-io/src/test/java/org/apache/ignite/internal/processors/cache/persistence/file/IgniteFileIOTest.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 48b0f34d155e4..5b6c1956f9557 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 @@ -537,7 +537,7 @@ private List retreiveHistory() throws IgniteCheckedException { ) { List checkpoints = new ArrayList<>(); - ByteBuffer buf = ByteBuffer.allocate(16); + ByteBuffer buf = ByteBuffer.allocate(FileWALPointer.POINTER_SIZE); buf.order(ByteOrder.nativeOrder()); for (Path cpFile : cpFiles) { @@ -875,7 +875,7 @@ private void nodeStart(WALPointer ptr) throws IgniteCheckedException { String fileName = U.currentTimeMillis() + NODE_STARTED_FILE_NAME_SUFFIX; String tmpFileName = fileName + FILE_TMP_SUFFIX; - ByteBuffer buf = ByteBuffer.allocate(20); + ByteBuffer buf = ByteBuffer.allocate(FileWALPointer.POINTER_SIZE); buf.order(ByteOrder.nativeOrder()); try { @@ -889,7 +889,7 @@ private void nodeStart(WALPointer ptr) throws IgniteCheckedException { buf.flip(); - io.write(buf); + io.writeFully(buf); buf.clear(); @@ -919,7 +919,7 @@ public List> nodeStartedPointers() throws IgniteCheckedExce cpDir.toPath(), path -> path.toFile().getName().endsWith(NODE_STARTED_FILE_NAME_SUFFIX)) ) { - ByteBuffer buf = ByteBuffer.allocate(20); + ByteBuffer buf = ByteBuffer.allocate(FileWALPointer.POINTER_SIZE); buf.order(ByteOrder.nativeOrder()); for (Path path : nodeStartedFiles) { @@ -930,7 +930,7 @@ public List> nodeStartedPointers() throws IgniteCheckedExce Long ts = Long.valueOf(name.substring(0, name.length() - NODE_STARTED_FILE_NAME_SUFFIX.length())); try (FileIO io = ioFactory.create(f, READ)) { - io.read(buf); + io.readFully(buf); buf.flip(); @@ -1216,8 +1216,7 @@ private int resolvePageSizeFromPartitionFile(Path partFile) throws IOException, ByteBuffer hdr = ByteBuffer.allocate(minimalHdr).order(ByteOrder.LITTLE_ENDIAN); - while (hdr.remaining() > 0) - fileIO.read(hdr); + fileIO.readFully(hdr); hdr.rewind(); @@ -1855,7 +1854,7 @@ else if (type == CheckpointEntryType.END && ts > lastEndTs) { } } - ByteBuffer buf = ByteBuffer.allocate(20); + ByteBuffer buf = ByteBuffer.allocate(FileWALPointer.POINTER_SIZE); buf.order(ByteOrder.nativeOrder()); if (startFile != null) @@ -1881,7 +1880,7 @@ private WALPointer readPointer(File cpMarkerFile, ByteBuffer buf) throws IgniteC buf.position(0); try (FileIO io = ioFactory.create(cpMarkerFile, READ)) { - io.read(buf); + io.readFully(buf); buf.flip(); @@ -2630,7 +2629,7 @@ public void writeCheckpointEntry(ByteBuffer entryBuf, CheckpointEntry cp, Checkp try (FileIO io = ioFactory.create(Paths.get(cpDir.getAbsolutePath(), skipSync ? fileName : tmpFileName).toFile(), StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE)) { - io.write(entryBuf); + io.writeFully(entryBuf); entryBuf.clear(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AbstractFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AbstractFileIO.java new file mode 100644 index 0000000000000..47236446b086f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AbstractFileIO.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.file; + +import java.io.EOFException; +import java.io.IOException; +import java.nio.ByteBuffer; +import org.apache.ignite.internal.util.typedef.internal.U; + +/** + * + */ +public abstract class AbstractFileIO implements FileIO { + /** Max io timeout milliseconds. */ + private static final int MAX_IO_TIMEOUT_MS = 2000; + + /** + * + */ + private interface IOOperation { + /** + * @param offs Offset. + * + * @return Number of bytes operated. + */ + public int run(int offs) throws IOException; + } + + /** + * @param operation IO operation. + * + * @param num Number of bytes to operate. + */ + private int fully(IOOperation operation, int num, boolean write) throws IOException { + if (num > 0) { + long time = 0; + + for (int i = 0; i < num; ) { + int n = operation.run(i); + + if (n > 0) { + i += n; + time = 0; + } + else if (n == 0) { + if (time == 0) + time = U.currentTimeMillis(); + else if ((U.currentTimeMillis() - time) >= MAX_IO_TIMEOUT_MS) + throw new IOException(write && position() == size() ? "Failed to extend file." : + "Probably disk is too busy, please check your device."); + } + else + throw new EOFException("EOF at position [" + position() + "] expected to read [" + num + "] bytes."); + } + } + + return num; + } + + /** {@inheritDoc} */ + @Override public int readFully(final ByteBuffer destBuf) throws IOException { + return fully(new IOOperation() { + @Override public int run(int offs) throws IOException { + return read(destBuf); + } + }, available(destBuf.remaining()), false); + } + + /** {@inheritDoc} */ + @Override public int readFully(final ByteBuffer destBuf, final long position) throws IOException { + return fully(new IOOperation() { + @Override public int run(int offs) throws IOException { + return read(destBuf, position + offs); + } + }, available(destBuf.remaining(), position), false); + } + + /** {@inheritDoc} */ + @Override public int readFully(final byte[] buf, final int off, final int len) throws IOException { + return fully(new IOOperation() { + @Override public int run(int offs) throws IOException { + return read(buf, off + offs, len - offs); + } + }, len, false); + } + + /** {@inheritDoc} */ + @Override public int writeFully(final ByteBuffer srcBuf) throws IOException { + return fully(new IOOperation() { + @Override public int run(int offs) throws IOException { + return write(srcBuf); + } + }, srcBuf.remaining(), true); + } + + /** {@inheritDoc} */ + @Override public int writeFully(final ByteBuffer srcBuf, final long position) throws IOException { + return fully(new IOOperation() { + @Override public int run(int offs) throws IOException { + return write(srcBuf, position + offs); + } + }, srcBuf.remaining(), true); + } + + /** {@inheritDoc} */ + @Override public int writeFully(final byte[] buf, final int off, final int len) throws IOException { + return fully(new IOOperation() { + @Override public int run(int offs) throws IOException { + return write(buf, off + offs, len - offs); + } + }, len, true); + } + + /** + * @param requested Requested. + */ + private int available(int requested) throws IOException { + return available(requested, position()); + } + + /** + * @param requested Requested. + * @param position Position. + */ + private int available(int requested, long position) throws IOException { + long avail = size() - position; + + return requested > avail ? (int) avail : requested; + } + +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java index 76142bb509626..fd00e255fc75f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIO.java @@ -31,7 +31,7 @@ /** * File I/O implementation based on {@link AsynchronousFileChannel}. */ -public class AsyncFileIO implements FileIO { +public class AsyncFileIO extends AbstractFileIO { /** * File channel associated with {@code file} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java index 50568aff0f14e..6f32d015ae410 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java @@ -59,6 +59,17 @@ public interface FileIO extends AutoCloseable { */ public int read(ByteBuffer destBuf) throws IOException; + /** + * Reads a sequence of bytes from this file into the {@code destinationBuffer}. + * + * @param destBuf Destination byte buffer. + * + * @return Number of written bytes. + * + * @throws IOException If some I/O error occurs. + */ + public int readFully(ByteBuffer destBuf) throws IOException; + /** * Reads a sequence of bytes from this file into the {@code destinationBuffer} * starting from specified file {@code position}. @@ -74,6 +85,19 @@ public interface FileIO extends AutoCloseable { */ public int read(ByteBuffer destBuf, long position) throws IOException; + /** + * Reads a sequence of bytes from this file into the {@code destinationBuffer} + * starting from specified file {@code position}. + * + * @param destBuf Destination byte buffer. + * @param position Starting position of file. + * + * @return Number of written bytes. + * + * @throws IOException If some I/O error occurs. + */ + public int readFully(ByteBuffer destBuf, long position) throws IOException; + /** * Reads a up to {@code length} bytes from this file into the {@code buffer}. * @@ -88,6 +112,20 @@ public interface FileIO extends AutoCloseable { */ public int read(byte[] buf, int off, int len) throws IOException; + /** + * Reads a up to {@code length} bytes from this file into the {@code buffer}. + * + * @param buf Destination byte array. + * @param off The start offset in array {@code b} + * at which the data is written. + * @param len Number of bytes read. + * + * @return Number of written bytes. + * + * @throws IOException If some I/O error occurs. + */ + public int readFully(byte[] buf, int off, int len) throws IOException; + /** * Writes a sequence of bytes to this file from the {@code sourceBuffer}. * @@ -99,6 +137,17 @@ public interface FileIO extends AutoCloseable { */ public int write(ByteBuffer srcBuf) throws IOException; + /** + * Writes a sequence of bytes to this file from the {@code sourceBuffer}. + * + * @param srcBuf Source buffer. + * + * @return Number of written bytes. + * + * @throws IOException If some I/O error occurs. + */ + public int writeFully(ByteBuffer srcBuf) throws IOException; + /** * Writes a sequence of bytes to this file from the {@code sourceBuffer} * starting from specified file {@code position} @@ -112,6 +161,19 @@ public interface FileIO extends AutoCloseable { */ public int write(ByteBuffer srcBuf, long position) throws IOException; + /** + * Writes a sequence of bytes to this file from the {@code sourceBuffer} + * starting from specified file {@code position} + * + * @param srcBuf Source buffer. + * @param position Starting file position. + * + * @return Number of written bytes. + * + * @throws IOException If some I/O error occurs. + */ + public int writeFully(ByteBuffer srcBuf, long position) throws IOException; + /** * Writes {@code length} bytes from the {@code buffer} * starting at offset {@code off} to this file. @@ -126,6 +188,20 @@ public interface FileIO extends AutoCloseable { */ public int write(byte[] buf, int off, int len) throws IOException; + /** + * Writes {@code length} bytes from the {@code buffer} + * starting at offset {@code off} to this file. + * + * @param buf Source byte array. + * @param off Start offset in the {@code buffer}. + * @param len Number of bytes to write. + * + * @return Number of written bytes. + * + * @throws IOException If some I/O error occurs. + */ + public int writeFully(byte[] buf, int off, int len) throws IOException; + /** * Allocates memory mapped buffer for this file with given size. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java index 9c389851ef248..8e79b54accbb5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java @@ -24,7 +24,7 @@ /** * Decorator class for File I/O */ -public class FileIODecorator implements FileIO { +public class FileIODecorator extends AbstractFileIO { /** File I/O delegate */ private final FileIO delegate; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java index 2d9e37f3b50a9..d2d5506f6cb01 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java @@ -165,8 +165,7 @@ private long initFile(FileIO fileIO) throws IOException { try { ByteBuffer hdr = header(type, dbCfg.getPageSize()); - while (hdr.remaining() > 0) - fileIO.write(hdr); + fileIO.writeFully(hdr); //there is 'super' page in every file return headerSize() + dbCfg.getPageSize(); @@ -188,8 +187,7 @@ private long initFile(FileIO fileIO) throws IOException { private long checkFile(FileIO fileIO) throws IOException { ByteBuffer hdr = ByteBuffer.allocate(headerSize()).order(ByteOrder.LITTLE_ENDIAN); - while (hdr.remaining() > 0) - fileIO.read(hdr); + fileIO.readFully(hdr); hdr.rewind(); @@ -596,19 +594,7 @@ private void reinit(FileIO fileIO) throws IOException { assert pageBuf.position() == 0 : pageBuf.position(); - int len = pageSize; - - if (fileIO == null) - throw new IOException("FileIO has stopped"); - - do { - int n = fileIO.write(pageBuf, off); - - off += n; - - len -= n; - } - while (len > 0); + fileIO.writeFully(pageBuf, off); PageIO.setCrc(pageBuf, 0); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileVersionCheckingFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileVersionCheckingFactory.java index ab36d7cbd5201..bc938a57912fc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileVersionCheckingFactory.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileVersionCheckingFactory.java @@ -86,8 +86,7 @@ public FileVersionCheckingFactory(FileIOFactory fileIOFactory, DataStorageConfig ByteBuffer hdr = ByteBuffer.allocate(minHdr).order(ByteOrder.LITTLE_ENDIAN); - while (hdr.remaining() > 0) - fileIO.read(hdr); + fileIO.readFully(hdr); hdr.rewind(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java index 018ed276de7ab..ef4a3df37be4f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java @@ -27,7 +27,7 @@ /** * File I/O implementation based on {@link FileChannel}. */ -public class RandomAccessFileIO implements FileIO { +public class RandomAccessFileIO extends AbstractFileIO { /** * File channel. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/UnzipFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/UnzipFileIO.java index 8194ba36eed14..6345b1fc0aab0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/UnzipFileIO.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/UnzipFileIO.java @@ -30,7 +30,7 @@ * Doesn't allow random access and setting {@link FileIO#position()} backwards. * Allows sequential reads including setting {@link FileIO#position()} forward. */ -public class UnzipFileIO implements FileIO { +public class UnzipFileIO extends AbstractFileIO { /** Zip input stream. */ private final ZipInputStream zis; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java index 0e095fa24f17f..6ea7e002b6e6d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java @@ -28,6 +28,9 @@ public class FileWALPointer implements WALPointer, Comparable { /** Serial version uid. */ private static final long serialVersionUID = 0L; + /** Pointer serialized size. */ + public static final int POINTER_SIZE = 16; + /** Absolute WAL segment file index (incrementing counter) */ private final long 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 4a696e47a7abc..44003039549b6 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 @@ -1402,22 +1402,9 @@ private void formatFile(File file, int bytesCntToFormat) throws StorageException try (FileIO fileIO = ioFactory.create(file, CREATE, READ, WRITE)) { int left = bytesCntToFormat; - if (mode == WALMode.FSYNC) { - while (left > 0) { - int toWrite = Math.min(FILL_BUF.length, left); - - if (fileIO.write(FILL_BUF, 0, toWrite) < toWrite) { - final StorageException ex = new StorageException("Failed to extend WAL segment file: " + - file.getName() + ". Probably disk is too busy, please check your device."); - - if (failureProcessor != null) - failureProcessor.process(new FailureContext(FailureType.CRITICAL_ERROR, ex)); - - throw ex; - } - - left -= toWrite; - } + if (mode == WALMode.FSYNC || mmap) { + while ((left -= fileIO.writeFully(FILL_BUF, 0, Math.min(FILL_BUF.length, left))) > 0) + ; fileIO.force(); } @@ -1425,7 +1412,12 @@ private void formatFile(File file, int bytesCntToFormat) throws StorageException fileIO.clear(); } catch (IOException e) { - throw new StorageException("Failed to format WAL segment file: " + file.getAbsolutePath(), e); + StorageException ex = new StorageException("Failed to format WAL segment file: " + file.getAbsolutePath(), e); + + if (failureProcessor != null) + failureProcessor.process(new FailureContext(FailureType.CRITICAL_ERROR, ex)); + + throw ex; } } @@ -2168,17 +2160,8 @@ private class FileDecompressor extends GridWorker { FileIO io = ioFactory.create(unzipTmp)) { zis.getNextEntry(); - int bytesRead; - while ((bytesRead = zis.read(arr)) > 0) - if (io.write(arr, 0, bytesRead) < bytesRead) { - final IgniteCheckedException ex = new IgniteCheckedException("Failed to extend file: " + - unzipTmp.getName() + ". Probably disk is too busy, please check your device."); - - if (failureProcessor != null) - failureProcessor.process(new FailureContext(FailureType.CRITICAL_ERROR, ex)); - - throw ex; - } + while (io.writeFully(arr, 0, zis.read(arr)) > 0) + ; } try { @@ -3475,12 +3458,7 @@ private void writeBuffer(long pos, ByteBuffer buf) throws StorageException, Igni try { assert hdl.written == hdl.fileIO.position(); - do { - hdl.fileIO.write(buf); - } - while (buf.hasRemaining()); - - hdl.written += size; + hdl.written += hdl.fileIO.writeFully(buf); metrics.onWalBytesWritten(size); 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 5db21d25ae4a3..45cb1a72b28bc 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 @@ -1223,13 +1223,8 @@ private void formatFile(File file, int bytesCntToFormat) throws StorageException int left = bytesCntToFormat; if (mode == WALMode.FSYNC) { - while (left > 0) { - int toWrite = Math.min(FILL_BUF.length, left); - - fileIO.write(FILL_BUF, 0, toWrite); - - left -= toWrite; - } + while ((left -= fileIO.writeFully(FILL_BUF, 0, Math.min(FILL_BUF.length, left))) > 0) + ; fileIO.force(); } @@ -1972,9 +1967,8 @@ private class FileDecompressor extends GridWorker { FileIO io = ioFactory.create(unzipTmp)) { zis.getNextEntry(); - int bytesRead; - while ((bytesRead = zis.read(arr)) > 0) - io.write(arr, 0, bytesRead); + while (io.writeFully(arr, 0, zis.read(arr)) > 0) + ; } try { @@ -2093,10 +2087,7 @@ else if (create) public static long writeSerializerVersion(FileIO io, long idx, int version, WALMode mode) throws IOException { ByteBuffer buffer = prepareSerializerVersionBuffer(idx, version, false); - do { - io.write(buffer); - } - while (buffer.hasRemaining()); + io.writeFully(buffer); // Flush if (mode == WALMode.FSYNC) @@ -2831,15 +2822,7 @@ private boolean close(boolean rollOver) throws StorageException { buf.rewind(); - int rem = buf.remaining(); - - while (rem > 0) { - int written0 = fileIO.write(buf, written); - - written += written0; - - rem -= written0; - } + written += fileIO.writeFully(buf, written); } } catch (IgniteCheckedException e) { @@ -2981,10 +2964,7 @@ private void writeBuffer(long pos, ByteBuffer buf) throws StorageException { try { assert written == fileIO.position(); - do { - fileIO.write(buf); - } - while (buf.hasRemaining()); + fileIO.writeFully(buf); written += size; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java index e03cf52035768..f7299b962eeaa 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java @@ -234,7 +234,7 @@ private void eraseDataFromDisk( long size = fileIO.size(); - fileIO.write(ByteBuffer.allocate((int)size - filePageStore.headerSize()), filePageStore.headerSize()); + fileIO.writeFully(ByteBuffer.allocate((int)size - filePageStore.headerSize()), filePageStore.headerSize()); fileIO.force(); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java index 41a18ec2e8c59..d6c244367cc10 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/LocalWalModeChangeDuringRebalancingSelfTest.java @@ -39,6 +39,7 @@ import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage; import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointHistory; +import org.apache.ignite.internal.processors.cache.persistence.file.AbstractFileIO; 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.util.lang.GridAbsPredicate; @@ -523,7 +524,7 @@ private static class TestFileIOFactory implements FileIOFactory { /** * */ - private static class TestFileIO implements FileIO { + private static class TestFileIO extends AbstractFileIO { /** */ private final FileIO delegate; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgniteDataIntegrityTests.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgniteDataIntegrityTests.java index 3d5250703de5a..c077b27cb8ee3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgniteDataIntegrityTests.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgniteDataIntegrityTests.java @@ -70,7 +70,7 @@ public class IgniteDataIntegrityTests extends TestCase { buf.rewind(); - fileInput.io().write(buf); + fileInput.io().writeFully(buf); fileInput.io().force(); } @@ -180,12 +180,12 @@ private void toggleOneRandomBit(int rangeFrom, int rangeTo) throws IOException { byte[] buf = new byte[1]; - fileInput.io().read(buf, 0, 1); + fileInput.io().readFully(buf, 0, 1); buf[0] ^= (1 << 3); fileInput.io().position(pos); - fileInput.io().write(buf, 0, 1); + fileInput.io().writeFully(buf, 0, 1); fileInput.io().force(); } diff --git a/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIO.java b/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIO.java index 88d77e017f4c0..2a7504cc0bc11 100644 --- a/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIO.java +++ b/modules/direct-io/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AlignedBuffersDirectFileIO.java @@ -42,7 +42,7 @@ * * Works only for Linux */ -public class AlignedBuffersDirectFileIO implements FileIO { +public class AlignedBuffersDirectFileIO extends AbstractFileIO { /** Negative value for file offset: read/write starting from current file position */ private static final int FILE_POS_USE_CURRENT = -1; diff --git a/modules/direct-io/src/test/java/org/apache/ignite/internal/processors/cache/persistence/file/IgniteFileIOTest.java b/modules/direct-io/src/test/java/org/apache/ignite/internal/processors/cache/persistence/file/IgniteFileIOTest.java new file mode 100644 index 0000000000000..9620eb0fca683 --- /dev/null +++ b/modules/direct-io/src/test/java/org/apache/ignite/internal/processors/cache/persistence/file/IgniteFileIOTest.java @@ -0,0 +1,304 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.file; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.MappedByteBuffer; +import java.util.concurrent.ThreadLocalRandom; +import junit.framework.TestCase; +import org.jetbrains.annotations.NotNull; + +/** + * File IO tests. + */ +public class IgniteFileIOTest extends TestCase { + /** Test data size. */ + private static final int TEST_DATA_SIZE = 16 * 1024 * 1024; + + /** + * + */ + private static class TestFileIO extends AbstractFileIO { + /** Data. */ + private final byte[] data; + /** Position. */ + private int position; + + /** + * @param maxSize Maximum size. + */ + TestFileIO(int maxSize) { + this.data = new byte[maxSize]; + } + + /** + * @param data Initial data. + */ + TestFileIO(byte[] data) { + this.data = data; + } + + /** {@inheritDoc} */ + @Override public long position() throws IOException { + return position; + } + + /** {@inheritDoc} */ + @Override public void position(long newPosition) throws IOException { + checkPosition(newPosition); + + this.position = (int)newPosition; + } + + /** {@inheritDoc} */ + @Override public int read(ByteBuffer destBuf) throws IOException { + final int len = Math.min(destBuf.remaining(), data.length - position); + + destBuf.put(data, position, len); + + position += len; + + return len; + } + + /** {@inheritDoc} */ + @Override public int read(ByteBuffer destBuf, long position) throws IOException { + checkPosition(position); + + final int len = Math.min(destBuf.remaining(), data.length - (int)position); + + destBuf.put(data, (int)position, len); + + return len; + } + + /** {@inheritDoc} */ + @Override public int read(byte[] buf, int off, int maxLen) throws IOException { + final int len = Math.min(maxLen, data.length - position); + + System.arraycopy(data, position, buf, off, len); + + position += len; + + return len; + } + + /** {@inheritDoc} */ + @Override public int write(ByteBuffer srcBuf) throws IOException { + final int len = Math.min(srcBuf.remaining(), data.length - position); + + srcBuf.get(data, position, len); + + position += len; + + return len; + } + + /** {@inheritDoc} */ + @Override public int write(ByteBuffer srcBuf, long position) throws IOException { + checkPosition(position); + + final int len = Math.min(srcBuf.remaining(), data.length - (int)position); + + srcBuf.get(data, (int)position, len); + + return len; + } + + /** {@inheritDoc} */ + @Override public int write(byte[] buf, int off, int maxLen) throws IOException { + final int len = Math.min(maxLen, data.length - position); + + System.arraycopy(buf, off, data, position, len); + + position += len; + + return len; + } + + /** {@inheritDoc} */ + @Override public MappedByteBuffer map(int sizeBytes) throws IOException { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public void force() throws IOException { + } + + /** {@inheritDoc} */ + @Override public void force(boolean withMetadata) throws IOException { + } + + /** {@inheritDoc} */ + @Override public long size() throws IOException { + return data.length; + } + + /** {@inheritDoc} */ + @Override public void clear() throws IOException { + position = 0; + } + + /** {@inheritDoc} */ + @Override public void close() throws IOException { + } + + /** + * @param position Position. + */ + private void checkPosition(long position) throws IOException { + if (position < 0 || position >= data.length) + throw new IOException("Invalid position: " + position); + } + } + + /** + * test for 'full read' functionality. + */ + public void testReadFully() throws Exception { + byte[] arr = new byte[TEST_DATA_SIZE]; + + fillRandomArray(arr); + + ByteBuffer buf = ByteBuffer.allocate(TEST_DATA_SIZE); + + TestFileIO fileIO = new TestFileIO(arr) { + @Override public int read(ByteBuffer destBuf) throws IOException { + if (destBuf.remaining() < 2) + return super.read(destBuf); + + int oldLimit = destBuf.limit(); + + destBuf.limit(destBuf.position() + (destBuf.remaining() >> 1)); + + try { + return super.read(destBuf); + } + finally { + destBuf.limit(oldLimit); + } + } + }; + + fileIO.readFully(buf); + + assert buf.remaining() == 0; + + assert compareArrays(arr, buf.array()); + } + + /** + * test for 'full read' functionality. + */ + public void testReadFullyArray() throws Exception { + byte[] arr = new byte[TEST_DATA_SIZE]; + + byte[] arrDst = new byte[TEST_DATA_SIZE]; + + fillRandomArray(arr); + + TestFileIO fileIO = new TestFileIO(arr) { + @Override public int read(byte[] buf, int off, int len) throws IOException { + return super.read(buf, off, len < 2 ? len : (len >> 1)); + } + }; + + fileIO.readFully(arrDst, 0, arrDst.length); + + assert compareArrays(arr, arrDst); + } + + /** + * test for 'full write' functionality. + */ + public void testWriteFully() throws Exception { + byte[] arr = new byte[TEST_DATA_SIZE]; + + ByteBuffer buf = ByteBuffer.allocate(TEST_DATA_SIZE); + + fillRandomArray(buf.array()); + + TestFileIO fileIO = new TestFileIO(arr) { + @Override public int write(ByteBuffer destBuf) throws IOException { + if (destBuf.remaining() < 2) + return super.write(destBuf); + + int oldLimit = destBuf.limit(); + + destBuf.limit(destBuf.position() + (destBuf.remaining() >> 1)); + + try { + return super.write(destBuf); + } + finally { + destBuf.limit(oldLimit); + } + } + }; + + fileIO.writeFully(buf); + + assert buf.remaining() == 0; + + assert compareArrays(arr, buf.array()); + } + + /** + * test for 'full write' functionality. + */ + public void testWriteFullyArray() throws Exception { + byte[] arr = new byte[TEST_DATA_SIZE]; + + byte[] arrSrc = new byte[TEST_DATA_SIZE]; + + fillRandomArray(arrSrc); + + TestFileIO fileIO = new TestFileIO(arr) { + @Override public int write(byte[] buf, int off, int len) throws IOException { + return super.write(buf, off, len < 2 ? len : (len >> 1)); + } + }; + + fileIO.writeFully(arrSrc, 0, arrSrc.length); + + assert compareArrays(arr, arrSrc); + } + + /** + * @param arr Array. + */ + private static void fillRandomArray(@NotNull final byte[] arr) { + ThreadLocalRandom.current().nextBytes(arr); + } + + /** + * @param arr1 Array 1. + * @param arr2 Array 2. + */ + private static boolean compareArrays(@NotNull final byte[] arr1, @NotNull final byte[] arr2) { + if (arr1.length != arr2.length) + return false; + + for (int i = 0; i < arr1.length; i++) + if (arr1[i] != arr2[i]) + return false; + + return true; + } +} From ee33c63963eaad2547d57d1bff3cb55572bbafa2 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Tue, 3 Jul 2018 19:19:47 +0300 Subject: [PATCH 0303/1463] IGNITE-8681 Using ExpiryPolicy with persistence causes significant slowdown - Fixes #4285. Signed-off-by: Ivan Rakov (cherry picked from commit a06c47f) --- .../apache/ignite/IgniteSystemProperties.java | 7 + .../processors/cache/GridCacheMapEntry.java | 193 ++++++++++++++---- .../processors/cache/GridCacheTtlManager.java | 3 + .../cache/IgniteCacheOffheapManagerImpl.java | 52 ++++- .../colocated/GridDhtDetachedCacheEntry.java | 3 +- .../distributed/near/GridNearCacheEntry.java | 2 +- .../persistence/GridCacheOffheapManager.java | 45 +++- .../IgniteCacheExpiryStoreLoadSelfTest.java | 2 +- .../GridCacheAbstractMetricsSelfTest.java | 8 +- .../IgniteCacheEntryListenerAbstractTest.java | 4 +- .../IgniteCacheExpiryPolicyTestSuite.java | 2 - .../expiry/IgniteCacheTtlCleanupSelfTest.java | 87 -------- .../CacheOperationsWithExpirationTest.java | 6 +- .../IgniteCacheAbstractQuerySelfTest.java | 4 +- ...hePartitionedQueryP2PDisabledSelfTest.java | 2 +- 15 files changed, 256 insertions(+), 164 deletions(-) delete mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheTtlCleanupSelfTest.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 85e13ad309ce4..ffb79c088b162 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -911,6 +911,13 @@ public final class IgniteSystemProperties { */ public static final String IGNITE_DUMP_THREADS_ON_FAILURE = "IGNITE_DUMP_THREADS_ON_FAILURE"; + /** + * Throttling timeout in millis which avoid excessive PendingTree access on unwind if there is nothing to clean yet. + * + * Default is 500 ms. + */ + public static final String IGNITE_UNWIND_THROTTLING_TIMEOUT = "IGNITE_UNWIND_THROTTLING_TIMEOUT"; + /** * Enforces singleton. */ 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 faf44601563ec..a3fcf921bf955 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 @@ -403,9 +403,7 @@ protected GridDhtLocalPartition localPartition() { update(val, read.expireTime(), 0, read.version(), false); - long delta = checkExpire ? - (read.expireTime() == 0 ? 0 : read.expireTime() - U.currentTimeMillis()) - : 0; + long delta = checkExpire && read.expireTime() > 0 ? read.expireTime() - U.currentTimeMillis() : 0; if (delta >= 0) return read; @@ -612,7 +610,7 @@ private Object innerGet0( if (val != null) { long expireTime = expireTimeExtras(); - if (expireTime > 0 && (expireTime - U.currentTimeMillis() <= 0)) { + if (expireTime > 0 && (expireTime < U.currentTimeMillis())) { if (onExpired((CacheObject)cctx.unwrapTemporary(val), null)) { val = null; evt = false; @@ -754,7 +752,7 @@ else if (tx.dht()) { long expTime = CU.toExpireTime(ttl); // Update indexes before actual write to entry. - storeValue(ret, expTime, nextVer, null); + storeValue(ret, expTime, nextVer); update(ret, expTime, ttl, nextVer, true); @@ -865,7 +863,7 @@ private EntryGetResult entryGetResult(CacheObject val, GridCacheVersion ver, boo // Update indexes. if (ret != null) { - storeValue(ret, expTime, nextVer, null); + storeValue(ret, expTime, nextVer); if (cctx.deferredDelete() && !isInternal() && !detached() && deletedUnlocked()) deletedUnlocked(false); @@ -1028,7 +1026,7 @@ else if (interceptorVal != val0) assert val != null; - storeValue(val, expireTime, newVer, null); + storeValue(val, expireTime, newVer); if (cctx.deferredDelete() && deletedUnlocked() && !isInternal() && !detached()) deletedUnlocked(false); @@ -1375,6 +1373,8 @@ private boolean notifyContinuousQueries(@Nullable IgniteInternalTx tx) { lockEntry(); try { + checkObsolete(); + boolean internal = isInternal() || !context().userCache(); Map lsnrCol = @@ -1386,16 +1386,22 @@ private boolean notifyContinuousQueries(@Nullable IgniteInternalTx tx) { !F.isEmpty(filter) || lsnrCol != null; - checkObsolete(); - - CacheDataRow oldRow = null; - // Load and remove from swap if it is new. if (isNew()) - oldRow = unswap(null, false); + unswap(null, false); old = val; + if (expireTimeExtras() > 0 && expireTimeExtras() < U.currentTimeMillis()) { + if (onExpired(val, null)) { + assert !deletedUnlocked(); + + update(null, CU.TTL_ETERNAL, CU.EXPIRE_TIME_ETERNAL, ver, true); + + old = null; + } + } + boolean readFromStore = false; Object old0 = null; @@ -1426,7 +1432,7 @@ else if (ttl == CU.TTL_NOT_CHANGED) old = cctx.kernalContext().cacheObjects().prepareForCache(old, cctx); if (old != null) - storeValue(old, expireTime, ver, oldRow); + storeValue(old, expireTime, ver); else removeValue(); @@ -1573,7 +1579,7 @@ else if (ttl != CU.TTL_ZERO) // Must persist inside synchronization in non-tx mode. cctx.store().put(null, key, updated, ver); - storeValue(updated, expireTime, ver, oldRow); + storeValue(updated, expireTime, ver); assert ttl != CU.TTL_ZERO; @@ -1761,7 +1767,8 @@ else if (ttl != CU.TTL_ZERO) assert updateRes != null : c; - CacheObject oldVal = c.oldRow != null ? c.oldRow.value() : null; + // We should ignore expired old row. Expired oldRow instance is needed for correct row replacement\deletion only. + CacheObject oldVal = c.oldRow != null && !c.oldRowExpiredFlag ? c.oldRow.value() : null; CacheObject updateVal = null; GridCacheVersion updateVer = c.newVer; @@ -2185,7 +2192,7 @@ protected void clearReader(UUID nodeId) throws GridCacheEntryRemovedException { if (hasValueUnlocked()) { long expireTime = expireTimeExtras(); - if (expireTime > 0 && (expireTime - U.currentTimeMillis() <= 0)) { + if (expireTime > 0 && (expireTime < U.currentTimeMillis())) { if (obsoleteVer == null) obsoleteVer = nextVersion(); @@ -2267,8 +2274,7 @@ protected boolean evictionDisabled() { /** *

    - * Note that {@link #onMarkedObsolete()} should always be called after this method - * returns {@code true}. + * Note that {@link #onMarkedObsolete()} should always be called after this method returns {@code true}. * * @param ver Version. * @param clear {@code True} to clear. @@ -2456,7 +2462,7 @@ private void updateTtl(long ttl) throws IgniteCheckedException, GridCacheEntryRe cctx.shared().database().checkpointReadLock(); try { - storeValue(val, expireTime, ver, null); + storeValue(val, expireTime, ver); } finally { cctx.shared().database().checkpointReadUnlock(); @@ -2751,9 +2757,8 @@ protected final boolean hasValueUnlocked() { if (update) { // 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 (oldExpTime > 0 && oldExpTime < U.currentTimeMillis()) { if (onExpired(this.val, null)) { if (cctx.deferredDelete()) { deferred = true; @@ -2764,11 +2769,11 @@ else if (val == null) } } - storeValue(val, expTime, ver, null); + storeValue(val, expTime, ver); } } else // Optimization to access storage only once. - update = storeValue(val, expTime, ver, null, p); + update = storeValue(val, expTime, ver, p); if (update) { update(val, expTime, ttl, ver, true); @@ -2957,7 +2962,7 @@ protected long nextPartitionCounter(AffinityTopologyVersion topVer, boolean prim val = cctx.kernalContext().cacheObjects().prepareForCache(val, cctx); if (val != null) { - storeValue(val, expTime, newVer, null); + storeValue(val, expTime, newVer); if (deletedUnlocked()) deletedUnlocked(false); @@ -3328,7 +3333,7 @@ private GridCacheVersion nextVersion() { long expireTime = expireTimeExtras(); - if (expireTime == 0 || (expireTime - U.currentTimeMillis() > 0)) + if (!(expireTime > 0 && expireTime < U.currentTimeMillis())) return false; CacheObject expiredVal = this.val; @@ -3577,14 +3582,12 @@ private IgniteTxLocalAdapter currentTx() { * @param val Value. * @param expireTime Expire time. * @param ver New entry version. - * @param oldRow Old row if available. * @throws IgniteCheckedException If update failed. */ protected boolean storeValue(@Nullable CacheObject val, long expireTime, - GridCacheVersion ver, - @Nullable CacheDataRow oldRow) throws IgniteCheckedException { - return storeValue(val, expireTime, ver, oldRow, null); + GridCacheVersion ver) throws IgniteCheckedException { + return storeValue(val, expireTime, ver, null); } /** @@ -3593,7 +3596,6 @@ protected boolean storeValue(@Nullable CacheObject val, * @param val Value. * @param expireTime Expire time. * @param ver New entry version. - * @param oldRow Old row if available. * @param predicate Optional predicate. * * @return {@code True} if storage was modified. @@ -3603,7 +3605,6 @@ protected boolean storeValue( @Nullable CacheObject val, long expireTime, GridCacheVersion ver, - @Nullable CacheDataRow oldRow, @Nullable IgnitePredicate predicate) throws IgniteCheckedException { assert lock.isHeldByCurrentThread(); @@ -4421,6 +4422,12 @@ private static class UpdateClosure implements IgniteCacheOffheapManager.OffheapI /** {@inheritDoc} */ @Override public void call(@Nullable CacheDataRow oldRow) throws IgniteCheckedException { + if (oldRow != null) { + oldRow.key(entry.key); + + oldRow = checkRowExpired(oldRow); + } + this.oldRow = oldRow; if (predicate != null && !predicate.apply(oldRow)) { @@ -4429,9 +4436,6 @@ private static class UpdateClosure implements IgniteCacheOffheapManager.OffheapI return; } - if (oldRow != null) - oldRow.key(entry.key); - if (val != null) { newRow = entry.cctx.offheap().dataStore(entry.localPartition()).createRow( entry.cctx, @@ -4462,6 +4466,53 @@ private static class UpdateClosure implements IgniteCacheOffheapManager.OffheapI @Nullable @Override public CacheDataRow oldRow() { return oldRow; } + + /** + * Checks row for expiration and fire expire events if needed. + * + * @param row old row. + * @return {@code Null} if row was expired, row itself otherwise. + * @throws IgniteCheckedException + */ + private CacheDataRow checkRowExpired(CacheDataRow row) throws IgniteCheckedException { + assert row != null; + + if (!(row.expireTime() > 0 && row.expireTime() < U.currentTimeMillis())) + return row; + + GridCacheContext cctx = entry.context(); + + CacheObject expiredVal = row.value(); + + if (cctx.deferredDelete() && !entry.detached() && !entry.isInternal()) { + entry.update(null, CU.TTL_ETERNAL, CU.EXPIRE_TIME_ETERNAL, entry.ver, true); + + if (!entry.deletedUnlocked() && !entry.isStartVersion()) + entry.deletedUnlocked(true); + } + else + entry.markObsolete0(cctx.versions().next(), true, null); + + if (cctx.events().isRecordable(EVT_CACHE_OBJECT_EXPIRED)) { + cctx.events().addEvent(entry.partition(), + entry.key(), + cctx.localNodeId(), + null, + EVT_CACHE_OBJECT_EXPIRED, + null, + false, + expiredVal, + expiredVal != null, + null, + null, + null, + true); + } + + cctx.continuousQueries().onEntryExpired(entry, entry.key(), expiredVal); + + return null; + } } /** @@ -4537,6 +4588,9 @@ private static class AtomicCacheUpdateClosure implements IgniteCacheOffheapManag /** */ private CacheDataRow oldRow; + /** OldRow expiration flag. */ + private boolean oldRowExpiredFlag = false; + AtomicCacheUpdateClosure( GridCacheMapEntry entry, AffinityTopologyVersion topVer, @@ -4611,23 +4665,27 @@ private static class AtomicCacheUpdateClosure implements IgniteCacheOffheapManag @Override public void call(@Nullable CacheDataRow oldRow) throws IgniteCheckedException { assert entry.isNear() || oldRow == null || oldRow.link() != 0 : oldRow; - if (oldRow != null) - oldRow.key(entry.key()); - - this.oldRow = oldRow; - GridCacheContext cctx = entry.context(); CacheObject oldVal; CacheObject storeLoadedVal = null; + this.oldRow = oldRow; + if (oldRow != null) { - oldVal = oldRow.value(); + oldRow.key(entry.key()); + + // unswap + entry.update(oldRow.value(), oldRow.expireTime(), 0, oldRow.version(), false); - entry.update(oldVal, oldRow.expireTime(), 0, oldRow.version(), false); + if (checkRowExpired(oldRow)) { + oldRowExpiredFlag = true; + + oldRow = null; + } } - else - oldVal = null; + + oldVal = (oldRow != null) ? oldRow.value() : null; if (oldVal == null && readThrough) { storeLoadedVal = cctx.toCacheObject(cctx.store().load(null, entry.key)); @@ -4764,6 +4822,53 @@ else if ((invokeRes == null || invokeRes.getValue() == null) && writeObj != null assert updateRes != null && treeOp != null; } + /** + * Check row expiration and fire expire events if needed. + * + * @param row Old row. + * @return {@code True} if row was expired, {@code False} otherwise. + * @throws IgniteCheckedException if failed. + */ + private boolean checkRowExpired(CacheDataRow row) throws IgniteCheckedException { + assert row != null; + + if (!(row.expireTime() > 0 && row.expireTime() < U.currentTimeMillis())) + return false; + + GridCacheContext cctx = entry.context(); + + CacheObject expiredVal = row.value(); + + if (cctx.deferredDelete() && !entry.detached() && !entry.isInternal()) { + entry.update(null, CU.TTL_ETERNAL, CU.EXPIRE_TIME_ETERNAL, entry.ver, true); + + if (!entry.deletedUnlocked()) + entry.deletedUnlocked(true); + } + else + entry.markObsolete0(cctx.versions().next(), true, null); + + if (cctx.events().isRecordable(EVT_CACHE_OBJECT_EXPIRED)) { + cctx.events().addEvent(entry.partition(), + entry.key(), + cctx.localNodeId(), + null, + EVT_CACHE_OBJECT_EXPIRED, + null, + false, + expiredVal, + expiredVal != null, + null, + null, + null, + true); + } + + cctx.continuousQueries().onEntryExpired(entry, entry.key(), expiredVal); + + return true; + } + /** * @param storeLoadedVal Value loaded from store. * @param updateExpireTime {@code True} if need update expire time. @@ -5063,7 +5168,7 @@ private void remove(@Nullable GridCacheVersionConflictContext conflictCtx, entry.update(null, CU.TTL_ETERNAL, CU.EXPIRE_TIME_ETERNAL, newVer, true); - treeOp = (oldVal == null || readFromStore) ? IgniteTree.OperationType.NOOP : + treeOp = (oldRow == null || readFromStore) ? IgniteTree.OperationType.NOOP : IgniteTree.OperationType.REMOVE; UpdateOutcome outcome = oldVal != null ? UpdateOutcome.SUCCESS : UpdateOutcome.REMOVE_NO_VAL; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java index ff6ed65bd03cd..8277c2a34a28c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java @@ -201,6 +201,9 @@ public boolean expire(int amount) { } } + if(!(cctx.affinityNode() && cctx.ttl().eagerTtlEnabled())) + return false; /* Pending tree never contains entries for that cache */ + boolean more = cctx.offheap().expire(dhtCtx, expireC, amount); if (more) 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 bf0de02cb3440..10281d607217b 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 @@ -33,6 +33,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.pagemem.FullPageId; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; @@ -85,6 +86,13 @@ */ @SuppressWarnings("PublicInnerClass") public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager { + /** + * Throttling timeout in millis which avoid excessive PendingTree access on unwind + * if there is nothing to clean yet. + */ + public static final long UNWIND_THROTTLING_TIMEOUT = Long.getLong( + IgniteSystemProperties.IGNITE_UNWIND_THROTTLING_TIMEOUT, 500L); + /** */ protected GridCacheSharedContext ctx; @@ -106,6 +114,9 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager /** */ protected volatile boolean hasPendingEntries; + /** Timestamp when next clean try will be allowed. Used for throttling on per-group basis. */ + protected volatile long nextCleanTime; + /** */ private final GridAtomicLong globalRmvId = new GridAtomicLong(U.currentTimeMillis() * 1000_000); @@ -1029,13 +1040,36 @@ protected final String treeName(int p) { ) throws IgniteCheckedException { assert !cctx.isNear() : cctx.name(); - if (!hasPendingEntries || pendingEntries == null) + if (!hasPendingEntries || nextCleanTime > U.currentTimeMillis()) return false; - GridCacheVersion obsoleteVer = null; + assert pendingEntries != null; + + int cleared = expireInternal(cctx, c, amount); + + // Throttle if there is nothing to clean anymore. + if (cleared < amount) + nextCleanTime = U.currentTimeMillis() + UNWIND_THROTTLING_TIMEOUT; + return amount != -1 && cleared >= amount; + } + + /** + * @param cctx Cache context. + * @param c Closure. + * @param amount Limit of processed entries by single call, {@code -1} for no limit. + * @return cleared entries count. + * @throws IgniteCheckedException If failed. + */ + private int expireInternal( + GridCacheContext cctx, + IgniteInClosure2X c, + int amount + ) throws IgniteCheckedException { long now = U.currentTimeMillis(); + GridCacheVersion obsoleteVer = null; + GridCursor cur; if (grp.sharedGroup()) @@ -1044,17 +1078,17 @@ protected final String treeName(int p) { cur = pendingEntries.find(null, new PendingRow(CU.UNDEFINED_CACHE_ID, now, 0)); if (!cur.next()) - return false; - - int cleared = 0; + return 0; if (!busyLock.enterBusy()) - return false; + return 0; try { + int cleared = 0; + do { if (amount != -1 && cleared > amount) - return true; + return cleared; PendingRow row = cur.get(); @@ -1076,12 +1110,12 @@ protected final String treeName(int p) { cleared++; } while (cur.next()); + + return cleared; } finally { busyLock.leaveBusy(); } - - return false; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java index d02015b665489..346a992f3ac88 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java @@ -67,8 +67,7 @@ public void resetFromPrimary(CacheObject val, GridCacheVersion ver) { /** {@inheritDoc} */ @Override protected boolean storeValue(CacheObject val, long expireTime, - GridCacheVersion ver, - CacheDataRow oldRow) throws IgniteCheckedException { + GridCacheVersion ver) throws IgniteCheckedException { return false; // No-op for detached entries, index is updated on primary nodes. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java index fb41f5c2e306f..c953bebbbfd13 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java @@ -458,7 +458,7 @@ public boolean loadedValue(@Nullable IgniteInternalTx tx, } /** {@inheritDoc} */ - @Override protected boolean storeValue(CacheObject val, long expireTime, GridCacheVersion ver, CacheDataRow oldRow) { + @Override protected boolean storeValue(CacheObject val, long expireTime, GridCacheVersion ver) { return false; // No-op: queries are disabled for near cache. } 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 8173e48ac07f6..956e334b4c5bb 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 @@ -776,9 +776,10 @@ private Metas getOrAllocateCacheMetas() throws IgniteCheckedException { ) throws IgniteCheckedException { assert !cctx.isNear() : cctx.name(); - if (!hasPendingEntries) + if (!hasPendingEntries || nextCleanTime > U.currentTimeMillis()) return false; + // Prevent manager being stopped in the middle of pds operation. if (!busyLock.enterBusy()) return false; @@ -791,6 +792,10 @@ private Metas getOrAllocateCacheMetas() throws IgniteCheckedException { if (amount != -1 && cleared >= amount) return true; } + + // Throttle if there is nothing to clean anymore. + if (cleared < amount) + nextCleanTime = U.currentTimeMillis() + UNWIND_THROTTLING_TIMEOUT; } finally { busyLock.leaveBusy(); @@ -1159,6 +1164,10 @@ public class GridCacheDataStore implements CacheDataStore { /** */ private volatile CacheDataStore delegate; + /** Timestamp when next clean try will be allowed for current partition. + * Used for fine-grained throttling on per-partition basis. */ + private volatile long nextStoreCleanTime; + /** */ private final boolean exists; @@ -1721,12 +1730,12 @@ public long expiredSize() throws IgniteCheckedException { } /** - * Removes expired entries from data store. + * Try to remove expired entries from data store. * * @param cctx Cache context. * @param c Expiry closure that should be applied to expired entry. See {@link GridCacheTtlManager} for details. * @param amount Limit of processed entries by single call, {@code -1} for no limit. - * @return {@code True} if unprocessed expired entries remains. + * @return cleared entries count. * @throws IgniteCheckedException If failed. */ public int purgeExpired(GridCacheContext cctx, @@ -1734,13 +1743,39 @@ public int purgeExpired(GridCacheContext cctx, int amount) throws IgniteCheckedException { CacheDataStore delegate0 = init0(true); - if (delegate0 == null || pendingTree == null) + long now = U.currentTimeMillis(); + + if (delegate0 == null || nextStoreCleanTime > now) return 0; + assert pendingTree != null : "Partition data store was not initialized."; + + int cleared = purgeExpiredInternal(cctx, c, amount); + + // Throttle if there is nothing to clean anymore. + if (cleared < amount) + nextStoreCleanTime = now + UNWIND_THROTTLING_TIMEOUT; + + return cleared; + } + + /** + * Removes expired entries from data store. + * + * @param cctx Cache context. + * @param c Expiry closure that should be applied to expired entry. See {@link GridCacheTtlManager} for details. + * @param amount Limit of processed entries by single call, {@code -1} for no limit. + * @return cleared entries count. + * @throws IgniteCheckedException If failed. + */ + private int purgeExpiredInternal(GridCacheContext cctx, + IgniteInClosure2X c, + int amount) throws IgniteCheckedException { + GridDhtLocalPartition part = cctx.topology().localPartition(partId, AffinityTopologyVersion.NONE, false, false); // Skip non-owned partitions. - if (part == null || part.state() != OWNING || pendingTree.size() == 0) + if (part == null || part.state() != OWNING) return 0; cctx.shared().database().checkpointReadLock(); diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/IgniteCacheExpiryStoreLoadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/IgniteCacheExpiryStoreLoadSelfTest.java index a6c9997f59a10..cdc4277974bf7 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/store/IgniteCacheExpiryStoreLoadSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/store/IgniteCacheExpiryStoreLoadSelfTest.java @@ -51,7 +51,7 @@ public class IgniteCacheExpiryStoreLoadSelfTest extends GridCacheAbstractSelfTes private static final int TIME_TO_LIVE = 1000; /** Additional time to wait expiry process in milliseconds. */ - private static final int WAIT_TIME = 1000; + private static final int WAIT_TIME = 1500; /** {@inheritDoc} */ @Override protected int gridCount() { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java index 7948569bf9327..721989b87665f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java @@ -177,7 +177,7 @@ public void testGetAndRemoveAsyncAvgTime() throws Exception { for (int i = 0; i < KEY_CNT; i++) cache.put(i, i); - assertEquals(cache.localMetrics().getAverageRemoveTime(), 0.0, 0.0); + assertEquals(0.0, cache.localMetrics().getAverageRemoveTime(), 0.0); for (int i = 0; i < KEY_CNT; i++) cache.getAndRemoveAsync(i).get(); @@ -221,7 +221,7 @@ public void testRemoveAvgTime() throws Exception { for (int i = 0; i < KEY_CNT; i++) cache.put(i, i); - assertEquals(cache.localMetrics().getAverageRemoveTime(), 0.0, 0.0); + assertEquals(0.0, cache.localMetrics().getAverageRemoveTime(), 0.0); for (int i = 0; i < KEY_CNT; i++) cache.remove(i); @@ -239,7 +239,7 @@ public void testRemoveAllAvgTime() throws Exception { cache.put(2, 2); cache.put(3, 3); - assertEquals(cache.localMetrics().getAverageRemoveTime(), 0.0, 0.0); + assertEquals(0.0, cache.localMetrics().getAverageRemoveTime(), 0.0); Set keys = new HashSet<>(4, 1); keys.add(1); @@ -272,7 +272,7 @@ public void testRemoveAllAsyncAvgTime() throws Exception { } } - assertEquals(cache.localMetrics().getAverageRemoveTime(), 0.0, 0.0); + assertEquals(0.0, cache.localMetrics().getAverageRemoveTime(), 0.0); IgniteFuture fut = cache.removeAllAsync(keys); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java index e473d5238289b..c6e355291ab63 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java @@ -896,7 +896,7 @@ private void checkEvents( U.sleep(700); if (!eagerTtl()) - assertNull(primaryCache(key, cache.getName()).get(key(key))); // Provoke expire event if eager ttl is disabled. + assertNull(primaryCache(key(key), cache.getName()).get(key(key))); // Provoke expire event if eager ttl is disabled. IgniteCache cache1 = cache; @@ -917,7 +917,7 @@ private void checkEvents( U.sleep(200); if (!eagerTtl()) - assertNull(primaryCache(key, cache.getName()).get(key(key))); // Provoke expire event if eager ttl is disabled. + assertNull(primaryCache(key(key), cache.getName()).get(key(key))); // Provoke expire event if eager ttl is disabled. evtsLatch.await(5000, MILLISECONDS); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyTestSuite.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyTestSuite.java index ebd9bb8da1132..7693260d13f0c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyTestSuite.java @@ -51,8 +51,6 @@ public static TestSuite suite() throws Exception { suite.addTestSuite(IgniteCacheExpiryStoreLoadSelfTest.class); - suite.addTestSuite(IgniteCacheTtlCleanupSelfTest.class); - suite.addTestSuite(IgniteCacheClientNearCacheExpiryTest.class); suite.addTestSuite(IgniteCacheEntryListenerExpiredEventsTest.class); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheTtlCleanupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheTtlCleanupSelfTest.java deleted file mode 100644 index 227fe1ffc0e3d..0000000000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheTtlCleanupSelfTest.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.ignite.internal.processors.cache.expiry; - -import java.util.concurrent.TimeUnit; -import javax.cache.expiry.CreatedExpiryPolicy; -import javax.cache.expiry.Duration; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.cache.CacheAtomicityMode; -import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.internal.IgniteKernal; -import org.apache.ignite.internal.processors.cache.CacheObjectContext; -import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest; -import org.apache.ignite.internal.processors.cache.GridCacheAdapter; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; -import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor; - -/** - * - */ -public class IgniteCacheTtlCleanupSelfTest extends GridCacheAbstractSelfTest { - /** Number of partitions. */ - private static final int PART_NUM = 10; - - /** {@inheritDoc} */ - @Override protected int gridCount() { - return 1; - } - - /** {@inheritDoc} */ - @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception { - CacheConfiguration ccfg = super.cacheConfiguration(igniteInstanceName); - - ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC); - - ccfg.setAffinity(new RendezvousAffinityFunction(false, 10)); - - ccfg.setNearConfiguration(null); - - return ccfg; - } - - /** - * @throws Exception If failed. - */ - public void testDeferredDeleteTtl() throws Exception { - IgniteCache cache = grid(0).cache(DEFAULT_CACHE_NAME) - .withExpiryPolicy(new CreatedExpiryPolicy(new Duration(TimeUnit.SECONDS, 5))); - - int cnt = GridDhtLocalPartition.MAX_DELETE_QUEUE_SIZE / PART_NUM + 100; - - for (long i = 0; i < cnt; i++) - grid(0).cache(DEFAULT_CACHE_NAME).put(i * PART_NUM, i); - - for (int i = 0; i < cnt; i++) - cache.put(i * PART_NUM, i); - - // Wait 5 seconds. - Thread.sleep(6_000); - - assertEquals(cnt, grid(0).cache(DEFAULT_CACHE_NAME).size()); - - GridCacheAdapter cacheAdapter = ((IgniteKernal)grid(0)).internalCache(DEFAULT_CACHE_NAME); - - IgniteCacheObjectProcessor cacheObjects = cacheAdapter.context().cacheObjects(); - - CacheObjectContext cacheObjCtx = cacheAdapter.context().cacheObjectContext(); - - for (int i = 0; i < 100; i++) - assertNull(cacheAdapter.map().getEntry(cacheAdapter.context(), cacheObjects.toCacheKeyObject(cacheObjCtx, null, i, true))); - } -} \ No newline at end of file diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheOperationsWithExpirationTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheOperationsWithExpirationTest.java index 4cd6960eda0b0..2b51aa51b3528 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheOperationsWithExpirationTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheOperationsWithExpirationTest.java @@ -50,12 +50,10 @@ public class CacheOperationsWithExpirationTest extends GridCommonAbstractTest { /** * @param atomicityMode Atomicity mode. - * @param offheapMem Offheap memory size. * @param idx Indexing enabled flag. * @return Cache configuration. */ private CacheConfiguration cacheConfiguration(CacheAtomicityMode atomicityMode, - long offheapMem, boolean idx) { CacheConfiguration ccfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME); @@ -88,14 +86,14 @@ private CacheConfiguration cacheConfiguration(CacheAtom * @throws Exception If failed. */ public void testAtomicIndexEnabled() throws Exception { - concurrentPutGetRemoveExpireAndQuery(cacheConfiguration(ATOMIC, 0, true)); + concurrentPutGetRemoveExpireAndQuery(cacheConfiguration(ATOMIC, true)); } /** * @throws Exception If failed. */ public void testAtomic() throws Exception { - concurrentPutGetRemoveExpireAndQuery(cacheConfiguration(ATOMIC, 0, false)); + concurrentPutGetRemoveExpireAndQuery(cacheConfiguration(ATOMIC, false)); } /** 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 ee1a652348c2d..15b308b6a5538 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 @@ -462,7 +462,7 @@ public void testExpiration() throws Exception { assertEquals(1, res.getValue().intValue()); - U.sleep(800); // Less than minimal amount of time that must pass before a cache entry is considered expired. + U.sleep(300); // Less than minimal amount of time that must pass before a cache entry is considered expired. qry = cache.query(new SqlQuery(Integer.class, "1=1")).getAll(); @@ -470,7 +470,7 @@ public void testExpiration() throws Exception { assertEquals(1, res.getValue().intValue()); - U.sleep(1200); // No expiry guarantee here. Test should be refactored in case of fails. + U.sleep(1800); // No expiry guarantee here. Test should be refactored in case of fails. qry = cache.query(new SqlQuery(Integer.class, "1=1")).getAll(); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedQueryP2PDisabledSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedQueryP2PDisabledSelfTest.java index 7e5d74d4de0a9..8925b01969dc8 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedQueryP2PDisabledSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCachePartitionedQueryP2PDisabledSelfTest.java @@ -27,7 +27,7 @@ public class IgniteCachePartitionedQueryP2PDisabledSelfTest extends IgniteCacheP @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { IgniteConfiguration c = super.getConfiguration(igniteInstanceName); - c.setPeerClassLoadingEnabled(true); + c.setPeerClassLoadingEnabled(false); return c; } From ec8f3ff9f4a02535623ae622689651a4835b63ea Mon Sep 17 00:00:00 2001 From: AMedvedev Date: Thu, 5 Jul 2018 13:19:23 +0300 Subject: [PATCH 0304/1463] IGNITE-8737 Improve checkpoint logging information - Fixes #4244. Signed-off-by: Ivan Rakov (cherry-picked from commit#a727a4c7135e57415de0756e8fdc235ba191109a) --- .../GridCacheDatabaseSharedManager.java | 13 ++- .../checkpoint/CheckpointHistory.java | 28 ++++- .../wal/FileWriteAheadLogManager.java | 12 +- .../FsyncModeFileWriteAheadLogManager.java | 8 +- .../checkpoint/IgniteMassLoadSandboxTest.java | 110 +++++++++++++++++- 5 files changed, 154 insertions(+), 17 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 5b6c1956f9557..49def6cc0b1f0 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 @@ -3193,12 +3193,13 @@ private void doCheckpoint() { if (printCheckpointStats) { if (log.isInfoEnabled()) log.info(String.format("Checkpoint finished [cpId=%s, pages=%d, markPos=%s, " + - "walSegmentsCleared=%d, markDuration=%dms, pagesWrite=%dms, fsync=%dms, " + + "walSegmentsCleared=%d, walSegmentsCovered=%s, markDuration=%dms, pagesWrite=%dms, fsync=%dms, " + "total=%dms]", chp.cpEntry != null ? chp.cpEntry.checkpointId() : "", chp.pagesSize, chp.cpEntry != null ? chp.cpEntry.checkpointMark() : "", chp.walFilesDeleted, + chp.walSegmentsCovered, tracker.markDuration(), tracker.pagesWriteDuration(), tracker.fsyncDuration(), @@ -3870,6 +3871,9 @@ public static class Checkpoint { /** Number of deleted WAL files. */ private int walFilesDeleted; + /** WAL segments fully covered by this checkpoint. */ + private List walSegmentsCovered; + /** */ private final int pagesSize; @@ -3903,6 +3907,13 @@ public boolean hasDelta() { public void walFilesDeleted(int walFilesDeleted) { this.walFilesDeleted = walFilesDeleted; } + + /** + * @param walSegmentsCovered WAL segments fully covered by this checkpoint. + */ + public void walSegmentsCovered(final List walSegmentsCovered) { + this.walSegmentsCovered = walSegmentsCovered; + } } /** 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 d6cc297d0fd4f..cef2093ab9e21 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 @@ -193,13 +193,39 @@ public List onWalTruncated(WALPointer ptr) { } /** - * Clears checkpoint history after checkpoint finish. + * Logs and clears checkpoint history after checkpoint finish. * * @return List of checkpoints removed from history. */ public List onCheckpointFinished(GridCacheDatabaseSharedManager.Checkpoint chp, boolean truncateWal) { List removed = new ArrayList<>(); + final Map.Entry lastEntry = histMap.lastEntry(); + + assert lastEntry != null; + + final Map.Entry previousEntry = histMap.lowerEntry(lastEntry.getKey()); + + final WALPointer lastWALPointer = lastEntry.getValue().checkpointMark(); + + long lastIdx = 0; + + long prevIdx = 0; + + final ArrayList walSegmentsCovered = new ArrayList<>(); + + if (lastWALPointer instanceof FileWALPointer) { + lastIdx = ((FileWALPointer)lastWALPointer).index(); + + if (previousEntry != null) + prevIdx = ((FileWALPointer)previousEntry.getValue().checkpointMark()).index(); + } + + for (long walCovered = prevIdx; walCovered < lastIdx; walCovered++) + walSegmentsCovered.add(walCovered); + + chp.walSegmentsCovered(walSegmentsCovered); + int deleted = 0; while (histMap.size() > maxCpHistMemSize) { 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 44003039549b6..2bb6cd9ce7ef6 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 @@ -745,8 +745,8 @@ private void checkWalRolloverRequiredDuringInactivityPeriod() { currWrHandle = rollOver(currWrHandle); - if (log != null && log.isDebugEnabled()) - log.debug("Rollover segment [" + idx + " to " + currWrHandle.idx + "], recordType=" + rec.type()); + if (log != null && log.isInfoEnabled()) + log.info("Rollover segment [" + idx + " to " + currWrHandle.idx + "], recordType=" + rec.type()); } WALPointer ptr = currWrHandle.addRecord(rec); @@ -1814,8 +1814,8 @@ private SegmentArchiveResult archiveSegment(long absIdx) throws StorageException File dstFile = new File(walArchiveDir, name); - if (log.isDebugEnabled()) - log.debug("Starting to copy WAL segment [absIdx=" + absIdx + ", segIdx=" + segIdx + + if (log.isInfoEnabled()) + log.info("Starting to copy WAL segment [absIdx=" + absIdx + ", segIdx=" + segIdx + ", origFile=" + origFile.getAbsolutePath() + ", dstFile=" + dstFile.getAbsolutePath() + ']'); try { @@ -1837,8 +1837,8 @@ private SegmentArchiveResult archiveSegment(long absIdx) throws StorageException ", dstFile=" + dstTmpFile.getAbsolutePath() + ']', e); } - if (log.isDebugEnabled()) - log.debug("Copied file [src=" + origFile.getAbsolutePath() + + if (log.isInfoEnabled()) + log.info("Copied file [src=" + origFile.getAbsolutePath() + ", dst=" + dstFile.getAbsolutePath() + ']'); return new SegmentArchiveResult(absIdx, origFile, dstFile); 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 45cb1a72b28bc..a75dd3100bf3f 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 @@ -1642,8 +1642,8 @@ private SegmentArchiveResult archiveSegment(long absIdx) throws IgniteCheckedExc File dstFile = new File(walArchiveDir, name); - if (log.isDebugEnabled()) - log.debug("Starting to copy WAL segment [absIdx=" + absIdx + ", segIdx=" + segIdx + + if (log.isInfoEnabled()) + log.info("Starting to copy WAL segment [absIdx=" + absIdx + ", segIdx=" + segIdx + ", origFile=" + origFile.getAbsolutePath() + ", dstFile=" + dstFile.getAbsolutePath() + ']'); try { @@ -1665,8 +1665,8 @@ private SegmentArchiveResult archiveSegment(long absIdx) throws IgniteCheckedExc ", dstFile=" + dstTmpFile.getAbsolutePath() + ']', e); } - if (log.isDebugEnabled()) - log.debug("Copied file [src=" + origFile.getAbsolutePath() + + if (log.isInfoEnabled()) + log.info("Copied file [src=" + origFile.getAbsolutePath() + ", dst=" + dstFile.getAbsolutePath() + ']'); return new SegmentArchiveResult(absIdx, origFile, dstFile); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/IgniteMassLoadSandboxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/IgniteMassLoadSandboxTest.java index 76cd5bdc7bf89..7fb277ca9f11e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/IgniteMassLoadSandboxTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/checkpoint/IgniteMassLoadSandboxTest.java @@ -19,6 +19,7 @@ import java.io.File; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Map; import java.util.Random; @@ -29,6 +30,8 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import javax.cache.Cache; import junit.framework.TestCase; import org.apache.ignite.Ignite; @@ -47,9 +50,11 @@ import org.apache.ignite.configuration.WALMode; import org.apache.ignite.internal.IgniteEx; 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.X; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.testframework.GridStringLogger; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; @@ -165,6 +170,7 @@ public class IgniteMassLoadSandboxTest extends GridCommonAbstractTest { /** * Runs multithreaded put scenario (no data streamer). Load is generated to page store and to WAL. + * * @throws Exception if failed. */ public void testContinuousPutMultithreaded() throws Exception { @@ -224,6 +230,7 @@ public void testContinuousPutMultithreaded() throws Exception { /** * Runs multithreaded put scenario (no data streamer). Load is generated to page store and to WAL. + * * @throws Exception if failed. */ public void testDataStreamerContinuousPutMultithreaded() throws Exception { @@ -234,7 +241,6 @@ public void testDataStreamerContinuousPutMultithreaded() throws Exception { System.setProperty(IgniteSystemProperties.IGNITE_OVERRIDE_WRITE_THROTTLING_ENABLED, "speed"); System.setProperty(IgniteSystemProperties.IGNITE_DELAYED_REPLACED_PAGE_WRITE, "true"); - setWalArchAndWorkToSameVal = true; customWalMode = WALMode.BACKGROUND; @@ -243,7 +249,8 @@ public void testDataStreamerContinuousPutMultithreaded() throws Exception { ignite.active(true); - final int threads = 1; Runtime.getRuntime().availableProcessors(); + final int threads = 1; + Runtime.getRuntime().availableProcessors(); final int recsPerThread = CONTINUOUS_PUT_RECS_CNT / threads; @@ -297,6 +304,100 @@ public void testDataStreamerContinuousPutMultithreaded() throws Exception { } } + /** + * Test that WAL segments that are fully covered by checkpoint are logged + * + * @throws Exception if failed. + */ + public void testCoveredWalLogged() throws Exception { + GridStringLogger log0 = null; + + try { + log0 = new GridStringLogger(); + + final IgniteConfiguration cfg = getConfiguration("testCoveredWalLogged"); + + cfg.setGridLogger(log0); + + cfg.getDataStorageConfiguration().setWalAutoArchiveAfterInactivity(10); + + final Ignite ignite = G.start(cfg); + + ignite.cluster().active(true); + + final IgniteCache cache = ignite.cache(CACHE_NAME); + + cache.put(1, new byte[cfg.getDataStorageConfiguration().getWalSegmentSize() - 1024]); + + forceCheckpoint(); + + cache.put(1, new byte[cfg.getDataStorageConfiguration().getWalSegmentSize() - 1024]); + + forceCheckpoint(); + + cache.put(1, new byte[cfg.getDataStorageConfiguration().getWalSegmentSize() - 1024]); + + forceCheckpoint(); + + Thread.sleep(200); // needed by GridStringLogger + + final String log = log0.toString(); + + final String lines[] = log.split("\\r?\\n"); + + final Pattern chPtrn = Pattern.compile("Checkpoint finished"); + + final Pattern idxPtrn = Pattern.compile("idx=([0-9]+),"); + + final Pattern covererdPtrn = Pattern.compile("walSegmentsCovered=\\[(.+)\\], "); + + boolean hasCheckpoint = false; + + long nextCovered = 0; + + for (String line : lines) { + if (!chPtrn.matcher(line).find()) + continue; + + hasCheckpoint = true; + + final Matcher idxMatcher = idxPtrn.matcher(line); + + assertTrue(idxMatcher.find()); + + final long idx = Long.valueOf(idxMatcher.group(1)); + + final Matcher coveredMatcher = covererdPtrn.matcher(line); + + if (!coveredMatcher.find()) { // no wal segments are covered by checkpoint + assertEquals(nextCovered, idx); + continue; + } + + final String coveredMatcherGrp = coveredMatcher.group(1); + + final long[] covered = coveredMatcherGrp.length() > 0 ? + Arrays.stream(coveredMatcherGrp.split(",")).mapToLong(e -> Integer.valueOf(e.trim())).toArray() : + new long[0]; + + assertEquals(nextCovered, covered[0]); + + final long lastCovered = covered[covered.length - 1]; + + assertEquals(idx - 1, lastCovered); // current wal is excluded + + nextCovered = lastCovered + 1; + } + + assertTrue(hasCheckpoint); + + } + finally { + System.out.println(log0 != null ? log0.toString() : "Error initializing GridStringLogger"); + + stopAllGrids(); + } + } /** * Verifies data from storage. @@ -457,14 +558,13 @@ public void testPutRemoveMultithreaded() throws Exception { } } - /** {@inheritDoc} */ @Override protected long getTestTimeout() { return TimeUnit.MINUTES.toMillis(20); } /** Object with additional 40 000 bytes of payload */ - public static class HugeIndexedObject { + public static class HugeIndexedObject { /** Data. */ private byte[] data; /** */ @@ -514,4 +614,4 @@ public byte[] data() { return S.toString(HugeIndexedObject.class, this); } } -} \ No newline at end of file +} From 15f5c93ec3ecfae6edccca695fae066015aa506e Mon Sep 17 00:00:00 2001 From: devozerov Date: Thu, 7 Jun 2018 11:18:08 +0300 Subject: [PATCH 0305/1463] IGNITE-8721: Fixed external security authentication in JDBC and ODBC handlers. This closes #4142. (cherry picked from commit 9d163ed) --- ...ientListenerAbstractConnectionContext.java | 115 ++++++++++++++++++ .../odbc/jdbc/JdbcConnectionContext.java | 36 ++---- .../odbc/odbc/OdbcConnectionContext.java | 33 +---- .../client/ClientConnectionContext.java | 77 +----------- 4 files changed, 137 insertions(+), 124 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerAbstractConnectionContext.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerAbstractConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerAbstractConnectionContext.java new file mode 100644 index 0000000000000..031205e97910d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerAbstractConnectionContext.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.odbc; + +import java.util.Collections; +import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.processors.authentication.AuthorizationContext; +import org.apache.ignite.internal.processors.authentication.IgniteAccessControlException; +import org.apache.ignite.internal.processors.security.SecurityContext; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.plugin.security.AuthenticationContext; +import org.apache.ignite.plugin.security.SecurityCredentials; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.plugin.security.SecuritySubjectType.REMOTE_CLIENT; + +/** + * Base connection context. + */ +public abstract class ClientListenerAbstractConnectionContext implements ClientListenerConnectionContext { + /** Kernal context. */ + protected final GridKernalContext ctx; + + /** Security context or {@code null} if security is disabled. */ + private SecurityContext secCtx; + + /** + * Constructor. + * + * @param ctx Kernal context. + */ + protected ClientListenerAbstractConnectionContext(GridKernalContext ctx) { + this.ctx = ctx; + } + + /** + * @return Kernal context. + */ + public GridKernalContext kernalContext() { + return ctx; + } + + /** + * @return Security context. + */ + @Nullable public SecurityContext securityContext() { + return secCtx; + } + + /** + * Perform authentication. + * + * @return Auth context. + * @throws IgniteCheckedException If failed. + */ + protected AuthorizationContext authenticate(String user, String pwd) throws IgniteCheckedException { + AuthorizationContext authCtx; + + if (ctx.security().enabled()) + authCtx = authenticateExternal(user, pwd).authorizationContext(); + else if (ctx.authentication().enabled()) { + if (F.isEmpty(user)) + throw new IgniteAccessControlException("Unauthenticated sessions are prohibited."); + + authCtx = ctx.authentication().authenticate(user, pwd); + + if (authCtx == null) + throw new IgniteAccessControlException("Unknown authentication error."); + } + else + authCtx = null; + + return authCtx; + } + + /** + * Do 3-rd party authentication. + */ + private AuthenticationContext authenticateExternal(String user, String pwd) throws IgniteCheckedException { + SecurityCredentials cred = new SecurityCredentials(user, pwd); + + AuthenticationContext authCtx = new AuthenticationContext(); + + authCtx.subjectType(REMOTE_CLIENT); + authCtx.subjectId(UUID.randomUUID()); + authCtx.nodeAttributes(Collections.emptyMap()); + authCtx.credentials(cred); + + secCtx = ctx.security().authenticate(authCtx); + + if (secCtx == null) + throw new IgniteAccessControlException( + String.format("The user name or password is incorrect [userName=%s]", user) + ); + + return authCtx; + } +} 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 d3c327b5a50f6..845ac794ef3fc 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 @@ -24,19 +24,18 @@ import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.processors.authentication.AuthorizationContext; -import org.apache.ignite.internal.processors.odbc.ClientListenerConnectionContext; +import org.apache.ignite.internal.processors.odbc.ClientListenerAbstractConnectionContext; 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.odbc.ClientListenerResponse; 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. */ -public class JdbcConnectionContext implements ClientListenerConnectionContext { +public class JdbcConnectionContext extends ClientListenerAbstractConnectionContext { /** Version 2.1.0. */ private static final ClientListenerProtocolVersion VER_2_1_0 = ClientListenerProtocolVersion.create(2, 1, 0); @@ -58,9 +57,6 @@ public class JdbcConnectionContext implements ClientListenerConnectionContext { /** Supported versions. */ private static final Set SUPPORTED_VERS = new HashSet<>(); - /** Context. */ - private final GridKernalContext ctx; - /** Session. */ private final GridNioSession ses; @@ -90,13 +86,15 @@ public class JdbcConnectionContext implements ClientListenerConnectionContext { /** * Constructor. + * * @param ctx Kernal Context. * @param ses Session. * @param busyLock Shutdown busy lock. * @param maxCursors Maximum allowed cursors. */ public JdbcConnectionContext(GridKernalContext ctx, GridNioSession ses, GridSpinBusyLock busyLock, int maxCursors) { - this.ctx = ctx; + super(ctx); + this.ses = ses; this.busyLock = busyLock; this.maxCursors = maxCursors; @@ -135,31 +133,21 @@ public JdbcConnectionContext(GridKernalContext ctx, GridNioSession ses, GridSpin if (ver.compareTo(VER_2_3_0) >= 0) skipReducerOnUpdate = reader.readBoolean(); - AuthorizationContext actx = null; + String user = null; + String passwd = null; try { if (reader.available() > 0) { - String user = reader.readString(); - String passwd = reader.readString(); - - if (ctx.authentication().enabled()) { - if (F.isEmpty(user)) - throw new IgniteCheckedException("Unauthenticated sessions are prohibited"); - - actx = ctx.authentication().authenticate(user, passwd); - - if (actx == null) - throw new IgniteCheckedException("Unknown authentication error"); - } - } - else { - if (ctx.authentication().enabled()) - throw new IgniteCheckedException("Unauthenticated sessions are prohibited"); + user = reader.readString(); + passwd = reader.readString(); } } catch (Exception e) { throw new IgniteCheckedException("Handshake error: " + e.getMessage(), e); } + + AuthorizationContext actx = authenticate(user, passwd); + parser = new JdbcMessageParser(ctx); JdbcResponseSender sender = new JdbcResponseSender() { 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 ef2371e272eb6..e65a7c1297f34 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 @@ -23,17 +23,16 @@ import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.processors.authentication.AuthorizationContext; -import org.apache.ignite.internal.processors.odbc.ClientListenerConnectionContext; +import org.apache.ignite.internal.processors.odbc.ClientListenerAbstractConnectionContext; 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.util.GridSpinBusyLock; -import org.apache.ignite.internal.util.typedef.F; /** * ODBC Connection Context. */ -public class OdbcConnectionContext implements ClientListenerConnectionContext { +public class OdbcConnectionContext extends ClientListenerAbstractConnectionContext { /** Version 2.1.0. */ public static final ClientListenerProtocolVersion VER_2_1_0 = ClientListenerProtocolVersion.create(2, 1, 0); @@ -55,9 +54,6 @@ public class OdbcConnectionContext implements ClientListenerConnectionContext { /** Supported versions. */ private static final Set SUPPORTED_VERS = new HashSet<>(); - /** Context. */ - private final GridKernalContext ctx; - /** Shutdown busy lock. */ private final GridSpinBusyLock busyLock; @@ -85,7 +81,8 @@ public class OdbcConnectionContext implements ClientListenerConnectionContext { * @param maxCursors Maximum allowed cursors. */ public OdbcConnectionContext(GridKernalContext ctx, GridSpinBusyLock busyLock, int maxCursors) { - this.ctx = ctx; + super(ctx); + this.busyLock = busyLock; this.maxCursors = maxCursors; } @@ -127,27 +124,7 @@ public OdbcConnectionContext(GridKernalContext ctx, GridSpinBusyLock busyLock, i passwd = reader.readString(); } - AuthorizationContext actx = null; - - try { - if (ctx.authentication().enabled()) - { - if (F.isEmpty(user)) - throw new IgniteCheckedException("Unauthenticated sessions are prohibited"); - - actx = ctx.authentication().authenticate(user, passwd); - - if (actx == null) - throw new IgniteCheckedException("Unknown authentication error"); - } - else { - if (!F.isEmpty(user)) - throw new IgniteCheckedException("Authentication is disabled for the node."); - } - } - catch (Exception e) { - throw new IgniteCheckedException("Handshake error: " + e.getMessage(), e); - } + AuthorizationContext actx = authenticate(user, passwd); handler = new OdbcRequestHandler(ctx, busyLock, maxCursors, distributedJoins, enforceJoinOrder, replicatedOnly, collocated, lazy, skipReducerOnUpdate, actx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java index 056ea8306fa53..80a561f6ef3ef 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java @@ -20,29 +20,20 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; -import java.util.UUID; +import java.util.concurrent.atomic.AtomicLong; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.processors.authentication.AuthorizationContext; -import org.apache.ignite.internal.processors.authentication.IgniteAccessControlException; -import org.apache.ignite.internal.processors.odbc.ClientListenerConnectionContext; +import org.apache.ignite.internal.processors.odbc.ClientListenerAbstractConnectionContext; 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 java.util.concurrent.atomic.AtomicLong; -import org.apache.ignite.internal.processors.security.SecurityContext; -import org.apache.ignite.plugin.security.AuthenticationContext; -import org.apache.ignite.plugin.security.SecurityCredentials; - -import static org.apache.ignite.plugin.security.SecuritySubjectType.REMOTE_CLIENT; - /** * Thin Client connection context. */ -public class ClientConnectionContext implements ClientListenerConnectionContext { +public class ClientConnectionContext extends ClientListenerAbstractConnectionContext { /** Version 1.0.0. */ public static final ClientListenerProtocolVersion VER_1_0_0 = ClientListenerProtocolVersion.create(1, 0, 0); @@ -61,18 +52,12 @@ public class ClientConnectionContext implements ClientListenerConnectionContext /** Handle registry. */ private final ClientResourceRegistry resReg = new ClientResourceRegistry(); - /** Kernal context. */ - private final GridKernalContext kernalCtx; - /** Max cursors. */ private final int maxCursors; /** Cursor counter. */ private final AtomicLong curCnt = new AtomicLong(); - /** Security context or {@code null} if security is disabled. */ - private SecurityContext secCtx = null; - /** * Ctor. * @@ -80,9 +65,7 @@ public class ClientConnectionContext implements ClientListenerConnectionContext * @param maxCursors Max active cursors. */ public ClientConnectionContext(GridKernalContext ctx, int maxCursors) { - assert ctx != null; - - kernalCtx = ctx; + super(ctx); parser = new ClientMessageParser(ctx); @@ -98,15 +81,6 @@ public ClientResourceRegistry resources() { return resReg; } - /** - * Gets the kernal context. - * - * @return Kernal context. - */ - public GridKernalContext kernalContext() { - return kernalCtx; - } - /** {@inheritDoc} */ @Override public boolean isVersionSupported(ClientListenerProtocolVersion ver) { return SUPPORTED_VERS.contains(ver); @@ -124,7 +98,6 @@ public GridKernalContext kernalContext() { String user = null; String pwd = null; - AuthorizationContext authCtx = null; if (ver.compareTo(VER_1_1_0) >= 0) { try { @@ -140,17 +113,7 @@ public GridKernalContext kernalContext() { } } - if (kernalCtx.security().enabled()) - authCtx = thirdPartyAuthentication(user, pwd).authorizationContext(); - else if (kernalCtx.authentication().enabled()) { - if (user == null || user.length() == 0) - throw new IgniteAccessControlException("Unauthenticated sessions are prohibited."); - - authCtx = kernalCtx.authentication().authenticate(user, pwd); - - if (authCtx == null) - throw new IgniteAccessControlException("Unknown authentication error."); - } + AuthorizationContext authCtx = authenticate(user, pwd); handler = new ClientRequestHandler(this, authCtx); } @@ -192,34 +155,4 @@ public void incrementCursors() { public void decrementCursors() { curCnt.decrementAndGet(); } - - /** - * @return Security context or {@code null} if security is disabled. - */ - public SecurityContext securityContext() { - return secCtx; - } - - /** - * Do 3-rd party authentication. - */ - private AuthenticationContext thirdPartyAuthentication(String user, String pwd) throws IgniteCheckedException { - SecurityCredentials cred = new SecurityCredentials(user, pwd); - - AuthenticationContext authCtx = new AuthenticationContext(); - - authCtx.subjectType(REMOTE_CLIENT); - authCtx.subjectId(UUID.randomUUID()); - authCtx.nodeAttributes(Collections.emptyMap()); - authCtx.credentials(cred); - - secCtx = kernalCtx.security().authenticate(authCtx); - - if (secCtx == null) - throw new IgniteAccessControlException( - String.format("The user name or password is incorrect [userName=%s]", user) - ); - - return authCtx; - } } From 284700780af5a80a29d10c0ae93fc4f22e8d6559 Mon Sep 17 00:00:00 2001 From: "Andrey V. Mashenkov" Date: Wed, 4 Jul 2018 15:14:33 +0300 Subject: [PATCH 0306/1463] IGNITE-8628: Expose list of SQL (ODBC\JDBC\Thin) clients via JMX. This closes #4082. (cherry picked from commit 53f903d) --- .../jdbc/thin/JdbcThinAbstractSelfTest.java | 2 - ...bcThinConnectionMultipleAddressesTest.java | 130 +++++++++++- .../jdbc/thin/JdbcThinConnection.java | 2 + .../internal/jdbc/thin/JdbcThinTcpIo.java | 78 ++++---- ...ientListenerAbstractConnectionContext.java | 21 +- .../odbc/ClientListenerConnectionContext.java | 14 ++ .../odbc/ClientListenerNioListener.java | 22 ++- .../odbc/ClientListenerProcessor.java | 185 +++++++++++++++++- .../odbc/jdbc/JdbcConnectionContext.java | 9 +- .../odbc/odbc/OdbcConnectionContext.java | 5 +- .../client/ClientConnectionContext.java | 5 +- .../ignite/mxbean/ClientProcessorMXBean.java | 55 ++++++ 12 files changed, 465 insertions(+), 63 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/mxbean/ClientProcessorMXBean.java 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 178cd3afba9d4..2ba36c369c227 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 @@ -29,14 +29,12 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.Callable; -import org.apache.ignite.IgniteException; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.processors.odbc.ClientListenerProcessor; import org.apache.ignite.internal.processors.port.GridPortRecord; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import org.jetbrains.annotations.Nullable; /** * Connection test. diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionMultipleAddressesTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionMultipleAddressesTest.java index e1fb295c4d056..12faed7dc81c5 100644 --- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionMultipleAddressesTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionMultipleAddressesTest.java @@ -17,6 +17,7 @@ package org.apache.ignite.jdbc.thin; +import java.lang.management.ManagementFactory; import java.sql.Connection; import java.sql.DatabaseMetaData; import java.sql.DriverManager; @@ -24,12 +25,21 @@ import java.sql.SQLException; import java.sql.Statement; import java.util.ArrayList; +import java.util.List; import java.util.concurrent.Callable; +import javax.management.MBeanServer; +import javax.management.MBeanServerInvocationHandler; +import javax.management.MalformedObjectNameException; +import javax.management.ObjectName; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.ClientConnectorConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.jdbc.thin.JdbcThinTcpIo; +import org.apache.ignite.internal.processors.odbc.ClientListenerProcessor; +import org.apache.ignite.internal.util.lang.GridAbsPredicate; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.mxbean.ClientProcessorMXBean; 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; @@ -82,7 +92,8 @@ private static String url() { cfg.setMarshaller(new BinaryMarshaller()); cfg.setClientConnectorConfiguration( - new ClientConnectorConfiguration().setPort(jdbcPorts.get(getTestIgniteInstanceIndex(name)))); + new ClientConnectorConfiguration() + .setPort(jdbcPorts.get(getTestIgniteInstanceIndex(name)))); return cfg; } @@ -221,8 +232,121 @@ public void testMultipleAddressesOneNodeFailoverOnStreaming() throws Exception { checkReconnectOnStreaming(url(), false); } + /** + * @throws Exception If failed. + */ + public void testClientConnectionMXBean() throws Exception { + Connection conn = DriverManager.getConnection(URL_PORT_RANGE); + + try { + final Statement stmt0 = conn.createStatement(); + + stmt0.execute("SELECT 1"); + + ResultSet rs0 = stmt0.getResultSet(); + + ClientProcessorMXBean serverMxBean = null; + + // Find node which client is connected to. + for (int i = 0; i < NODES_CNT; i++) { + serverMxBean = clientProcessorBean(i); + + if (!serverMxBean.getConnections().isEmpty()) + break; + } + + assertNotNull("No ClientConnections MXBean found.", serverMxBean); + + serverMxBean.dropAllConnections(); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + stmt0.execute("SELECT 1"); + + return null; + } + }, SQLException.class, "Failed to communicate with Ignite cluster"); + + assertTrue(rs0.isClosed()); + assertTrue(stmt0.isClosed()); + + assertTrue(getActiveClients().isEmpty()); + + final Statement stmt1 = conn.createStatement(); + + stmt1.execute("SELECT 1"); + + ResultSet rs1 = stmt1.getResultSet(); + + // Check active clients. + List activeClients = getActiveClients(); + + assertEquals(1, activeClients.size()); + + assertTrue(rs1.next()); + assertEquals(1, rs1.getInt(1)); + + rs1.close(); + stmt1.close(); + } + finally { + conn.close(); + } + + boolean allClosed = GridTestUtils.waitForCondition(new GridAbsPredicate() { + @Override public boolean apply() { + return getActiveClients().isEmpty(); + } + }, 10_000); + + assertTrue(allClosed); + } + + /** + * Return active client list. + * + * @return clients. + */ + @NotNull private List getActiveClients() { + List activeClients = new ArrayList<>(1); + + for (int i = 0; i < NODES_CNT; i++) { + ClientProcessorMXBean mxBean = clientProcessorBean(i); + + assertNotNull(mxBean); + + activeClients.addAll(mxBean.getConnections()); + } + return activeClients; + } + + /** + * Return ClientProcessorMXBean. + * + * @return MBean. + */ + private ClientProcessorMXBean clientProcessorBean(int igniteInt) { + ObjectName mbeanName = null; + + try { + mbeanName = U.makeMBeanName(getTestIgniteInstanceName(igniteInt), "Clients", + ClientListenerProcessor.class.getSimpleName()); + } + catch (MalformedObjectNameException e) { + fail("Failed to register MBean."); + } + + MBeanServer mbeanSrv = ManagementFactory.getPlatformMBeanServer(); + + if (!mbeanSrv.isRegistered(mbeanName)) + fail("MBean is not registered: " + mbeanName.getCanonicalName()); + + return MBeanServerInvocationHandler.newProxyInstance(mbeanSrv, mbeanName, ClientProcessorMXBean.class, true); + } + /** * Check failover on restart cluster ar stop one node. + * * @param url Connection URL. * @param allNodes Restart all nodes flag. * @throws Exception If failed. @@ -254,6 +378,7 @@ private void checkReconnectOnMeta(String url, boolean allNodes) throws Exception /** * Check failover on restart cluster ar stop one node. + * * @param url Connection URL. * @param allNodes Restart all nodes flag. * @throws Exception If failed. @@ -298,6 +423,7 @@ private void checkReconnectOnStatementExecute(String url, boolean allNodes) thro /** * Check failover on restart cluster ar stop one node. + * * @param url Connection URL. * @param allNodes Restart all nodes flag. * @throws Exception If failed. @@ -340,9 +466,9 @@ private void checkReconnectOnResultSet(String url, boolean allNodes) throws Exce } } - /** * Check failover on restart cluster ar stop one node. + * * @param url Connection URL. * @param allNodes Restart all nodes flag. * @throws Exception If failed. 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 633d65753f98e..b36b3194c1a78 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 @@ -145,6 +145,8 @@ private synchronized void ensureConnected() throws SQLException { if (connected) return; + assert !closed; + cliIo.start(); connected = true; 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 2e060954230a5..9dcf74dfbef2d 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 @@ -106,8 +106,8 @@ public class JdbcThinTcpIo { /** Input stream. */ private BufferedInputStream in; - /** Closed flag. */ - private boolean closed; + /** Connected flag. */ + private boolean connected; /** Ignite server version. */ private IgniteProductVersion igniteVer; @@ -160,6 +160,8 @@ public void start(int timeout) throws SQLException, IOException { ownThread = Thread.currentThread(); } + assert !connected; + try { List inaccessibleAddrs = null; @@ -167,8 +169,6 @@ public void start(int timeout) throws SQLException, IOException { HostAndPortRange[] srvs = connProps.getAddresses(); - boolean connected = false; - for (int i = 0; i < srvs.length; i++, srvIdx = (srvIdx + 1) % srvs.length) { HostAndPortRange srv = srvs[srvIdx]; @@ -179,8 +179,6 @@ public void start(int timeout) throws SQLException, IOException { try { connect(new InetSocketAddress(addr, port), timeout); - connected = true; - break; } catch (IOException | SQLException exception) { @@ -238,43 +236,53 @@ else if (ex instanceof IOException) * @throws SQLException On connection reject. */ private void connect(InetSocketAddress addr, int timeout) throws IOException, SQLException { - Socket sock; + Socket sock = null; - if (ConnectionProperties.SSL_MODE_REQUIRE.equalsIgnoreCase(connProps.getSslMode())) - sock = JdbcThinSSLUtil.createSSLSocket(addr, connProps); - else if (ConnectionProperties.SSL_MODE_DISABLE.equalsIgnoreCase(connProps.getSslMode())) { - sock = new Socket(); + try { + if (ConnectionProperties.SSL_MODE_REQUIRE.equalsIgnoreCase(connProps.getSslMode())) + sock = JdbcThinSSLUtil.createSSLSocket(addr, connProps); + else if (ConnectionProperties.SSL_MODE_DISABLE.equalsIgnoreCase(connProps.getSslMode())) { + sock = new Socket(); - try { - sock.connect(addr, timeout); + try { + sock.connect(addr, timeout); + } + catch (IOException e) { + throw new SQLException("Failed to connect to server [host=" + addr.getHostName() + + ", port=" + addr.getPort() + ']', SqlStateCode.CLIENT_CONNECTION_FAILED, e); + } } - catch (IOException e) { - throw new SQLException("Failed to connect to server [host=" + addr.getHostName() + - ", port=" + addr.getPort() + ']', SqlStateCode.CLIENT_CONNECTION_FAILED, e); + else { + throw new SQLException("Unknown sslMode. [sslMode=" + connProps.getSslMode() + ']', + SqlStateCode.CLIENT_CONNECTION_FAILED); } - } - else { - throw new SQLException("Unknown sslMode. [sslMode=" + connProps.getSslMode() + ']', - SqlStateCode.CLIENT_CONNECTION_FAILED); - } - if (connProps.getSocketSendBuffer() != 0) - sock.setSendBufferSize(connProps.getSocketSendBuffer()); + if (connProps.getSocketSendBuffer() != 0) + sock.setSendBufferSize(connProps.getSocketSendBuffer()); - if (connProps.getSocketReceiveBuffer() != 0) - sock.setReceiveBufferSize(connProps.getSocketReceiveBuffer()); + if (connProps.getSocketReceiveBuffer() != 0) + sock.setReceiveBufferSize(connProps.getSocketReceiveBuffer()); - sock.setTcpNoDelay(connProps.isTcpNoDelay()); + sock.setTcpNoDelay(connProps.isTcpNoDelay()); - try { - endpoint = new IpcClientTcpEndpoint(sock); + try { + endpoint = new IpcClientTcpEndpoint(sock); - out = new BufferedOutputStream(endpoint.outputStream()); - in = new BufferedInputStream(endpoint.inputStream()); + out = new BufferedOutputStream(endpoint.outputStream()); + in = new BufferedInputStream(endpoint.inputStream()); + + connected = true; + } + catch (IgniteCheckedException e) { + throw new SQLException("Failed to connect to server [url=" + connProps.getUrl() + ']', + SqlStateCode.CLIENT_CONNECTION_FAILED, e); + } } - catch (IgniteCheckedException e) { - throw new SQLException("Failed to connect to server [url=" + connProps.getUrl() + ']', - SqlStateCode.CLIENT_CONNECTION_FAILED, e); + catch (Exception e) { + if (sock != null && !sock.isClosed()) + U.closeQuiet(sock); + + throw e; } } @@ -601,7 +609,7 @@ private byte[] read(int size) throws IOException { * Close the client IO. */ public void close() { - if (closed) + if (!connected) return; // Clean up resources. @@ -611,7 +619,7 @@ public void close() { if (endpoint != null) endpoint.close(); - closed = true; + connected = false; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerAbstractConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerAbstractConnectionContext.java index 031205e97910d..024f2ffa182fd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerAbstractConnectionContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerAbstractConnectionContext.java @@ -41,13 +41,20 @@ public abstract class ClientListenerAbstractConnectionContext implements ClientL /** Security context or {@code null} if security is disabled. */ private SecurityContext secCtx; + /** Connection ID. */ + private long connId; + + /** Authorization context. */ + private AuthorizationContext authCtx; + /** * Constructor. * * @param ctx Kernal context. */ - protected ClientListenerAbstractConnectionContext(GridKernalContext ctx) { + protected ClientListenerAbstractConnectionContext(GridKernalContext ctx, long connId) { this.ctx = ctx; + this.connId = connId; } /** @@ -64,6 +71,16 @@ public GridKernalContext kernalContext() { return secCtx; } + /** {@inheritDoc} */ + @Nullable @Override public AuthorizationContext authorizationContext() { + return authCtx; + } + + /** {@inheritDoc} */ + @Override public long connectionId() { + return connId; + } + /** * Perform authentication. * @@ -71,8 +88,6 @@ public GridKernalContext kernalContext() { * @throws IgniteCheckedException If failed. */ protected AuthorizationContext authenticate(String user, String pwd) throws IgniteCheckedException { - AuthorizationContext authCtx; - if (ctx.security().enabled()) authCtx = authenticateExternal(user, pwd).authorizationContext(); else if (ctx.authentication().enabled()) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerConnectionContext.java index 28c15046021fa..b693cb6f6ef38 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerConnectionContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerConnectionContext.java @@ -19,11 +19,18 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.binary.BinaryReaderExImpl; +import org.apache.ignite.internal.processors.authentication.AuthorizationContext; +import org.jetbrains.annotations.Nullable; /** * SQL listener connection context. */ public interface ClientListenerConnectionContext { + /** + * @return Current connection id. + */ + long connectionId(); + /** * @param ver Version to check. * @return {@code true} if version is supported. @@ -62,4 +69,11 @@ void initializeFromHandshake(ClientListenerProtocolVersion ver, BinaryReaderExIm * or due to {@code IOException} during network operations. */ void onDisconnected(); + + /** + * Return connection authorization context. + * + * @return authorization context. + */ + @Nullable AuthorizationContext authorizationContext(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java index be55ab9a6210e..eea391cfef0d9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.odbc; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.configuration.ClientConnectorConfiguration; @@ -58,7 +59,10 @@ public class ClientListenerNioListener extends GridNioServerListenerAdapter getConnections() { + Collection sessions = srv.sessions(); + + List res = new ArrayList<>(sessions.size()); + + for (GridNioSession ses : sessions) { + ClientListenerConnectionContext connCtx = ses.meta(CONN_CTX_META_KEY); + + if (connCtx == null || ses.closeTime() != 0) + continue; // Skip non-initialized or closed session. + + String desc = clientConnectionDescription(ses, connCtx); + + res.add(desc); + } + + return res; + } + + /** {@inheritDoc} */ + @Override public void dropAllConnections() { + Collection sessions = srv.sessions(); + + for (GridNioSession ses : sessions) { + ClientListenerConnectionContext connCtx = ses.meta(CONN_CTX_META_KEY); + + if (connCtx == null || ses.closeTime() != 0) + continue; // Skip non-initialized or closed session. + + srv.close(ses); + + log.info("Client session has been dropped: " + clientConnectionDescription(ses, connCtx)); + } + } + + /** {@inheritDoc} */ + @Override public boolean dropConnection(long id) { + assert (id >> 32) == ctx.discovery().localNode().order() : "Invalid connection id."; + + Collection sessions = srv.sessions(); + + for (GridNioSession ses : sessions) { + ClientListenerConnectionContext connCtx = ses.meta(CONN_CTX_META_KEY); + + if (connCtx == null || connCtx.connectionId() != id) + continue; + + if (ses.closeTime() != 0) { + if (log.isDebugEnabled()) + log.debug("Client session is already closed: " + clientConnectionDescription(ses, connCtx)); + + return false; + } + + srv.close(ses); + + log.info("Client session has been dropped: " + clientConnectionDescription(ses, connCtx)); + + return true; + } + + return false; + } + + /** + * Compose connection description string. + * @param ses client NIO session. + * @param ctx client connection context. + * @return connection description + */ + @SuppressWarnings("StringConcatenationInsideStringBufferAppend") + private String clientConnectionDescription(GridNioSession ses, ClientListenerConnectionContext ctx) { + AuthorizationContext authCtx = ctx.authorizationContext(); + + StringBuilder sb = new StringBuilder(); + + if(ctx instanceof JdbcConnectionContext) + sb.append("JdbcClient ["); + else if (ctx instanceof OdbcConnectionContext) + sb.append("OdbcClient ["); + else + sb.append("ThinClient ["); + + InetSocketAddress rmtAddr = ses.remoteAddress(); + InetSocketAddress locAddr = ses.localAddress(); + + assert rmtAddr != null; + assert locAddr != null; + + String rmtAddrStr = rmtAddr.getHostString() + ":" + rmtAddr.getPort(); + String locAddrStr = locAddr.getHostString() + ":" + locAddr.getPort(); + + sb.append("id=" + ctx.connectionId()); + sb.append(", user=").append(authCtx == null ? "" : authCtx.userName()); + sb.append(", rmtAddr=" + rmtAddrStr); + sb.append(", locAddr=" + locAddrStr); + + return sb.append(']').toString(); + } + } +} \ No newline at end of file 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 845ac794ef3fc..2985712095183 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 @@ -86,14 +86,15 @@ public class JdbcConnectionContext extends ClientListenerAbstractConnectionConte /** * Constructor. - * - * @param ctx Kernal Context. + * @param ctx Kernal Context. * @param ses Session. * @param busyLock Shutdown busy lock. + * @param connId * @param maxCursors Maximum allowed cursors. */ - public JdbcConnectionContext(GridKernalContext ctx, GridNioSession ses, GridSpinBusyLock busyLock, int maxCursors) { - super(ctx); + public JdbcConnectionContext(GridKernalContext ctx, GridNioSession ses, GridSpinBusyLock busyLock, long connId, + int maxCursors) { + super(ctx, connId); this.ses = ses; this.busyLock = busyLock; 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 e65a7c1297f34..8d8c745a768ce 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 @@ -78,10 +78,11 @@ public class OdbcConnectionContext extends ClientListenerAbstractConnectionConte * Constructor. * @param ctx Kernal Context. * @param busyLock Shutdown busy lock. + * @param connId * @param maxCursors Maximum allowed cursors. */ - public OdbcConnectionContext(GridKernalContext ctx, GridSpinBusyLock busyLock, int maxCursors) { - super(ctx); + public OdbcConnectionContext(GridKernalContext ctx, GridSpinBusyLock busyLock, long connId, int maxCursors) { + super(ctx, connId); this.busyLock = busyLock; this.maxCursors = maxCursors; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java index 80a561f6ef3ef..1f7cc3c7ade6e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java @@ -62,10 +62,11 @@ public class ClientConnectionContext extends ClientListenerAbstractConnectionCon * Ctor. * * @param ctx Kernal context. + * @param connId Connection ID. * @param maxCursors Max active cursors. */ - public ClientConnectionContext(GridKernalContext ctx, int maxCursors) { - super(ctx); + public ClientConnectionContext(GridKernalContext ctx, long connId, int maxCursors) { + super(ctx, connId); parser = new ClientMessageParser(ctx); diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/ClientProcessorMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/ClientProcessorMXBean.java new file mode 100644 index 0000000000000..d71ec01a2e2cc --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/mxbean/ClientProcessorMXBean.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.mxbean; + +import java.util.List; + +/** + * MXBean interface that provides access to ODBC\JDBC\Thin client connections. + */ +@MXBeanDescription("MBean that provides information about client connections.") +public interface ClientProcessorMXBean { + /** + * Returns list of active connections. + * + * @return Sessions. + */ + @MXBeanDescription("List of client connections.") + List getConnections(); + + /** + * Drop all active connections. + */ + @MXBeanDescription("Drop all client connections.") + void dropAllConnections(); + + /** + * Drops client connection by {@code id}, if exists. + * + * @param id connection id. + * @return {@code True} if connection has been dropped successfully, {@code false} otherwise. + */ + @MXBeanDescription("Drop client connection by ID.") + @MXBeanParametersNames( + "id" + ) + @MXBeanParametersDescriptions( + "Client connection ID." + ) + public boolean dropConnection(long id); +} From 751d4632308b92ed45c061c7650a99333cdbe246 Mon Sep 17 00:00:00 2001 From: dkarachentsev Date: Wed, 4 Jul 2018 19:34:33 +0300 Subject: [PATCH 0307/1463] IGNITE-7163 Validate connection from a pre-previous node. - Fixes #4088. Signed-off-by: Dmitriy Pavlov (cherry picked from commit a837800) --- .../ignite/spi/discovery/tcp/ServerImpl.java | 324 ++++++++++++++- .../spi/discovery/tcp/TcpDiscoveryImpl.java | 7 + .../spi/discovery/tcp/TcpDiscoverySpi.java | 61 ++- .../discovery/tcp/TcpDiscoverySpiMBean.java | 8 + .../tcp/internal/TcpDiscoveryNodesRing.java | 59 ++- .../messages/TcpDiscoveryAbstractMessage.java | 3 + .../TcpDiscoveryHandshakeRequest.java | 23 +- .../TcpDiscoveryHandshakeResponse.java | 23 +- .../IgniteDiscoveryMassiveNodeFailTest.java | 388 ++++++++++++++++++ ...entDiscoverySpiFailureTimeoutSelfTest.java | 18 +- .../TcpDiscoverySegmentationPolicyTest.java | 3 + .../discovery/tcp/TcpDiscoverySelfTest.java | 43 +- .../IgniteSpiDiscoverySelfTestSuite.java | 3 + 13 files changed, 927 insertions(+), 36 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/IgniteDiscoveryMassiveNodeFailTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java index bb76895bd204c..50bb3835507c0 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java @@ -251,6 +251,9 @@ class ServerImpl extends TcpDiscoveryImpl { /** Discovery state. */ protected TcpDiscoverySpiState spiState = DISCONNECTED; + /** Last time received message from ring. */ + private volatile long lastRingMsgReceivedTime; + /** Map with proceeding ping requests. */ private final ConcurrentMap>> pingMap = new ConcurrentHashMap<>(); @@ -312,12 +315,19 @@ class ServerImpl extends TcpDiscoveryImpl { return tcpSrvr.port; } + /** {@inheritDoc} */ + @Override public long connectionCheckInterval() { + return msgWorker.connCheckFreq; + } + /** {@inheritDoc} */ @Override public void spiStart(String igniteInstanceName) throws IgniteSpiException { synchronized (mux) { spiState = DISCONNECTED; } + lastRingMsgReceivedTime = 0; + utilityPool = new IgniteThreadPoolExecutor("disco-pool", spi.ignite().name(), 0, @@ -2726,8 +2736,8 @@ private void initConnectionCheckFrequency() { assert connCheckFreq > 0; - if (log.isDebugEnabled()) - log.debug("Connection check frequency is calculated: " + connCheckFreq); + if (log.isInfoEnabled()) + log.info("Connection check frequency is calculated: " + connCheckFreq); } /** @@ -2905,7 +2915,7 @@ private void sendMessageAcrossRing(TcpDiscoveryAbstractMessage msg) { sendMessageToClients(msg); - Collection failedNodes; + List failedNodes; TcpDiscoverySpiState state; @@ -2921,9 +2931,12 @@ private void sendMessageAcrossRing(TcpDiscoveryAbstractMessage msg) { boolean newNextNode = false; + // Used only if spi.getEffectiveConnectionRecoveryTimeout > 0 + CrossRingMessageSendState sndState = null; + UUID locNodeId = getLocalNodeId(); - while (true) { + ringLoop: while (true) { TcpDiscoveryNode newNext = ring.nextNode(failedNodes); if (newNext == null) { @@ -2947,6 +2960,8 @@ private void sendMessageAcrossRing(TcpDiscoveryAbstractMessage msg) { if (log.isDebugEnabled()) log.debug("New next node [newNext=" + newNext + ", formerNext=" + next + ", ring=" + ring + ", failedNodes=" + failedNodes + ']'); + else if (log.isInfoEnabled()) + log.info("New next node [newNext=" + newNext + ']'); if (debugMode) debugLog(msg, "New next node [newNext=" + newNext + ", formerNext=" + next + @@ -3003,12 +3018,52 @@ else if (log.isTraceEnabled()) openSock = true; // Handshake. - spi.writeToSocket(sock, out, new TcpDiscoveryHandshakeRequest(locNodeId), + TcpDiscoveryHandshakeRequest hndMsg = new TcpDiscoveryHandshakeRequest(locNodeId); + + // Topology treated as changes if next node is not available. + hndMsg.changeTopology(sndState != null && !sndState.isStartingPoint()); + + if (log.isDebugEnabled()) + log.debug("Sending handshake [hndMsg=" + hndMsg + ", sndState=" + sndState + ']'); + + spi.writeToSocket(sock, out, hndMsg, timeoutHelper.nextTimeoutChunk(spi.getSocketTimeout())); TcpDiscoveryHandshakeResponse res = spi.readMessage(sock, null, timeoutHelper.nextTimeoutChunk(ackTimeout0)); + if (log.isDebugEnabled()) + log.debug("Handshake response: " + res); + + if (res.previousNodeAlive() && sndState != null) { + // Remote node checked connection to it's previous and got success. + boolean previousNode = sndState.markLastFailedNodeAlive(); + + if (previousNode) + failedNodes.remove(failedNodes.size() - 1); + else { + newNextNode = false; + + next = ring.nextNode(failedNodes); + } + + U.closeQuiet(sock); + + sock = null; + + if (sndState.isFailed()) { + segmentLocalNodeOnSendFail(); + + return; // Nothing to do here. + } + + if (previousNode) + U.warn(log, "New next node has connection to it's previous, trying previous " + + "again. [next=" + next + ']'); + + continue ringLoop; + } + if (locNodeId.equals(res.creatorNodeId())) { if (log.isDebugEnabled()) log.debug("Handshake response from local node: " + res); @@ -3298,7 +3353,12 @@ else if (e instanceof SocketTimeoutException || } // Iterating node's addresses. if (!sent) { - if (!failedNodes.contains(next)) { + if (sndState == null && spi.getEffectiveConnectionRecoveryTimeout() > 0) + sndState = new CrossRingMessageSendState(); + + boolean failedNextNode = sndState == null || sndState.markNextNodeFailed(); + + if (failedNextNode && !failedNodes.contains(next)) { failedNodes.add(next); if (state == CONNECTED) { @@ -3313,6 +3373,26 @@ else if (e instanceof SocketTimeoutException || ", errMsg=" + (err != null ? err.getMessage() : "N/A") + ']'); } } + else if (!failedNextNode && sndState != null && sndState.isBackward()) { + boolean prev = sndState.markLastFailedNodeAlive(); + + U.warn(log, "Failed to send message to next node, try previous [msg=" + msg + + ", next=" + next + ']'); + + if (prev) + failedNodes.remove(failedNodes.size() - 1); + else { + newNextNode = false; + + next = ring.nextNode(failedNodes); + } + } + + if (sndState != null && sndState.isFailed()) { + segmentLocalNodeOnSendFail(); + + return; // Nothing to do here. + } next = null; @@ -3383,6 +3463,23 @@ else if (e instanceof SocketTimeoutException || } } + /** + * Segment local node on failed message send. + */ + private void segmentLocalNodeOnSendFail() { + U.warn(log, "Unable to connect to next nodes in a ring, " + + "it seems local node is experiencing connectivity issues. Segmenting local node " + + "to avoid case when one node fails a big part of cluster. To disable" + + " that behavior set TcpDiscoverySpi.setConnectionRecoveryTimeout() to 0. " + + "[connRecoveryTimeout=" + spi.connRecoveryTimeout + ", effectiveConnRecoveryTimeout=" + + spi.getEffectiveConnectionRecoveryTimeout() + ']'); + + // Remove any queued messages to avoid new connect tries. + queue.clear(); + + notifyDiscovery(EVT_NODE_SEGMENTED, ring.topologyVersion(), locNode); + } + /** * @param msg Message. * @return Whether to redirect message to client nodes. @@ -5923,6 +6020,60 @@ else if (log.isDebugEnabled()) if (req.client()) res.clientAck(true); + else if (req.changeTopology()) { + // Node cannot connect to it's next (for local node it's previous). + // Need to check connectivity to it. + long rcvdTime = lastRingMsgReceivedTime; + long now = U.currentTimeMillis(); + + // We got message from previous in less than double connection check interval. + boolean ok = rcvdTime + msgWorker.connCheckFreq * 2 >= now; + + if (ok) { + // Check case when previous node suddenly died. This will speed up + // node failing. + Set failed; + + synchronized (mux) { + failed = failedNodes.keySet(); + } + + TcpDiscoveryNode previous = ring.previousNode(failed); + + InetSocketAddress liveAddr = null; + + if (previous != null && !previous.id().equals(nodeId)) { + Collection nodeAddrs = + spi.getNodeAddresses(previous, false); + + for (InetSocketAddress addr : nodeAddrs) { + // Connection refused may be got if node doesn't listen + // (or blocked by firewall, but anyway assume it is dead). + if (!isConnectionRefused(addr)) { + liveAddr = addr; + + break; + } + } + + if (log.isInfoEnabled()) + log.info("Connection check done: [liveAddr=" + liveAddr + + ", previousNode=" + previous + ", addressesToCheck=" + nodeAddrs + + ", connectingNodeId=" + nodeId + ']'); + } + + // If local node was able to connect to previous, confirm that it's alive. + ok = liveAddr != null && (!liveAddr.getAddress().isLoopbackAddress() + || !locNode.socketAddresses().contains(liveAddr)); + } + + res.previousNodeAlive(ok); + + if (log.isInfoEnabled()) { + log.info("Previous node alive: [alive=" + ok + ", lastMessageReceivedTime=" + + rcvdTime + ", now=" + now + ", connCheckFreq=" + msgWorker.connCheckFreq + ']'); + } + } spi.writeToSocket(sock, res, spi.getEffectiveSocketTimeout(srvSock)); @@ -6062,6 +6213,8 @@ else if (e.hasCause(ObjectStreamException.class) || debugLog(msg, "Message has been received: " + msg); if (msg instanceof TcpDiscoveryConnectionCheckMessage) { + ringMessageReceived(); + spi.writeToSocket(msg, sock, RES_OK, sockTimeout); continue; @@ -6239,6 +6392,8 @@ else if (msg instanceof TcpDiscoveryPingResponse) { continue; } else if (msg instanceof TcpDiscoveryRingLatencyCheckMessage) { + ringMessageReceived(); + if (log.isInfoEnabled()) log.info("Latency check message has been read: " + msg.id()); @@ -6249,8 +6404,11 @@ else if (msg instanceof TcpDiscoveryRingLatencyCheckMessage) { if (msg instanceof TcpDiscoveryClientMetricsUpdateMessage) metricsUpdateMsg = (TcpDiscoveryClientMetricsUpdateMessage)msg; - else + else { + ringMessageReceived(); + msgWorker.addMessage(msg); + } // Send receipt back. if (clientMsgWrk != null) { @@ -6338,6 +6496,31 @@ else if (msg instanceof TcpDiscoveryRingLatencyCheckMessage) { } } + /** + * Update last ring message received timestamp. + */ + private void ringMessageReceived() { + lastRingMsgReceivedTime = U.currentTimeMillis(); + } + + /** + * @param addr Address to check. + * @return {@code True} if got connection refused on connect try. + */ + private boolean isConnectionRefused(SocketAddress addr) { + try (Socket sock = new Socket()) { + sock.connect(addr, 100); + } + catch (ConnectException e) { + return true; + } + catch (IOException e) { + return false; + } + + return false; + } + /** * Processes client reconnect message. * @@ -6967,4 +7150,131 @@ public void sock(Socket sock) { this.sock = sock; } } + + /** + * + */ + private enum RingMessageSendState { + /** */ + STARTING_POINT, + + /** */ + FORWARD_PASS, + + /** */ + BACKWARD_PASS, + + /** */ + FAILED + } + + /** + * Initial state is {@link RingMessageSendState#STARTING_POINT}.
    + * States could be switched:
    + * {@link RingMessageSendState#STARTING_POINT} => {@link RingMessageSendState#FORWARD_PASS} when next node failed.
    + * {@link RingMessageSendState#FORWARD_PASS} => {@link RingMessageSendState#FORWARD_PASS} when new next node failed.
    + * {@link RingMessageSendState#FORWARD_PASS} => {@link RingMessageSendState#BACKWARD_PASS} when new next node has + * connection to it's previous node and forces local node to try it again.
    + * {@link RingMessageSendState#BACKWARD_PASS} => {@link RingMessageSendState#BACKWARD_PASS} when previously tried node + * has connection to it's previous and forces local node to try it again.
    + * {@link RingMessageSendState#BACKWARD_PASS} => {@link RingMessageSendState#STARTING_POINT} when local node came back + * to initial next node and no topology changes should be performed.
    + * {@link RingMessageSendState#BACKWARD_PASS} => {@link RingMessageSendState#FAILED} when recovery timeout is over and + * all new next nodes have connections to their previous nodes. That means local node has connectivity + * issue and should be stopped.
    + */ + private class CrossRingMessageSendState { + /** */ + private RingMessageSendState state = RingMessageSendState.STARTING_POINT; + + /** */ + private int failedNodes; + + /** */ + private final long failTime; + + /** + * + */ + CrossRingMessageSendState() { + failTime = spi.getEffectiveConnectionRecoveryTimeout() + U.currentTimeMillis(); + } + + /** + * @return {@code True} if state is {@link RingMessageSendState#STARTING_POINT}. + */ + boolean isStartingPoint() { + return state == RingMessageSendState.STARTING_POINT; + } + + /** + * @return {@code True} if state is {@link RingMessageSendState#BACKWARD_PASS}. + */ + boolean isBackward() { + return state == RingMessageSendState.BACKWARD_PASS; + } + + /** + * @return {@code True} if state is {@link RingMessageSendState#FAILED}. + */ + boolean isFailed() { + return state == RingMessageSendState.FAILED; + } + + /** + * Marks next node as failed. + * + * @return {@code True} node marked as failed. + */ + boolean markNextNodeFailed() { + if (state == RingMessageSendState.STARTING_POINT || state == RingMessageSendState.FORWARD_PASS) { + state = RingMessageSendState.FORWARD_PASS; + + failedNodes++; + + return true; + } + + return false; + } + + /** + * Marks last failed node as alive. + * + * @return {@code False} if all failed nodes marked as alive or incorrect state. + */ + boolean markLastFailedNodeAlive() { + if (state == RingMessageSendState.FORWARD_PASS || state == RingMessageSendState.BACKWARD_PASS) { + state = RingMessageSendState.BACKWARD_PASS; + + if (--failedNodes <= 0) { + failedNodes = 0; + + if (U.currentTimeMillis() >= failTime) { + state = RingMessageSendState.FAILED; + + return false; + } + + state = RingMessageSendState.STARTING_POINT; + + try { + Thread.sleep(200); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + + return true; + } + + return false; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(CrossRingMessageSendState.class, this); + } + } } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java index 00d83dd3ee154..64cc2b956bc5e 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java @@ -247,6 +247,13 @@ public int boundPort() throws IgniteSpiException { return 0; } + /** + * @return connection check interval. + */ + public long connectionCheckInterval() { + return 0; + } + /** * @throws IgniteSpiException If failed. */ diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java index 801f2b6565521..0c7a56f5e9dda 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java @@ -55,9 +55,9 @@ import org.apache.ignite.configuration.AddressResolver; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpiInternalListener; -import org.apache.ignite.internal.IgniteKernal; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.X; @@ -277,6 +277,9 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements IgniteDiscovery /** Maximum ack timeout value for receiving message acknowledgement in milliseconds (value is 600,000ms). */ public static final long DFLT_MAX_ACK_TIMEOUT = 10 * 60 * 1000; + /** Default connection recovery timeout in ms. */ + public static final long DFLT_CONNECTION_RECOVERY_TIMEOUT = IgniteConfiguration.DFLT_FAILURE_DETECTION_TIMEOUT; + /** Ssl message pattern for StreamCorruptedException. */ private static Pattern sslMsgPattern = Pattern.compile("invalid stream header: 150\\d0\\d00"); @@ -311,6 +314,9 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements IgniteDiscovery /** Size of topology snapshots history. */ protected int topHistSize = DFLT_TOP_HISTORY_SIZE; + /** Default connection recovery timeout in ms. */ + protected long connRecoveryTimeout = DFLT_CONNECTION_RECOVERY_TIMEOUT; + /** Grid discovery listener. */ protected volatile DiscoverySpiListener lsnr; @@ -994,6 +1000,54 @@ public TcpDiscoverySpi setTopHistorySize(int topHistSize) { return this; } + /** + * Gets timeout that defines how long server node would try to recovery connection.
    + * See {@link #setConnectionRecoveryTimeout(long)} for details. + * + * @return Timeout that defines how long server node would try to recovery connection. + */ + public long getConnectionRecoveryTimeout() { + return connRecoveryTimeout; + } + + /** + * @return Connection recovery timeout that is not greater than failureDetectionTimeout if enabled. + */ + long getEffectiveConnectionRecoveryTimeout() { + if (failureDetectionTimeoutEnabled() && failureDetectionTimeout() < connRecoveryTimeout) + return failureDetectionTimeout(); + + return connRecoveryTimeout; + } + + /** + * Sets timeout that defines how long server node would try to recovery connection. + *

    In case local node has temporary connectivity issues with part of the cluster, + * it may sequentially fail nodes one-by-one till successfully connect to one that + * has a fine connection with. + * This leads to fail of big number of nodes. + *

    + *

    + * To overcome that issue, local node will do a sequential connection tries to next + * nodes. But if new next node has connection to previous it forces local node to + * retry connect to previous. These tries will last till timeout will not + * finished. When timeout is over, but no success in connecting to nodes it will + * segment itself. + *

    + *

    + * Cannot be greater than {@link #failureDetectionTimeout()}. + *

    + *

    + * Default is {@link #DFLT_CONNECTION_RECOVERY_TIMEOUT}. + *

    + * + * @param connRecoveryTimeout Timeout that defines how long server node would try to recovery connection. + * {@code 0} means node will not recheck failed nodes. + */ + public void setConnectionRecoveryTimeout(long connRecoveryTimeout) { + this.connRecoveryTimeout = connRecoveryTimeout; + } + /** {@inheritDoc} */ @Override public void setNodeAttributes(Map attrs, IgniteProductVersion ver) { assert locNodeAttrs == null; @@ -2437,6 +2491,11 @@ private class TcpDiscoverySpiMBeanImpl extends IgniteSpiMBeanAdapter implements return TcpDiscoverySpi.this.getReconnectCount(); } + /** {@inheritDoc} */ + @Override public long getConnectionCheckInterval() { + return impl.connectionCheckInterval(); + } + /** {@inheritDoc} */ @Override public boolean isClientMode() { return TcpDiscoverySpi.this.isClientMode(); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java index cb0fd362c4a54..176cc07dfd8db 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java @@ -55,6 +55,14 @@ public interface TcpDiscoverySpiMBean extends IgniteSpiManagementMBean, Discover @MXBeanDescription("Reconnect count.") public int getReconnectCount(); + /** + * Gets connection check interval in ms. + * + * @return Number of connection attempts. + */ + @MXBeanDescription("Connection check interval.") + public long getConnectionCheckInterval(); + /** * Gets network timeout. * diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java index 54ddc9ebff686..7fc394b17c30f 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java @@ -17,18 +17,6 @@ package org.apache.ignite.spi.discovery.tcp.internal; -import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.internal.util.tostring.GridToStringExclude; -import org.apache.ignite.internal.util.tostring.GridToStringInclude; -import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.P1; -import org.apache.ignite.internal.util.typedef.PN; -import org.apache.ignite.internal.util.typedef.internal.S; -import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.lang.IgnitePredicate; -import org.apache.ignite.lang.IgniteProductVersion; -import org.jetbrains.annotations.Nullable; - import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -40,6 +28,17 @@ import java.util.UUID; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.P1; +import org.apache.ignite.internal.util.typedef.PN; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.lang.IgniteProductVersion; +import org.jetbrains.annotations.Nullable; /** * Convenient way to represent topology for {@link org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi} @@ -505,6 +504,42 @@ public void clear() { } } + /** + * Finds previous node in the topology filtering excluded nodes from search. + *

    + * This may be used when detecting and handling nodes failure. + * + * @param excluded Nodes to exclude from the search (optional). If provided, + * cannot contain local node. + * @return Previous node or {@code null} if all nodes were filtered out or + * topology contains less than two nodes. + */ + @Nullable public TcpDiscoveryNode previousNode(@Nullable Collection excluded) { + rwLock.readLock().lock(); + + try { + Collection filtered = serverNodes(excluded); + + if (filtered.size() < 2) + return null; + + TcpDiscoveryNode previous = null; + + // Get last node that is previous in a ring + for (TcpDiscoveryNode node : filtered) { + if (locNode.equals(node) && previous != null) + break; + + previous = node; + } + + return previous; + } + finally { + rwLock.readLock().unlock(); + } + } + /** * Gets current topology version. * diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java index a23cb6306a525..93d8bcf9e5e1e 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java @@ -45,6 +45,9 @@ public abstract class TcpDiscoveryAbstractMessage implements Serializable { /** */ protected static final int CLIENT_RECON_SUCCESS_FLAG_POS = 2; + /** */ + protected static final int CHANGE_TOPOLOGY_FLAG_POS = 3; + /** */ protected static final int CLIENT_ACK_FLAG_POS = 4; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeRequest.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeRequest.java index ea5b8688701d9..90c9f945bac7d 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeRequest.java @@ -36,8 +36,29 @@ public TcpDiscoveryHandshakeRequest(UUID creatorNodeId) { super(creatorNodeId); } + /** + * Gets topology change flag.
    + * {@code True} means node intent to fail nodes in a ring. + * + * @return Change topology flag. + */ + public boolean changeTopology() { + return getFlag(CHANGE_TOPOLOGY_FLAG_POS); + } + + /** + * Gets topology change flag.
    + * {@code True} means node intent to fail nodes in a ring. + * + * @param changeTop Change topology flag. + */ + public void changeTopology(boolean changeTop) { + setFlag(CHANGE_TOPOLOGY_FLAG_POS, changeTop); + } + /** {@inheritDoc} */ @Override public String toString() { - return S.toString(TcpDiscoveryHandshakeRequest.class, this, "super", super.toString()); + return S.toString(TcpDiscoveryHandshakeRequest.class, this, "super", super.toString(), + "isChangeTopology", changeTopology()); } } \ No newline at end of file diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java index 0d350af68a568..75df5c23f9311 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java @@ -42,6 +42,26 @@ public TcpDiscoveryHandshakeResponse(UUID creatorNodeId, long locNodeOrder) { order = locNodeOrder; } + /** + * Gets previous node alive flag.
    + * {@code True} means node has connectivity to it's previous node in a ring. + * + * @return previous node alive flag. + */ + public boolean previousNodeAlive() { + return getFlag(CHANGE_TOPOLOGY_FLAG_POS); + } + + /** + * Sets topology change flag.
    + * {@code True} means node has connectivity to it's previous node in a ring. + * + * @param prevNodeAlive previous node alive flag. + */ + public void previousNodeAlive(boolean prevNodeAlive) { + setFlag(CHANGE_TOPOLOGY_FLAG_POS, prevNodeAlive); + } + /** * Gets order of the node sent the response. * @@ -76,6 +96,7 @@ public void clientAck(boolean clientAck) { /** {@inheritDoc} */ @Override public String toString() { - return S.toString(TcpDiscoveryHandshakeResponse.class, this, "super", super.toString()); + return S.toString(TcpDiscoveryHandshakeResponse.class, this, "super", super.toString(), + "isPreviousNodeAlive", previousNodeAlive()); } } \ No newline at end of file diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteDiscoveryMassiveNodeFailTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteDiscoveryMassiveNodeFailTest.java new file mode 100644 index 0000000000000..32ce9783c0332 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteDiscoveryMassiveNodeFailTest.java @@ -0,0 +1,388 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal; + +import java.io.IOException; +import java.io.OutputStream; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.events.DiscoveryEvent; +import org.apache.ignite.events.EventType; +import org.apache.ignite.internal.util.GridConcurrentHashSet; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * Tests checks case when one node is unable to connect to next in a ring, + * but those nodes are not experiencing any connectivity troubles between + * each other. + */ +public class IgniteDiscoveryMassiveNodeFailTest extends GridCommonAbstractTest { + /** */ + private static final int FAILURE_DETECTION_TIMEOUT = 5_000; + + /** */ + private Set failedAddrs = new GridConcurrentHashSet<>(); + + /** */ + private volatile TcpDiscoveryNode compromisedNode; + + /** */ + private volatile boolean forceFailConnectivity; + + /** */ + private volatile boolean failNodes; + + /** */ + private long timeout; + + /** */ + private volatile Set failedNodes = Collections.emptySet(); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + FailDiscoverySpi disco = new FailDiscoverySpi(); + + disco.setIpFinder(LOCAL_IP_FINDER); + + cfg.setDiscoverySpi(disco); + + disco.setConnectionRecoveryTimeout(timeout); + + cfg.setFailureDetectionTimeout(FAILURE_DETECTION_TIMEOUT); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + System.setProperty(IgniteSystemProperties.IGNITE_DUMP_THREADS_ON_FAILURE, "false"); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + System.setProperty(IgniteSystemProperties.IGNITE_DUMP_THREADS_ON_FAILURE, "true"); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + super.afterTest(); + + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + timeout = 2_000; + failNodes = false; + forceFailConnectivity = false; + } + + /** + * Node fails 2 nodes when connection check is disabled. + * + * @throws Exception If failed. + */ + public void testMassiveFailDisabledRecovery() throws Exception { + timeout = 0; // Disable previous node check. + + doFailNodes(false); + } + + /** + * + */ + private void doFailNodes(boolean simulateNodeFailure) throws Exception { + startGrids(5); + + grid(0).events().enabledEvents(); + + failedNodes = new HashSet<>(Arrays.asList(grid(3).cluster().localNode(), grid(4).cluster().localNode())); + + CountDownLatch latch = new CountDownLatch(failedNodes.size()); + + grid(0).events().localListen(e -> { + DiscoveryEvent evt = (DiscoveryEvent)e; + + if (failedNodes.contains(evt.eventNode())) + latch.countDown(); + + return true; + }, EventType.EVT_NODE_FAILED); + + compromisedNode = (TcpDiscoveryNode)grid(2).localNode(); + + for (int i = 3; i < 5; i++) + failedAddrs.addAll(((TcpDiscoveryNode)grid(i).localNode()).socketAddresses()); + + System.out.println(">> Start failing nodes"); + + forceFailConnectivity = true; + + if (simulateNodeFailure) { + for (int i = 3; i < 5; i++) + ((TcpDiscoverySpi)grid(i).configuration().getDiscoverySpi()).simulateNodeFailure(); + } + + assert latch.await(waitTime(), TimeUnit.MILLISECONDS); + + assertEquals(3, grid(0).cluster().forServers().nodes().size()); + } + + /** + * + */ + private long waitTime() { + return timeout + 5000; + } + + /** + * Node fail itself. + * + * @throws Exception If failed. + */ + public void testMassiveFailSelfKill() throws Exception { + startGrids(5); + + grid(0).events().enabledEvents(); + + CountDownLatch latch = new CountDownLatch(1); + + grid(0).events().localListen((e) -> { + DiscoveryEvent evt = (DiscoveryEvent)e; + + if (evt.eventNode().equals(compromisedNode)) + latch.countDown(); + + return true; + }, EventType.EVT_NODE_FAILED); + + compromisedNode = (TcpDiscoveryNode)grid(2).localNode(); + + for (int i = 3; i < 5; i++) + failedAddrs.addAll(((TcpDiscoveryNode)grid(i).localNode()).socketAddresses()); + + System.out.println(">> Start failing nodes"); + + forceFailConnectivity = true; + + assert latch.await(waitTime(), TimeUnit.MILLISECONDS); + + assertEquals(4, grid(0).cluster().forServers().nodes().size()); + } + + /** + * When connectivity restored, no topology changes will be applied. + * + * @throws Exception If failed. + */ + public void testMassiveFailAndRecovery() throws Exception { + startGrids(5); + + grid(0).events().enabledEvents(); + + CountDownLatch latch = new CountDownLatch(1); + + grid(0).events().localListen(e -> { + DiscoveryEvent evt = (DiscoveryEvent)e; + + if (evt.eventNode().equals(compromisedNode)) + latch.countDown(); + + return true; + }, EventType.EVT_NODE_FAILED); + + compromisedNode = (TcpDiscoveryNode)grid(2).localNode(); + + for (int i = 3; i < 5; i++) + failedAddrs.addAll(((TcpDiscoveryNode)grid(i).localNode()).socketAddresses()); + + System.out.println(">> Start failing nodes"); + + forceFailConnectivity = true; + + doSleep(timeout / 4); // wait 1 try + + forceFailConnectivity = false; + + System.out.println(">> Stop failing nodes"); + + assert !latch.await(waitTime(), TimeUnit.MILLISECONDS); + + // Topology is not changed + assertEquals(5, grid(0).cluster().forServers().nodes().size()); + assertEquals(5, grid(0).cluster().topologyVersion()); + } + + /** + * Regular nodes fail by timeout. + * + * @throws Exception If failed. + */ + public void testMassiveFail() throws Exception { + failNodes = true; + + // Must be greater than failureDetectionTimeout / 3 as it calculated into + // connection check frequency. + timeout = FAILURE_DETECTION_TIMEOUT; + + doFailNodes(false); + } + + /** + * Regular node fail by crash. Should be faster due to + * + * + * @throws Exception If failed. + */ + public void testMassiveFailForceNodeFail() throws Exception { + failNodes = true; + + // Must be greater than failureDetectionTimeout / 3 as it calculated into + // connection check frequency. + timeout = FAILURE_DETECTION_TIMEOUT / 2; + + doFailNodes(true); + } + + /** + * Check that cluster recovers from temporal connection breakage. + * + * @throws Exception If failed. + */ + public void testRecoveryOnDisconnect() throws Exception { + startGrids(3); + + IgniteEx ignite1 = grid(1); + IgniteEx ignite2 = grid(2); + + ((TcpDiscoverySpi)ignite1.configuration().getDiscoverySpi()).brakeConnection(); + ((TcpDiscoverySpi)ignite2.configuration().getDiscoverySpi()).brakeConnection(); + + doSleep(FAILURE_DETECTION_TIMEOUT); + + assertEquals(3, grid(0).cluster().nodes().size()); + assertEquals(3, grid(1).cluster().nodes().size()); + assertEquals(3, grid(2).cluster().nodes().size()); + } + + /** + * + */ + private class FailDiscoverySpi extends TcpDiscoverySpi { + /** {@inheritDoc} */ + @Override protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg, byte[] data, + long timeout) throws IOException { + assertNotFailedNode(sock); + + if (isDrop(msg)) + return; + + super.writeToSocket(sock, msg, data, timeout); + } + + /** {@inheritDoc} */ + @Override protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg, + long timeout) throws IOException, IgniteCheckedException { + assertNotFailedNode(sock); + + if (isDrop(msg)) + return; + + super.writeToSocket(sock, msg, timeout); + } + + /** {@inheritDoc} */ + @Override protected void writeToSocket(ClusterNode node, Socket sock, OutputStream out, + TcpDiscoveryAbstractMessage msg, long timeout) throws IOException, IgniteCheckedException { + assertNotFailedNode(sock); + + if (isDrop(msg)) + return; + + super.writeToSocket(node, sock, out, msg, timeout); + } + + /** {@inheritDoc} */ + @Override protected void writeToSocket(Socket sock, OutputStream out, TcpDiscoveryAbstractMessage msg, + long timeout) throws IOException, IgniteCheckedException { + assertNotFailedNode(sock); + + if (isDrop(msg)) + return; + + super.writeToSocket(sock, out, msg, timeout); + } + + /** + * + */ + private boolean isDrop(TcpDiscoveryAbstractMessage msg) { + boolean drop = failNodes && forceFailConnectivity && failedNodes.contains(ignite.cluster().localNode()); + + if (drop) + ignite.log().info(">> Drop message " + msg); + + return drop; + } + + /** {@inheritDoc} */ + @Override protected void writeToSocket(TcpDiscoveryAbstractMessage msg, Socket sock, int res, + long timeout) throws IOException { + assertNotFailedNode(sock); + + if (isDrop(msg)) + return; + + super.writeToSocket(msg, sock, res, timeout); + } + + /** + * @param sock Socket. + * @throws IOException To break connection. + */ + @SuppressWarnings("SuspiciousMethodCalls") + private void assertNotFailedNode(Socket sock) throws IOException { + if (forceFailConnectivity && getLocalNode().equals(compromisedNode) && failedAddrs.contains(sock.getRemoteSocketAddress())) { + log.info(">> Force fail connection " + sock.getRemoteSocketAddress()); + + throw new IOException("Force fail connection " + sock.getRemoteSocketAddress()); + } + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiFailureTimeoutSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiFailureTimeoutSelfTest.java index f1c826ac21853..c167a902d23dd 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiFailureTimeoutSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiFailureTimeoutSelfTest.java @@ -62,6 +62,9 @@ public class TcpClientDiscoverySpiFailureTimeoutSelfTest extends TcpClientDiscov /** */ private static boolean useTestSpi; + /** */ + private static boolean disableTopChangeRecovery; + /** {@inheritDoc} */ @Override protected boolean useFailureDetectionTimeout() { return true; @@ -89,7 +92,19 @@ public class TcpClientDiscoverySpiFailureTimeoutSelfTest extends TcpClientDiscov /** {@inheritDoc} */ @Override protected TcpDiscoverySpi getDiscoverySpi() { - return useTestSpi ? new TestTcpDiscoverySpi2() : super.getDiscoverySpi(); + TcpDiscoverySpi spi = useTestSpi ? new TestTcpDiscoverySpi2() : super.getDiscoverySpi(); + + if (disableTopChangeRecovery) + spi.setConnectionRecoveryTimeout(0); + + return spi; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + disableTopChangeRecovery = false; } /** @@ -202,6 +217,7 @@ public void testFailureTimeout3Server() throws Exception { failureThreshold = 1000; clientFailureDetectionTimeout = 10000; useTestSpi = true; + disableTopChangeRecovery = true; try { startServerNodes(3); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySegmentationPolicyTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySegmentationPolicyTest.java index df76afc2317e4..7d4d802b8d493 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySegmentationPolicyTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySegmentationPolicyTest.java @@ -45,6 +45,9 @@ public class TcpDiscoverySegmentationPolicyTest extends GridCommonAbstractTest { if (igniteInstanceName.endsWith("2")) cfg.setFailureHandler(new TestFailureHandler()); + // Disable recovery + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setConnectionRecoveryTimeout(0); + return cfg; } diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java index d50a967aae172..1aae8fbc3133a 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java @@ -90,6 +90,7 @@ import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.eclipse.jetty.util.ConcurrentHashSet; +import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import static java.util.concurrent.TimeUnit.SECONDS; @@ -1675,15 +1676,15 @@ public void testFailedNodes1() throws Exception { try { final int FAIL_ORDER = 3; - nodeSpi.set(new TestFailedNodesSpi(FAIL_ORDER)); + nodeSpi.set(createFailedNodeSpi(FAIL_ORDER)); final Ignite ignite0 = startGrid(0); - nodeSpi.set(new TestFailedNodesSpi(FAIL_ORDER)); + nodeSpi.set(createFailedNodeSpi(FAIL_ORDER)); startGrid(1); - nodeSpi.set(new TestFailedNodesSpi(FAIL_ORDER)); + nodeSpi.set(createFailedNodeSpi(FAIL_ORDER)); Ignite ignite2 = startGrid(2); @@ -1698,6 +1699,18 @@ public void testFailedNodes1() throws Exception { } } + /** + * @param failOrder Fail order. + * @return Failed node spi. + */ + @NotNull private TestFailedNodesSpi createFailedNodeSpi(int failOrder) { + TestFailedNodesSpi spi = new TestFailedNodesSpi(failOrder); + + spi.setConnectionRecoveryTimeout(0); + + return spi; + } + /** * Coordinator is added in failed list, concurrent nodes start. * @@ -1707,11 +1720,11 @@ public void testFailedNodes2() throws Exception { try { final int FAIL_ORDER = 3; - nodeSpi.set(new TestFailedNodesSpi(FAIL_ORDER)); + nodeSpi.set(createFailedNodeSpi(FAIL_ORDER)); Ignite ignite0 = startGrid(0); - nodeSpi.set(new TestFailedNodesSpi(FAIL_ORDER)); + nodeSpi.set(createFailedNodeSpi(FAIL_ORDER)); startGrid(1); @@ -1721,7 +1734,7 @@ public void testFailedNodes2() throws Exception { @Override public Void call() throws Exception { int idx = nodeIdx.incrementAndGet(); - nodeSpi.set(new TestFailedNodesSpi(FAIL_ORDER)); + nodeSpi.set(createFailedNodeSpi(FAIL_ORDER)); startGrid(idx); @@ -1749,11 +1762,11 @@ public void testFailedNodes2() throws Exception { */ public void testFailedNodes3() throws Exception { try { - nodeSpi.set(new TestFailedNodesSpi(-1)); + nodeSpi.set(createFailedNodeSpi(-1)); Ignite ignite0 = startGrid(0); - nodeSpi.set(new TestFailedNodesSpi(2)); + nodeSpi.set(createFailedNodeSpi(2)); Ignite ignite1 = startGrid(1); @@ -1784,15 +1797,15 @@ public void testFailedNodes4() throws Exception { try { final int FAIL_ORDER = 3; - nodeSpi.set(new TestFailedNodesSpi(FAIL_ORDER)); + nodeSpi.set(createFailedNodeSpi(FAIL_ORDER)); final Ignite ignite0 = startGrid(0); - nodeSpi.set(new TestFailedNodesSpi(FAIL_ORDER)); + nodeSpi.set(createFailedNodeSpi(FAIL_ORDER)); Ignite ignite1 = startGrid(1); - TestFailedNodesSpi spi = new TestFailedNodesSpi(FAIL_ORDER); + TestFailedNodesSpi spi = createFailedNodeSpi(FAIL_ORDER); spi.stopBeforeSndFail = true; @@ -1831,7 +1844,7 @@ public void testFailedNodes5() throws Exception { final int NODES = iter == 0 ? 2 : rnd.nextInt(3, 6); for (int i = 0; i < NODES; i++) { - nodeSpi.set(new TestFailedNodesSpi(-1)); + nodeSpi.set(createFailedNodeSpi(-1)); startGrid(i); } @@ -2084,7 +2097,11 @@ public void testFailedNodeRestoreConnection() throws Exception { TestRestoreConnectedSpi.startTest = false; for (int i = 1; i < 5; i++) { - nodeSpi.set(new TestRestoreConnectedSpi(3)); + TestRestoreConnectedSpi spi = new TestRestoreConnectedSpi(3); + + spi.setConnectionRecoveryTimeout(0); + + nodeSpi.set(spi); startGrid(i); } diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java index ef582a5df7565..1d10b4e057d6c 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java @@ -18,6 +18,7 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; +import org.apache.ignite.internal.IgniteDiscoveryMassiveNodeFailTest; import org.apache.ignite.spi.GridTcpSpiForwardingSelfTest; import org.apache.ignite.spi.discovery.AuthenticationRestartTest; import org.apache.ignite.spi.discovery.FilterDataForClientNodeDiscoveryTest; @@ -106,6 +107,8 @@ public static TestSuite suite() throws Exception { suite.addTest(new TestSuite(TcpDiscoverySpiReconnectDelayTest.class)); + suite.addTest(new TestSuite(IgniteDiscoveryMassiveNodeFailTest.class)); + // Client connect. suite.addTest(new TestSuite(IgniteClientConnectTest.class)); suite.addTest(new TestSuite(IgniteClientReconnectMassiveShutdownTest.class)); From 00b161e2b5a50f9aadb1549ecfe98dd2307b24c6 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 6 Jul 2018 23:00:25 +0300 Subject: [PATCH 0308/1463] IGNITE-8904 Add rebalanceThreadPoolSize to nodes configuration consistency check Signed-off-by: Ivan Rakov (cherry picked from commit b490982) Signed-off-by: Ivan Rakov --- .../apache/ignite/internal/IgniteKernal.java | 2 + .../ignite/internal/IgniteNodeAttributes.java | 3 + .../processors/cache/GridCacheProcessor.java | 24 ++++++++ .../CacheRebalanceConfigValidationTest.java | 55 +++++++++++++++++++ .../testsuites/IgniteBasicTestSuite.java | 3 + 5 files changed, 87 insertions(+) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalanceConfigValidationTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 798644346d79f..2654347433802 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 @@ -258,6 +258,7 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PEER_CLASSLOADING; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PHY_RAM; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PREFIX; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REBALANCE_POOL_SIZE; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_RESTART_ENABLED; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REST_PORT_RANGE; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SPI_CLASS; @@ -1519,6 +1520,7 @@ private void suggestOptimizations(IgniteConfiguration cfg) { */ @SuppressWarnings({"SuspiciousMethodCalls", "unchecked", "TypeMayBeWeakened"}) private void fillNodeAttributes(boolean notifyEnabled) throws IgniteCheckedException { + ctx.addNodeAttribute(ATTR_REBALANCE_POOL_SIZE, configuration().getRebalanceThreadPoolSize()); ctx.addNodeAttribute(ATTR_DATA_STREAMER_POOL_SIZE, configuration().getDataStreamerThreadPoolSize()); final String[] incProps = cfg.getIncludeProperties(); 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 6a4beebac0cba..663a6f9ad244e 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 @@ -199,6 +199,9 @@ public final class IgniteNodeAttributes { /** User authentication enabled flag. */ public static final String ATTR_AUTHENTICATION_ENABLED = ATTR_PREFIX + ".authentication.enabled"; + /** Rebalance thread pool size. */ + public static final String ATTR_REBALANCE_POOL_SIZE = ATTR_PREFIX + ".rebalance.pool.size"; + /** * Enforces singleton. */ 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 d995a88774113..0030ef329d654 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 @@ -927,6 +927,8 @@ private void checkConsistency() throws IgniteCheckedException { if (Boolean.TRUE.equals(n.attribute(ATTR_CONSISTENCY_CHECK_SKIPPED))) continue; + checkRebalanceConfiguration(n); + checkTransactionConfiguration(n); checkMemoryConfiguration(n); @@ -3663,6 +3665,28 @@ private void checkMemoryConfiguration(ClusterNode rmt) throws IgniteCheckedExcep } } + /** + * @param rmt Remote node to check. + * @throws IgniteCheckedException If check failed. + */ + private void checkRebalanceConfiguration(ClusterNode rmt) throws IgniteCheckedException { + ClusterNode locNode = ctx.discovery().localNode(); + + if (ctx.config().isClientMode() || locNode.isDaemon() || rmt.isClient() || rmt.isDaemon()) + return; + + Integer rebalanceThreadPoolSize = rmt.attribute(IgniteNodeAttributes.ATTR_REBALANCE_POOL_SIZE); + + if (rebalanceThreadPoolSize != null && rebalanceThreadPoolSize != ctx.config().getRebalanceThreadPoolSize()) { + throw new IgniteCheckedException("Rebalance configuration mismatch (fix configuration or set -D" + + IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK + "=true system property)." + + " Different values of such parameter may lead to rebalance process instability and hanging. " + + " [rmtNodeId=" + rmt.id() + + ", locRebalanceThreadPoolSize = " + ctx.config().getRebalanceThreadPoolSize() + + ", rmtRebalanceThreadPoolSize = " + rebalanceThreadPoolSize + "]"); + } + } + /** * @param cfg Cache configuration. * @return Query manager. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalanceConfigValidationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalanceConfigValidationTest.java new file mode 100644 index 0000000000000..2f31b31372396 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalanceConfigValidationTest.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; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * + */ +public class CacheRebalanceConfigValidationTest extends GridCommonAbstractTest { + /** Rebalance pool size. */ + private int rebalancePoolSize; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.setRebalanceThreadPoolSize(rebalancePoolSize); + + return cfg; + } + + /** + * Checks that node is not allowed to join to cluster if has different value of {@link IgniteConfiguration#rebalanceThreadPoolSize}. + * + * @throws Exception If failed. + */ + public void testParameterConsistency() throws Exception { + rebalancePoolSize = 2; + + startGrid(0); + + rebalancePoolSize = 1; + + GridTestUtils.assertThrows(log, () -> startGrid(1), IgniteCheckedException.class, "Rebalance configuration mismatch"); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java index a1a45dcda9295..2661b417f4a7d 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java @@ -47,6 +47,7 @@ import org.apache.ignite.internal.managers.IgniteDiagnosticMessagesTest; import org.apache.ignite.internal.processors.affinity.GridAffinityProcessorMemoryLeakTest; import org.apache.ignite.internal.processors.affinity.GridAffinityProcessorRendezvousSelfTest; +import org.apache.ignite.internal.processors.cache.CacheRebalanceConfigValidationTest; import org.apache.ignite.internal.processors.cache.GridLocalIgniteSerializationTest; import org.apache.ignite.internal.processors.cache.GridProjectionForCachesOnDaemonNodeSelfTest; import org.apache.ignite.internal.processors.cache.IgniteDaemonNodeMarshallerCacheTest; @@ -210,6 +211,8 @@ public static TestSuite suite(@Nullable final Set ignoredTests) throws Ex suite.addTestSuite(OomFailureHandlerTest.class); suite.addTestSuite(AccountTransferTransactionTest.class); + suite.addTestSuite(CacheRebalanceConfigValidationTest.class); + return suite; } } From 9a3d08af49cdde86ed1766882543153d70356a7e Mon Sep 17 00:00:00 2001 From: vd-pyatkov Date: Fri, 6 Jul 2018 18:11:55 +0300 Subject: [PATCH 0309/1463] IGNITE-8754 Node outside of baseline does not start when service configured Signed-off-by: Andrey Gura (cherry picked from commit b1832673eba37919ad33f9d1c42b9b6bfa7e5a34) --- .../service/GridServiceProcessor.java | 6 +- .../ServiceDeploymentOutsideBaselineTest.java | 99 ++++++++++++++++++- 2 files changed, 100 insertions(+), 5 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java index b8022a3184c27..7a72fa5a2c4d5 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 @@ -1528,7 +1528,11 @@ private Iterator> serviceEntries(IgniteBiPredicate { + try { + deployServiceFromNewNode(staticDeploy); + } + catch (Exception e) { + fail(e.getMessage()); + } + }); + + try { + startFut.get(10, TimeUnit.SECONDS); + } + catch (IgniteFutureTimeoutCheckedException e) { + GridStringBuilder sb = new SB() + .a("Node can not start out of baseline till ") + .a(10_000L) + .a("ms") + .a(U.nl()); + + for (Thread t: Thread.getAllStackTraces().keySet()) + if (t.getName().startsWith("async-runnable-runner")) + U.printStackTrace(t.getId(), sb); + + fail(sb.toString()); + } + } + /** * @param persistence If {@code true}, then persistence will be enabled. * @param staticDeploy If {@code true}, then static deployment will be used instead of a dynamic one. @@ -251,19 +329,32 @@ private void checkDeployFromNodeRemovedFromBlt(boolean from, boolean staticDeplo * @param staticDeploy If {@code true}, then static deployment will be used instead of a dynamic one. * @throws Exception If node failed to start. */ - private void deployServiceFromNewNode(boolean staticDeploy) throws Exception { + private Ignite deployServiceFromNewNode(boolean staticDeploy) throws Exception { + return deployServiceFromNewNode(staticDeploy, 1); + } + + /** + * @param staticDeploy If {@code true}, then static deployment will be used instead of a dynamic one. + * @param nodeNum Nouber of test node. + * @throws Exception If node failed to start. + */ + private Ignite deployServiceFromNewNode(boolean staticDeploy, int nodeNum) throws Exception { + Ignite ignite; + if (staticDeploy) { srvcCfg = getClusterSingletonServiceConfiguration(); - startGrid(1); + ignite = startGrid(nodeNum); } else { - Ignite node = startGrid(1); + ignite = startGrid(nodeNum); - IgniteFuture depFut = node.services().deployClusterSingletonAsync(SERVICE_NAME, new DummyService()); + IgniteFuture depFut = ignite.services().deployClusterSingletonAsync(SERVICE_NAME, new DummyService()); depFut.get(10, TimeUnit.SECONDS); } + + return ignite; } /** From 3cb46b28bd507d27f46a32027b8654c35b126268 Mon Sep 17 00:00:00 2001 From: Igor Sapego Date: Mon, 9 Jul 2018 19:42:07 +0300 Subject: [PATCH 0310/1463] IGNITE-8495: Thin C++ client. (cherry picked from commit ed658597eb0d2e38c02f75ea48d4c6e38bb315bb) --- .gitignore | 3 +- .../odbc/ClientConnectableNodePartitions.java | 83 ++ .../odbc/ClientListenerProcessor.java | 5 + .../platform/client/ClientMessageParser.java | 16 + .../cache/ClientCacheLocalPeekRequest.java | 50 ++ .../ClientCacheNodePartitionsRequest.java | 75 ++ .../ClientCacheNodePartitionsResponse.java | 54 ++ .../plugin/security/SecurityPermission.java | 2 +- modules/platforms/cpp/DEVNOTES.txt | 7 +- modules/platforms/cpp/Makefile.am | 16 +- modules/platforms/cpp/Makefile.amrel | 6 + modules/platforms/cpp/README.txt | 16 +- .../include/ignite/binary/binary_raw_writer.h | 2 +- .../ignite/impl/binary/binary_reader_impl.h | 7 + .../ignite/impl/binary/binary_type_manager.h | 5 +- .../ignite/impl/binary/binary_writer_impl.h | 26 +- .../impl/interop/interop_output_stream.h | 8 + .../src/impl/binary/binary_object_impl.cpp | 2 + .../src/impl/binary/binary_reader_impl.cpp | 32 +- .../src/impl/binary/binary_type_manager.cpp | 53 +- .../impl/interop/interop_output_stream.cpp | 7 + .../include/ignite/common/platform_utils.h | 6 + .../cpp/common/include/ignite/common/utils.h | 62 ++ .../os/linux/src/common/platform_utils.cpp | 16 +- .../os/win/src/common/platform_utils.cpp | 23 + modules/platforms/cpp/configure.ac | 16 + modules/platforms/cpp/configure.acrel | 14 + .../core-test/project/vs/core-test.vcxproj | 2 +- .../project/vs/core-test.vcxproj.filters | 23 +- .../cpp/core-test/src/cache_query_test.cpp | 7 + .../ignite/cache/mutable_cache_entry.h | 8 +- .../cpp/core/include/ignite/ignite_binding.h | 5 + .../impl/interop/interop_external_memory.h | 2 +- modules/platforms/cpp/core/namespaces.dox | 72 +- modules/platforms/cpp/cpp.dxg | 6 +- modules/platforms/cpp/odbc-test/Makefile.am | 1 + .../cpp/odbc-test/include/test_utils.h | 5 + .../cpp/odbc-test/src/authentication_test.cpp | 12 +- .../cpp/odbc-test/src/queries_test.cpp | 4 + .../cpp/odbc-test/src/test_utils.cpp | 17 + .../cpp/odbc-test/src/utility_test.cpp | 3 +- .../include/ignite/odbc/config/config_tools.h | 1 - .../cpp/odbc/include/ignite/odbc/utility.h | 65 -- .../cpp/odbc/project/vs/odbc.vcxproj | 6 +- .../cpp/odbc/src/config/config_tools.cpp | 13 +- .../src/config/connection_string_parser.cpp | 4 +- modules/platforms/cpp/project/vs/ignite.sln | 23 + .../platforms/cpp/project/vs/ignite.slnrel | 13 + .../cpp/project/vs/ignite_x86.slnrel | 13 + .../cpp/thin-client-test/Makefile.am | 74 ++ .../cpp/thin-client-test/config/auth-32.xml | 48 + .../thin-client-test/config/auth-default.xml | 77 ++ .../cpp/thin-client-test/config/auth.xml | 31 + .../cpp/thin-client-test/config/cache-32.xml | 52 ++ .../thin-client-test/config/cache-default.xml | 141 +++ .../cpp/thin-client-test/config/cache.xml | 35 + .../cpp/thin-client-test/config/ssl-32.xml | 48 + .../thin-client-test/config/ssl-default.xml | 84 ++ .../cpp/thin-client-test/config/ssl.xml | 31 + .../cpp/thin-client-test/config/ssl/ca.pem | 24 + .../config/ssl/client_full.pem | 84 ++ .../config/ssl/client_unknown.pem | 50 ++ .../thin-client-test/config/ssl/server.jks | Bin 0 -> 4256 bytes .../cpp/thin-client-test/config/ssl/trust.jks | Bin 0 -> 1089 bytes .../cpp/thin-client-test/configure.ac | 62 ++ .../cpp/thin-client-test/include/Makefile.am | 24 + .../include/ignite/complex_type.h | 123 +++ .../include/teamcity/teamcity_messages.h | 55 ++ .../cpp/thin-client-test/include/test_utils.h | 72 ++ .../project/vs/thin-client-test.vcxproj | 186 ++++ .../vs/thin-client-test.vcxproj.filters | 82 ++ .../cpp/thin-client-test/src/auth_test.cpp | 76 ++ .../src/cache_client_test.cpp | 743 ++++++++++++++++ .../src/ignite_client_test.cpp | 72 ++ .../cpp/thin-client-test/src/ssl_test.cpp | 103 +++ .../src/teamcity/teamcity_boost.cpp | 159 ++++ .../src/teamcity/teamcity_messages.cpp | 150 ++++ .../cpp/thin-client-test/src/test_utils.cpp | 106 +++ modules/platforms/cpp/thin-client/Makefile.am | 72 ++ modules/platforms/cpp/thin-client/README.md | 4 + .../cpp/thin-client/include/Makefile.am | 31 + .../impl/thin/cache/cache_client_proxy.h | 165 ++++ .../include/ignite/impl/thin/readable.h | 98 +++ .../include/ignite/impl/thin/writable.h | 98 +++ .../include/ignite/impl/thin/writable_key.h | 681 +++++++++++++++ .../include/ignite/thin/cache/cache_client.h | 223 +++++ .../ignite/thin/cache/cache_peek_mode.h | 77 ++ .../include/ignite/thin/ignite_client.h | 157 ++++ .../ignite/thin/ignite_client_configuration.h | 228 +++++ .../include/ignite/thin/ssl_mode.h | 45 + .../os/linux/src/net/net_utils.cpp | 109 +++ .../os/linux/src/net/tcp_socket_client.cpp | 361 ++++++++ .../thin-client/os/win/src/net/net_utils.cpp | 106 +++ .../os/win/src/net/tcp_socket_client.cpp | 417 +++++++++ .../project/vs/thin-client.vcxproj | 213 +++++ .../project/vs/thin-client.vcxproj.filters | 157 ++++ .../cpp/thin-client/src/ignite_client.cpp | 101 +++ .../src/impl/cache/cache_affinity_info.cpp | 110 +++ .../src/impl/cache/cache_affinity_info.h | 106 +++ .../src/impl/cache/cache_client_impl.cpp | 179 ++++ .../src/impl/cache/cache_client_impl.h | 185 ++++ .../src/impl/cache/cache_client_proxy.cpp | 101 +++ .../src/impl/connectable_node_partitions.h | 118 +++ .../cpp/thin-client/src/impl/data_channel.cpp | 387 +++++++++ .../cpp/thin-client/src/impl/data_channel.h | 333 +++++++ .../cpp/thin-client/src/impl/data_router.cpp | 265 ++++++ .../cpp/thin-client/src/impl/data_router.h | 291 +++++++ .../src/impl/ignite_client_impl.cpp | 138 +++ .../thin-client/src/impl/ignite_client_impl.h | 132 +++ .../cpp/thin-client/src/impl/message.cpp | 276 ++++++ .../cpp/thin-client/src/impl/message.h | 820 ++++++++++++++++++ .../cpp/thin-client/src/impl/net/end_point.h | 161 ++++ .../cpp/thin-client/src/impl/net/net_utils.h | 46 + .../src/impl/net/remote_type_updater.cpp | 84 ++ .../src/impl/net/remote_type_updater.h | 81 ++ .../cpp/thin-client/src/impl/net/tcp_range.h | 177 ++++ .../src/impl/net/tcp_socket_client.h | 157 ++++ .../thin-client/src/impl/protocol_version.cpp | 151 ++++ .../thin-client/src/impl/protocol_version.h | 163 ++++ .../thin-client/src/impl/response_status.h | 63 ++ .../cpp/thin-client/src/impl/socket_client.h | 105 +++ .../src/impl/ssl/secure_socket_client.cpp | 445 ++++++++++ .../src/impl/ssl/secure_socket_client.h | 187 ++++ .../thin-client/src/impl/ssl/ssl_bindings.h | 360 ++++++++ .../thin-client/src/impl/ssl/ssl_gateway.cpp | 237 +++++ .../thin-client/src/impl/ssl/ssl_gateway.h | 161 ++++ .../cpp/thin-client/src/impl/utility.cpp | 181 ++++ .../cpp/thin-client/src/impl/utility.h | 82 ++ 128 files changed, 13152 insertions(+), 182 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientConnectableNodePartitions.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheLocalPeekRequest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheNodePartitionsRequest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheNodePartitionsResponse.java create mode 100644 modules/platforms/cpp/thin-client-test/Makefile.am create mode 100644 modules/platforms/cpp/thin-client-test/config/auth-32.xml create mode 100644 modules/platforms/cpp/thin-client-test/config/auth-default.xml create mode 100644 modules/platforms/cpp/thin-client-test/config/auth.xml create mode 100644 modules/platforms/cpp/thin-client-test/config/cache-32.xml create mode 100644 modules/platforms/cpp/thin-client-test/config/cache-default.xml create mode 100644 modules/platforms/cpp/thin-client-test/config/cache.xml create mode 100644 modules/platforms/cpp/thin-client-test/config/ssl-32.xml create mode 100644 modules/platforms/cpp/thin-client-test/config/ssl-default.xml create mode 100644 modules/platforms/cpp/thin-client-test/config/ssl.xml create mode 100644 modules/platforms/cpp/thin-client-test/config/ssl/ca.pem create mode 100644 modules/platforms/cpp/thin-client-test/config/ssl/client_full.pem create mode 100644 modules/platforms/cpp/thin-client-test/config/ssl/client_unknown.pem create mode 100644 modules/platforms/cpp/thin-client-test/config/ssl/server.jks create mode 100644 modules/platforms/cpp/thin-client-test/config/ssl/trust.jks create mode 100644 modules/platforms/cpp/thin-client-test/configure.ac create mode 100644 modules/platforms/cpp/thin-client-test/include/Makefile.am create mode 100644 modules/platforms/cpp/thin-client-test/include/ignite/complex_type.h create mode 100644 modules/platforms/cpp/thin-client-test/include/teamcity/teamcity_messages.h create mode 100644 modules/platforms/cpp/thin-client-test/include/test_utils.h create mode 100644 modules/platforms/cpp/thin-client-test/project/vs/thin-client-test.vcxproj create mode 100644 modules/platforms/cpp/thin-client-test/project/vs/thin-client-test.vcxproj.filters create mode 100644 modules/platforms/cpp/thin-client-test/src/auth_test.cpp create mode 100644 modules/platforms/cpp/thin-client-test/src/cache_client_test.cpp create mode 100644 modules/platforms/cpp/thin-client-test/src/ignite_client_test.cpp create mode 100644 modules/platforms/cpp/thin-client-test/src/ssl_test.cpp create mode 100644 modules/platforms/cpp/thin-client-test/src/teamcity/teamcity_boost.cpp create mode 100644 modules/platforms/cpp/thin-client-test/src/teamcity/teamcity_messages.cpp create mode 100644 modules/platforms/cpp/thin-client-test/src/test_utils.cpp create mode 100644 modules/platforms/cpp/thin-client/Makefile.am create mode 100644 modules/platforms/cpp/thin-client/README.md create mode 100644 modules/platforms/cpp/thin-client/include/Makefile.am create mode 100644 modules/platforms/cpp/thin-client/include/ignite/impl/thin/cache/cache_client_proxy.h create mode 100644 modules/platforms/cpp/thin-client/include/ignite/impl/thin/readable.h create mode 100644 modules/platforms/cpp/thin-client/include/ignite/impl/thin/writable.h create mode 100644 modules/platforms/cpp/thin-client/include/ignite/impl/thin/writable_key.h create mode 100644 modules/platforms/cpp/thin-client/include/ignite/thin/cache/cache_client.h create mode 100644 modules/platforms/cpp/thin-client/include/ignite/thin/cache/cache_peek_mode.h create mode 100644 modules/platforms/cpp/thin-client/include/ignite/thin/ignite_client.h create mode 100644 modules/platforms/cpp/thin-client/include/ignite/thin/ignite_client_configuration.h create mode 100644 modules/platforms/cpp/thin-client/include/ignite/thin/ssl_mode.h create mode 100644 modules/platforms/cpp/thin-client/os/linux/src/net/net_utils.cpp create mode 100644 modules/platforms/cpp/thin-client/os/linux/src/net/tcp_socket_client.cpp create mode 100644 modules/platforms/cpp/thin-client/os/win/src/net/net_utils.cpp create mode 100644 modules/platforms/cpp/thin-client/os/win/src/net/tcp_socket_client.cpp create mode 100644 modules/platforms/cpp/thin-client/project/vs/thin-client.vcxproj create mode 100644 modules/platforms/cpp/thin-client/project/vs/thin-client.vcxproj.filters create mode 100644 modules/platforms/cpp/thin-client/src/ignite_client.cpp create mode 100644 modules/platforms/cpp/thin-client/src/impl/cache/cache_affinity_info.cpp create mode 100644 modules/platforms/cpp/thin-client/src/impl/cache/cache_affinity_info.h create mode 100644 modules/platforms/cpp/thin-client/src/impl/cache/cache_client_impl.cpp create mode 100644 modules/platforms/cpp/thin-client/src/impl/cache/cache_client_impl.h create mode 100644 modules/platforms/cpp/thin-client/src/impl/cache/cache_client_proxy.cpp create mode 100644 modules/platforms/cpp/thin-client/src/impl/connectable_node_partitions.h create mode 100644 modules/platforms/cpp/thin-client/src/impl/data_channel.cpp create mode 100644 modules/platforms/cpp/thin-client/src/impl/data_channel.h create mode 100644 modules/platforms/cpp/thin-client/src/impl/data_router.cpp create mode 100644 modules/platforms/cpp/thin-client/src/impl/data_router.h create mode 100644 modules/platforms/cpp/thin-client/src/impl/ignite_client_impl.cpp create mode 100644 modules/platforms/cpp/thin-client/src/impl/ignite_client_impl.h create mode 100644 modules/platforms/cpp/thin-client/src/impl/message.cpp create mode 100644 modules/platforms/cpp/thin-client/src/impl/message.h create mode 100644 modules/platforms/cpp/thin-client/src/impl/net/end_point.h create mode 100644 modules/platforms/cpp/thin-client/src/impl/net/net_utils.h create mode 100644 modules/platforms/cpp/thin-client/src/impl/net/remote_type_updater.cpp create mode 100644 modules/platforms/cpp/thin-client/src/impl/net/remote_type_updater.h create mode 100644 modules/platforms/cpp/thin-client/src/impl/net/tcp_range.h create mode 100644 modules/platforms/cpp/thin-client/src/impl/net/tcp_socket_client.h create mode 100644 modules/platforms/cpp/thin-client/src/impl/protocol_version.cpp create mode 100644 modules/platforms/cpp/thin-client/src/impl/protocol_version.h create mode 100644 modules/platforms/cpp/thin-client/src/impl/response_status.h create mode 100644 modules/platforms/cpp/thin-client/src/impl/socket_client.h create mode 100644 modules/platforms/cpp/thin-client/src/impl/ssl/secure_socket_client.cpp create mode 100644 modules/platforms/cpp/thin-client/src/impl/ssl/secure_socket_client.h create mode 100644 modules/platforms/cpp/thin-client/src/impl/ssl/ssl_bindings.h create mode 100644 modules/platforms/cpp/thin-client/src/impl/ssl/ssl_gateway.cpp create mode 100644 modules/platforms/cpp/thin-client/src/impl/ssl/ssl_gateway.h create mode 100644 modules/platforms/cpp/thin-client/src/impl/utility.cpp create mode 100644 modules/platforms/cpp/thin-client/src/impl/utility.h diff --git a/.gitignore b/.gitignore index 47220b27df9b8..cd52dfa752478 100644 --- a/.gitignore +++ b/.gitignore @@ -94,6 +94,7 @@ packages /modules/platforms/cpp/missing /modules/platforms/cpp/odbc-test/ignite-odbc-tests /modules/platforms/cpp/stamp-h1 +/modules/platforms/cpp/thin-client-test/ignite-thin-client-tests #Files related to ML manual-runnable tests -/modules/ml/src/test/resources/manualrun/trees/columntrees.manualrun.properties \ No newline at end of file +/modules/ml/src/test/resources/manualrun/trees/columntrees.manualrun.properties diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientConnectableNodePartitions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientConnectableNodePartitions.java new file mode 100644 index 0000000000000..0fbba4ef84b76 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientConnectableNodePartitions.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.odbc; + +import java.util.Collection; +import org.apache.ignite.binary.BinaryRawWriter; + +/** + * Address of the node, connectible for the thin client, associated with cache partitions info. + */ +public class ClientConnectableNodePartitions { + /** Client listener port */ + private final int port; + + /** Addresses. */ + private final Collection addrs; + + /** Cache partitions. */ + private final int[] parts; + + /** + * @param port Client listener port. + * @param addrs Node addresses. + * @param parts Partitions. + */ + public ClientConnectableNodePartitions(int port, Collection addrs, int[] parts) { + this.port = port; + this.addrs = addrs; + this.parts = parts; + } + + /** + * @return Client listener port of the node. + */ + public int getPort() { + return port; + } + + /** + * @return Node's addresses. + */ + public Collection getAddress() { + return addrs; + } + + /** + * @return Cache partitions mapped to the node. + */ + public int[] getPartitions() { + return parts; + } + + /** + * Write using writer. + * @param writer Writer. + */ + public void write(BinaryRawWriter writer) { + writer.writeInt(port); + + writer.writeInt(addrs.size()); + for (String addr : addrs) + writer.writeString(addr); + + writer.writeInt(parts.length); + for (int part : parts) + writer.writeInt(part); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerProcessor.java index 666af92d342ae..c8c026039ac77 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerProcessor.java @@ -64,6 +64,9 @@ public class ClientListenerProcessor extends GridProcessorAdapter { /** Default client connector configuration. */ public static final ClientConnectorConfiguration DFLT_CLI_CFG = new ClientConnectorConfigurationEx(); + /** Client listener port. */ + public static final String CLIENT_LISTENER_PORT = "clientListenerPort"; + /** Default number of selectors. */ private static final int DFLT_SELECTOR_CNT = Math.min(4, Runtime.getRuntime().availableProcessors()); @@ -168,6 +171,8 @@ public ClientListenerProcessor(GridKernalContext ctx) { lastErr = null; + ctx.addNodeAttribute(CLIENT_LISTENER_PORT, port); + break; } catch (Exception e) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientMessageParser.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientMessageParser.java index 057995d885bbf..c887b3ccb7efc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientMessageParser.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientMessageParser.java @@ -52,6 +52,8 @@ import org.apache.ignite.internal.processors.platform.client.cache.ClientCacheGetOrCreateWithNameRequest; import org.apache.ignite.internal.processors.platform.client.cache.ClientCacheGetRequest; import org.apache.ignite.internal.processors.platform.client.cache.ClientCacheGetSizeRequest; +import org.apache.ignite.internal.processors.platform.client.cache.ClientCacheLocalPeekRequest; +import org.apache.ignite.internal.processors.platform.client.cache.ClientCacheNodePartitionsRequest; import org.apache.ignite.internal.processors.platform.client.cache.ClientCachePutAllRequest; import org.apache.ignite.internal.processors.platform.client.cache.ClientCachePutIfAbsentRequest; import org.apache.ignite.internal.processors.platform.client.cache.ClientCachePutRequest; @@ -138,6 +140,9 @@ public class ClientMessageParser implements ClientListenerMessageParser { /** */ private static final short OP_CACHE_GET_SIZE = 1020; + /** */ + private static final short OP_CACHE_LOCAL_PEEK = 1021; + /* Cache create / destroy, configuration. */ /** */ private static final short OP_CACHE_GET_NAMES = 1050; @@ -160,6 +165,11 @@ public class ClientMessageParser implements ClientListenerMessageParser { /** */ private static final short OP_CACHE_DESTROY = 1056; + /* Cache service info. */ + + /** */ + private static final short OP_CACHE_NODE_PARTITIONS = 1100; + /* Query operations. */ /** */ private static final short OP_QUERY_SCAN = 2000; @@ -311,6 +321,9 @@ public ClientListenerRequest decode(BinaryRawReaderEx reader) { case OP_CACHE_REMOVE_KEYS: return new ClientCacheRemoveKeysRequest(reader); + case OP_CACHE_LOCAL_PEEK: + return new ClientCacheLocalPeekRequest(reader); + case OP_CACHE_REMOVE_ALL: return new ClientCacheRemoveAllRequest(reader); @@ -323,6 +336,9 @@ public ClientListenerRequest decode(BinaryRawReaderEx reader) { case OP_CACHE_DESTROY: return new ClientCacheDestroyRequest(reader); + case OP_CACHE_NODE_PARTITIONS: + return new ClientCacheNodePartitionsRequest(reader); + case OP_CACHE_GET_NAMES: return new ClientCacheGetNamesRequest(reader); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheLocalPeekRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheLocalPeekRequest.java new file mode 100644 index 0000000000000..068bbc9792c18 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheLocalPeekRequest.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.platform.client.cache; + +import org.apache.ignite.cache.CachePeekMode; +import org.apache.ignite.internal.binary.BinaryRawReaderEx; +import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; +import org.apache.ignite.internal.processors.platform.client.ClientObjectResponse; +import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; + +/** + * Cache local peek request. + * Only should be used in testing purposes. + */ +public class ClientCacheLocalPeekRequest extends ClientCacheKeyRequest { + /** + * Constructor. + * + * @param reader Reader. + */ + public ClientCacheLocalPeekRequest(BinaryRawReaderEx reader) { + super(reader); + } + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_READ); + + Object val = cache(ctx).localPeek(key(), CachePeekMode.ALL); + + return new ClientObjectResponse(requestId(), val); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheNodePartitionsRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheNodePartitionsRequest.java new file mode 100644 index 0000000000000..b9bf80e60608f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheNodePartitionsRequest.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.platform.client.cache; + +import java.util.ArrayList; +import java.util.Collection; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.binary.BinaryRawReader; +import org.apache.ignite.cache.affinity.Affinity; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.odbc.ClientConnectableNodePartitions; +import org.apache.ignite.internal.processors.odbc.ClientListenerProcessor; +import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; +import org.apache.ignite.internal.processors.platform.client.ClientResponse; +import org.apache.ignite.plugin.security.SecurityPermission; + +/** + * Cluster node list request. + * Currently used to request list of nodes, to calculate affinity on the client side. + */ +public class ClientCacheNodePartitionsRequest extends ClientCacheRequest { + /** + * Initializes a new instance of ClientRawRequest class. + * @param reader Reader. + */ + public ClientCacheNodePartitionsRequest(BinaryRawReader reader) { + super(reader); + } + + /** {@inheritDoc} */ + @Override public ClientResponse process(ClientConnectionContext ctx) { + authorize(ctx, SecurityPermission.CACHE_READ); + IgniteCache cache = cache(ctx); + + GridDiscoveryManager discovery = ctx.kernalContext().discovery(); + Collection nodes = discovery.cacheNodes(cache.getName(), + new AffinityTopologyVersion(discovery.topologyVersion())); + + Affinity aff = ctx.kernalContext().affinity().affinityProxy(cache.getName()); + + ArrayList res = new ArrayList<>(); + + for (ClusterNode node : nodes) { + Integer port = node.attribute(ClientListenerProcessor.CLIENT_LISTENER_PORT); + + if (port == null) + continue; + + Collection addrs = node.addresses(); + + int[] parts = aff.primaryPartitions(node); + + res.add(new ClientConnectableNodePartitions(port, addrs, parts)); + } + + return new ClientCacheNodePartitionsResponse(requestId(), res); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheNodePartitionsResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheNodePartitionsResponse.java new file mode 100644 index 0000000000000..3b6067bcd20ad --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheNodePartitionsResponse.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.platform.client.cache; + +import java.util.Collection; +import org.apache.ignite.internal.binary.BinaryRawWriterEx; +import org.apache.ignite.internal.processors.odbc.ClientConnectableNodePartitions; +import org.apache.ignite.internal.processors.platform.client.ClientResponse; + +/** + * Client cache nodes partitions response. + */ +class ClientCacheNodePartitionsResponse extends ClientResponse { + /** Node partitions. */ + private final Collection nodeParts; + + /** + * @param requestId Request id. + * @param nodeParts Node partitions info. + */ + ClientCacheNodePartitionsResponse(long requestId, Collection nodeParts) { + super(requestId); + + assert nodeParts != null; + + this.nodeParts = nodeParts; + } + + /** {@inheritDoc} */ + @Override public void encode(BinaryRawWriterEx writer) { + super.encode(writer); + + writer.writeInt(nodeParts.size()); + + for (ClientConnectableNodePartitions nodePart : nodeParts) { + nodePart.write(writer); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java index bca667ddb3d7a..a2023516de0a7 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java @@ -87,4 +87,4 @@ public enum SecurityPermission { @Nullable public static SecurityPermission fromOrdinal(int ord) { return ord >= 0 && ord < VALS.length ? VALS[ord] : null; } -} \ No newline at end of file +} diff --git a/modules/platforms/cpp/DEVNOTES.txt b/modules/platforms/cpp/DEVNOTES.txt index 473af23633ac4..132d163f8fcc1 100644 --- a/modules/platforms/cpp/DEVNOTES.txt +++ b/modules/platforms/cpp/DEVNOTES.txt @@ -33,14 +33,19 @@ Among standard included by autotools options configure script also include follo Ignite-specific options: * --enable-odbc - build included ODBC driver. Disabled by default. * --enable-core - build Ignite core library. Enabled by default. + * --enable-thin-client - build thin client library. Enabled by default. * --enable-node - build stand-alone node executable. Enabled by default. If you only need to build ODBC driver and do not want to build anything else you may just use configure script with the following arguments: - ./configure --enable-odbc --disable-core --disable-node + ./configure --enable-odbc --disable-core --disable-thin-client --disable-node With the config like that you should not have any ignite dependencies as jvm.dll installed to build ODBC driver. + +Similarly if you only want to build only thin client you may use configure script with +the following arguments: + ./configure --disable-core --disable-node NOTE: You may want to use "make install" command to install Ignite libraries and headers for your operation system. Note however that this action may require superuser privileges. diff --git a/modules/platforms/cpp/Makefile.am b/modules/platforms/cpp/Makefile.am index 6b643bc2671a1..84712595bd39d 100644 --- a/modules/platforms/cpp/Makefile.am +++ b/modules/platforms/cpp/Makefile.am @@ -17,6 +17,14 @@ ACLOCAL_AMFLAGS =-I m4 +if COND_THIN_CLIENT + MAYBE_THIN_CLIENT = thin-client + +if COND_TESTS + MAYBE_THIN_CLIENT_TESTS = thin-client-test +endif +endif + if COND_ODBC MAYBE_ODBC = odbc @@ -46,9 +54,11 @@ SUBDIRS = \ $(MAYBE_JNI) \ $(MAYBE_ODBC) \ $(MAYBE_CORE) \ + $(MAYBE_THIN_CLIENT) \ $(MAYBE_NODE) \ $(MAYBE_ODBC_TESTS) \ - $(MAYBE_CORE_TESTS) + $(MAYBE_CORE_TESTS) \ + $(MAYBE_THIN_CLIENT_TESTS) DIST_SUBDIRS = \ common \ @@ -56,6 +66,8 @@ DIST_SUBDIRS = \ jni \ odbc \ core \ + thin-client \ ignite \ core-test \ - odbc-test + odbc-test \ + thin-client-test diff --git a/modules/platforms/cpp/Makefile.amrel b/modules/platforms/cpp/Makefile.amrel index ae34b03de93b1..522e2fc7968f2 100644 --- a/modules/platforms/cpp/Makefile.amrel +++ b/modules/platforms/cpp/Makefile.amrel @@ -17,6 +17,10 @@ ACLOCAL_AMFLAGS =-I m4 +if COND_THIN_CLIENT + MAYBE_THIN_CLIENT = thin-client +endif + if COND_ODBC MAYBE_ODBC = odbc endif @@ -38,6 +42,7 @@ SUBDIRS = \ $(MAYBE_JNI) \ $(MAYBE_ODBC) \ $(MAYBE_CORE) \ + $(MAYBE_THIN_CLIENT) \ $(MAYBE_NODE) DIST_SUBDIRS = \ @@ -46,4 +51,5 @@ DIST_SUBDIRS = \ jni \ odbc \ core \ + thin-client \ ignite diff --git a/modules/platforms/cpp/README.txt b/modules/platforms/cpp/README.txt index b1d7eef307bbd..dc740ebc6999a 100644 --- a/modules/platforms/cpp/README.txt +++ b/modules/platforms/cpp/README.txt @@ -31,6 +31,7 @@ Files list: * ignite - executable to start standalone Ignite C++ node. * libignite.so - Ignite C++ API library. * libignite-odbc.so - Ignite ODBC driver. + * libignite-thin-client.so - Ignite C++ thin client library. Development: @@ -54,6 +55,7 @@ Files list: * ignite.exe - executable to start standalone Ignite C++ node. * ignite.core.dll - Ignite C++ API library. * ignite.odbc.dll - Ignite ODBC driver. + * ignite.thin-client.dll - Ignite ODBC driver. Development: @@ -64,7 +66,8 @@ Development: * $(IGNITE_HOME)\platforms\cpp\jni\include * $(IGNITE_HOME)\platforms\cpp\jni\os\win\include * $(IGNITE_HOME)\platforms\cpp\binary\include - * $(IGNITE_HOME)\platforms\cpp\core\include + * $(IGNITE_HOME)\platforms\cpp\core\include to use thick client + * $(IGNITE_HOME)\platforms\cpp\thin-client\include to use thin client * $(JAVA_HOME)\include * $(JAVA_HOME)\include\win32 * Update Library Directories with path to the built binaries @@ -72,9 +75,10 @@ Development: * ignite.common.lib * ignite.jni.lib * ignite.binary.lib - * ignite.core.lib - * Make sure that your application is aware about ignite.jni.dll and ignite.core.dll - libraries. The easiest way to achieve this is to either make sure these files are in - %PATH%, or to put them into the output directory of your project with help of - PostBuild events. + * ignite.core.lib to use thick client + * ignite.thin-client.lib to use thin client + * Make sure that your application is aware about ignite.jni.dll and ignite.core.dll or + ignite.thin-client.dll libraries. The easiest way to achieve this is to either make + sure these files are in %PATH%, or to put them into the output directory of your + project with help of PostBuild events. * To start Apache Ignite as a standalone node or Windows service use ignite.exe diff --git a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_writer.h b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_writer.h index 81e34b0a48e98..9dc73dfffb3ac 100644 --- a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_writer.h +++ b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_writer.h @@ -266,7 +266,7 @@ namespace ignite */ void WriteString(const std::string& val) { - WriteString(val.c_str()); + WriteString(val.c_str(), static_cast(val.size())); } /** 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 4a0e2d43383f2..998c7c35679de 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 @@ -565,6 +565,13 @@ namespace ignite */ int32_t ReadString(char* res, const int32_t len); + /** + * Read string. + * + * @param res String to store result. + */ + void ReadString(std::string& res); + /** * Read string. * diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_manager.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_manager.h index dc147fa1b37c7..4a848ccf01a0c 100644 --- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_manager.h +++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_manager.h @@ -89,10 +89,7 @@ namespace ignite * * @param updater Updater. */ - void SetUpdater(BinaryTypeUpdater* updater) - { - this->updater = updater; - } + void SetUpdater(BinaryTypeUpdater* updater); /** * Get metadata snapshop for the type. diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_writer_impl.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_writer_impl.h index d896f3e658b76..dcfd1dd8ecabf 100644 --- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_writer_impl.h +++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_writer_impl.h @@ -461,7 +461,7 @@ namespace ignite * @param val String. * @param len String length (characters). */ - void WriteString(const char* val, const int32_t len); + void WriteString(const char* val, int32_t len); /** * Write string. @@ -470,12 +470,32 @@ namespace ignite * @param val String. * @param len String length (characters). */ - void WriteString(const char* fieldName, const char* val, const int32_t len); + void WriteString(const char* fieldName, const char* val, int32_t len); + + /** + * Write string. + * + * @param val String. + */ + void WriteString(const std::string& val) + { + WriteString(val.c_str(), static_cast(val.size())); + } + + /** + * Write string. + * + * @param fieldName Field name. + * @param val String. + */ + void WriteString(const char* fieldName, const std::string& val) + { + WriteString(fieldName, val.c_str(), static_cast(val.size())); + } /** * Start string array write. * - * @param typ Collection type. * @return Session ID. */ int32_t WriteStringArray(); diff --git a/modules/platforms/cpp/binary/include/ignite/impl/interop/interop_output_stream.h b/modules/platforms/cpp/binary/include/ignite/impl/interop/interop_output_stream.h index 9df3b1fb4bd2c..810b7c09a7c74 100644 --- a/modules/platforms/cpp/binary/include/ignite/impl/interop/interop_output_stream.h +++ b/modules/platforms/cpp/binary/include/ignite/impl/interop/interop_output_stream.h @@ -143,6 +143,14 @@ namespace ignite */ void WriteInt64(const int64_t val); + /** + * Write signed 64-byte integer. + * + * @param pos Position. + * @param val Value. + */ + void WriteInt64(const int32_t pos, const int64_t val); + /** * Write signed 64-byte integer array. * diff --git a/modules/platforms/cpp/binary/src/impl/binary/binary_object_impl.cpp b/modules/platforms/cpp/binary/src/impl/binary/binary_object_impl.cpp index 38e7670b03ffd..b48e790e0a9cd 100644 --- a/modules/platforms/cpp/binary/src/impl/binary/binary_object_impl.cpp +++ b/modules/platforms/cpp/binary/src/impl/binary/binary_object_impl.cpp @@ -15,6 +15,8 @@ * limitations under the License. */ +#include + #include #include diff --git a/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp b/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp index 6fd5aea409669..5e3788796f2e6 100644 --- a/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp +++ b/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp @@ -38,7 +38,7 @@ namespace ignite BinaryReaderImpl::BinaryReaderImpl(InteropInputStream* stream, BinaryIdResolver* idRslvr, int32_t pos, bool usrType, int32_t typeId, int32_t hashCode, int32_t len, int32_t rawOff, int32_t footerBegin, int32_t footerEnd, BinaryOffsetType::Type schemaType) : - stream(stream), idRslvr(idRslvr), pos(pos), usrType(usrType), typeId(typeId), + stream(stream), idRslvr(idRslvr), pos(pos), usrType(usrType), typeId(typeId), hashCode(hashCode), len(len), rawOff(rawOff), rawMode(false), elemIdGen(0), elemId(0), elemCnt(-1), elemRead(0), footerBegin(footerBegin), footerEnd(footerEnd), schemaType(schemaType) { @@ -55,9 +55,9 @@ namespace ignite int8_t BinaryReaderImpl::ReadInt8() { - return ReadRaw(BinaryUtils::ReadInt8); + return ReadRaw(BinaryUtils::ReadInt8); } - + int32_t BinaryReaderImpl::ReadInt8Array(int8_t* res, const int32_t len) { return ReadRawArray(res, len, BinaryUtils::ReadInt8Array, IGNITE_TYPE_ARRAY_BYTE); @@ -445,6 +445,26 @@ namespace ignite return ReadStringInternal(res, len); } + void BinaryReaderImpl::ReadString(std::string& res) + { + CheckRawMode(true); + CheckSingleMode(true); + + int8_t hdr = stream->ReadInt8(); + + if (hdr == IGNITE_HDR_NULL) + res.clear(); + + if (hdr != IGNITE_TYPE_STRING) + ThrowOnInvalidHeader(IGNITE_TYPE_STRING, hdr); + + int32_t realLen = stream->ReadInt32(); + + res.resize(static_cast(realLen)); + + stream->ReadInt8Array(reinterpret_cast(&res[0]), realLen); + } + int32_t BinaryReaderImpl::ReadString(const char* fieldName, char* res, const int32_t len) { CheckRawMode(false); @@ -526,7 +546,7 @@ namespace ignite return realLen; } else if (hdr != IGNITE_HDR_NULL) - ThrowOnInvalidHeader(IGNITE_TYPE_ARRAY, hdr); + ThrowOnInvalidHeader(IGNITE_TYPE_STRING, hdr); return -1; } @@ -650,7 +670,7 @@ namespace ignite CollectionType::Type BinaryReaderImpl::ReadCollectionType() { InteropStreamPositionGuard positionGuard(*stream); - + return ReadCollectionTypeUnprotected(); } @@ -958,7 +978,7 @@ namespace ignite { int32_t cnt = stream->ReadInt32(); - if (cnt != 0) + if (cnt != 0) { elemId = ++elemIdGen; elemCnt = cnt; diff --git a/modules/platforms/cpp/binary/src/impl/binary/binary_type_manager.cpp b/modules/platforms/cpp/binary/src/impl/binary/binary_type_manager.cpp index 98d4602fd997b..8c8d4f96a5c85 100644 --- a/modules/platforms/cpp/binary/src/impl/binary/binary_type_manager.cpp +++ b/modules/platforms/cpp/binary/src/impl/binary/binary_type_manager.cpp @@ -15,10 +15,13 @@ * limitations under the License. */ +#include +#include +#include + #include #include "ignite/impl/binary/binary_type_manager.h" -#include using namespace ignite::common::concurrent; @@ -90,10 +93,14 @@ namespace ignite bool BinaryTypeManager::ProcessPendingUpdates(IgniteError& err) { + CsLockGuard guard(cs); + if (!updater) - return false; + { + err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, "Updater is not set"); - CsLockGuard guard(cs); + return false; + } for (std::vector::iterator it = pending->begin(); it != pending->end(); ++it) { @@ -103,7 +110,11 @@ namespace ignite continue; // Snapshot has been processed already. if (!updater->Update(*pendingSnap, err)) + { + err = IgniteError(IgniteError::IGNITE_ERR_GENERIC, "Can not send update"); + return false; // Stop as we cannot move further. + } std::map::iterator elem = snapshots->lower_bound(pendingSnap->GetTypeId()); @@ -132,25 +143,33 @@ namespace ignite return true; } + void BinaryTypeManager::SetUpdater(BinaryTypeUpdater* updater) + { + CsLockGuard guard(cs); + + this->updater = updater; + } + SPSnap BinaryTypeManager::GetMeta(int32_t typeId) { - { // Locking scope. - CsLockGuard guard(cs); + CsLockGuard guard(cs); - std::map::iterator it = snapshots->find(typeId); + std::map::iterator it = snapshots->find(typeId); - if (it != snapshots->end() && it->second.Get()) - return it->second; + if (it != snapshots->end() && it->second.Get()) + return it->second; - for (int32_t i = 0; i < pending->size(); ++i) - { - SPSnap& snap = (*pending)[i]; + for (int32_t i = 0; i < pending->size(); ++i) + { + SPSnap& snap = (*pending)[i]; - if (snap.Get()->GetTypeId() == typeId) - return snap; - } + if (snap.Get()->GetTypeId() == typeId) + return snap; } + if (!updater) + throw IgniteError(IgniteError::IGNITE_ERR_BINARY, "Metadata updater is not available."); + IgniteError err; SPSnap snap = updater->GetMeta(typeId, err); @@ -158,11 +177,7 @@ namespace ignite IgniteError::ThrowIfNeeded(err); // Caching meta snapshot for faster access in future. - { // Locking scope. - CsLockGuard guard(cs); - - snapshots->insert(std::make_pair(typeId, snap)); - } + snapshots->insert(std::make_pair(typeId, snap)); return snap; } diff --git a/modules/platforms/cpp/binary/src/impl/interop/interop_output_stream.cpp b/modules/platforms/cpp/binary/src/impl/interop/interop_output_stream.cpp index ab58f1edbbcc1..3e8160d8ecd4b 100644 --- a/modules/platforms/cpp/binary/src/impl/interop/interop_output_stream.cpp +++ b/modules/platforms/cpp/binary/src/impl/interop/interop_output_stream.cpp @@ -141,6 +141,13 @@ namespace ignite IGNITE_INTEROP_OUT_WRITE(val, int64_t, 8); } + void InteropOutputStream::WriteInt64(const int32_t pos, const int64_t val) + { + EnsureCapacity(pos + 8); + + *reinterpret_cast(data + pos) = val; + } + void InteropOutputStream::WriteInt64Array(const int64_t* val, const int32_t len) { IGNITE_INTEROP_OUT_WRITE_ARRAY(val, len << 3); diff --git a/modules/platforms/cpp/common/include/ignite/common/platform_utils.h b/modules/platforms/cpp/common/include/ignite/common/platform_utils.h index a46caba275468..d9d5ec44a5345 100644 --- a/modules/platforms/cpp/common/include/ignite/common/platform_utils.h +++ b/modules/platforms/cpp/common/include/ignite/common/platform_utils.h @@ -91,6 +91,12 @@ namespace ignite */ IGNITE_IMPORT_EXPORT bool IsValidDirectory(const std::string& path); + /** + * Deletes provided filesystem element if exists. + * @return @c true if the provided path exists. + */ + IGNITE_IMPORT_EXPORT bool DeletePath(const std::string& path); + /** * Write file separator to a stream. * @param ostr Stream. diff --git a/modules/platforms/cpp/common/include/ignite/common/utils.h b/modules/platforms/cpp/common/include/ignite/common/utils.h index 792bd60b7e78a..81b5432cb57d6 100644 --- a/modules/platforms/cpp/common/include/ignite/common/utils.h +++ b/modules/platforms/cpp/common/include/ignite/common/utils.h @@ -23,6 +23,7 @@ #include #include #include +#include #include #include @@ -71,6 +72,67 @@ namespace ignite */ IGNITE_IMPORT_EXPORT void StripSurroundingWhitespaces(std::string& str); + /** + * Skip leading spaces. + * + * @param begin Iterator to the beginning of the character sequence. + * @param end Iterator to the end of the character sequence. + * @return Iterator to first non-blanc character. + */ + template + Iterator SkipLeadingSpaces(Iterator begin, Iterator end) + { + Iterator res = begin; + + while (isspace(*res) && res != end) + ++res; + + return res; + } + + /** + * Skip trailing spaces. + * + * @param begin Iterator to the beginning of the character sequence. + * @param end Iterator to the end of the character sequence. + * @return Iterator to last non-blanc character. + */ + template + Iterator SkipTrailingSpaces(Iterator begin, Iterator end) + { + Iterator res = end - 1; + + while (isspace(*res) && res != begin - 1) + --res; + + return res + 1; + } + + /** + * Remove leading and trailing spaces. + * + * @param begin Iterator to the beginning of the character sequence. + * @param end Iterator to the end of the character sequence. + * @return String without leading and trailing spaces. + */ + template + std::string StripSurroundingWhitespaces(Iterator begin, Iterator end) + { + std::string res; + + if (begin >= end) + return res; + + Iterator skipped_leading = SkipLeadingSpaces(begin, end); + Iterator skipped_trailing = SkipTrailingSpaces(skipped_leading, end); + + res.reserve(skipped_trailing - skipped_leading); + + std::copy(skipped_leading, skipped_trailing, std::back_inserter(res)); + + return res; + } + /** * Get string representation of long in decimal form. * diff --git a/modules/platforms/cpp/common/os/linux/src/common/platform_utils.cpp b/modules/platforms/cpp/common/os/linux/src/common/platform_utils.cpp index 2cbbd1d19971b..cc8970f74ed9e 100644 --- a/modules/platforms/cpp/common/os/linux/src/common/platform_utils.cpp +++ b/modules/platforms/cpp/common/os/linux/src/common/platform_utils.cpp @@ -15,7 +15,8 @@ * limitations under the License. */ -#include +#include +#include #include #include @@ -23,6 +24,7 @@ #include #include #include +#include #include @@ -92,6 +94,18 @@ namespace ignite return stat(path.c_str(), &pathStat) != -1 && S_ISDIR(pathStat.st_mode); } + static int rmFiles(const char *pathname, const struct stat *sbuf, int type, struct FTW *ftwb) + { + remove(pathname); + + return 0; + } + + bool DeletePath(const std::string& path) + { + return nftw(path.c_str(), rmFiles, 10, FTW_DEPTH | FTW_MOUNT | FTW_PHYS) == 0; + } + StdCharOutStream& Fs(StdCharOutStream& ostr) { ostr.put('/'); diff --git a/modules/platforms/cpp/common/os/win/src/common/platform_utils.cpp b/modules/platforms/cpp/common/os/win/src/common/platform_utils.cpp index c2244c53af453..b3f403d1da44d 100644 --- a/modules/platforms/cpp/common/os/win/src/common/platform_utils.cpp +++ b/modules/platforms/cpp/common/os/win/src/common/platform_utils.cpp @@ -16,6 +16,7 @@ */ #include +#include #include @@ -92,6 +93,28 @@ namespace ignite return attrs != INVALID_FILE_ATTRIBUTES && (attrs & FILE_ATTRIBUTE_DIRECTORY) != 0; } + bool DeletePath(const std::string& path) + { + std::vector path0(path.begin(), path.end()); + path0.push_back('\0'); + path0.push_back('\0'); + + SHFILEOPSTRUCT fileop; + fileop.hwnd = NULL; + fileop.wFunc = FO_DELETE; + fileop.pFrom = &path0[0]; + fileop.pTo = NULL; + fileop.fFlags = FOF_NOCONFIRMATION | FOF_SILENT; + + fileop.fAnyOperationsAborted = FALSE; + fileop.lpszProgressTitle = NULL; + fileop.hNameMappings = NULL; + + int ret = SHFileOperation(&fileop); + + return ret == 0; + } + StdCharOutStream& Fs(StdCharOutStream& ostr) { ostr.put('\\'); diff --git a/modules/platforms/cpp/configure.ac b/modules/platforms/cpp/configure.ac index d7fae965525b6..59581635d17ff 100644 --- a/modules/platforms/cpp/configure.ac +++ b/modules/platforms/cpp/configure.ac @@ -59,6 +59,17 @@ AC_ARG_ENABLE([core], [core=true] ) +AC_ARG_ENABLE([thin-client], + [AS_HELP_STRING([--enable-thin-client],[build Thin C++ client [default=yes]])], + [ case "${enableval}" in + yes) thin_client=true ;; + no) thin_client=false ;; + *) AC_MSG_ERROR(bad value ${enableval} for --enable-thin-client) ;; + esac + ], + [thin_client=true] +) + AC_ARG_ENABLE([node], [AS_HELP_STRING([--enable-node],[build stand-alone node binary [default=yes]])], [ case "${enableval}" in @@ -81,6 +92,7 @@ AC_ARG_ENABLE([tests], AM_CONDITIONAL([COND_ODBC], [test "x$odbc" = "xtrue"]) AM_CONDITIONAL([COND_CORE], [test "x$core" = "xtrue"]) +AM_CONDITIONAL([COND_THIN_CLIENT], [test "x$thin_client" = "xtrue"]) AM_CONDITIONAL([COND_NODE], [test "x$node" = "xtrue"]) AM_CONDITIONAL([COND_TESTS], [test "x$tests" = "xtrue"]) @@ -102,6 +114,10 @@ AC_CONFIG_FILES([ \ core/ignite.pc \ jni/include/Makefile \ jni/Makefile \ + thin-client/include/Makefile \ + thin-client/Makefile \ + thin-client-test/include/Makefile \ + thin-client-test/Makefile \ ignite/Makefile \ ]) diff --git a/modules/platforms/cpp/configure.acrel b/modules/platforms/cpp/configure.acrel index 8d9479b19befa..d617c886c04c2 100644 --- a/modules/platforms/cpp/configure.acrel +++ b/modules/platforms/cpp/configure.acrel @@ -59,6 +59,17 @@ AC_ARG_ENABLE([core], [core=true] ) +AC_ARG_ENABLE([thin-client], + [AS_HELP_STRING([--enable-thin-client],[build Thin C++ client [default=yes]])], + [ case "${enableval}" in + yes) thin_client=true ;; + no) thin_client=false ;; + *) AC_MSG_ERROR(bad value ${enableval} for --enable-thin-client) ;; + esac + ], + [thin_client=true] +) + AC_ARG_ENABLE([node], [AS_HELP_STRING([--enable-node],[build stand-alone node binary [default=yes]])], [ case "${enableval}" in @@ -71,6 +82,7 @@ AC_ARG_ENABLE([node], AM_CONDITIONAL([COND_ODBC], [test "x$odbc" = "xtrue"]) AM_CONDITIONAL([COND_CORE], [test "x$core" = "xtrue"]) +AM_CONDITIONAL([COND_THIN_CLIENT], [test "x$thin_client" = "xtrue"]) AM_CONDITIONAL([COND_NODE], [test "x$node" = "xtrue"]) AC_CONFIG_FILES([ \ @@ -87,6 +99,8 @@ AC_CONFIG_FILES([ \ core/ignite.pc \ jni/include/Makefile \ jni/Makefile \ + thin-client/include/Makefile \ + thin-client/Makefile \ ignite/Makefile \ ]) diff --git a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj index 0b0166f30511f..f8dee2f4c6241 100644 --- a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj +++ b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj @@ -219,4 +219,4 @@ - \ No newline at end of file + diff --git a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters index b16877c875d49..3d6e01ab9102a 100644 --- a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters +++ b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters @@ -125,39 +125,36 @@ {fb43524e-3694-44ee-b153-770cd9cf6c7a} - - {fb43524e-3694-44ee-b153-770cd9cf6c7a} - - + Configs Configs - + Configs - + Configs Configs - + Configs - + Configs Configs - + Configs - + Configs @@ -169,15 +166,15 @@ Configs - - Configs - Configs Configs + + Configs + Configs diff --git a/modules/platforms/cpp/core-test/src/cache_query_test.cpp b/modules/platforms/cpp/core-test/src/cache_query_test.cpp index 8375bad440dc9..d5a361751ddc7 100644 --- a/modules/platforms/cpp/core-test/src/cache_query_test.cpp +++ b/modules/platforms/cpp/core-test/src/cache_query_test.cpp @@ -34,6 +34,7 @@ #include "ignite/ignite.h" #include "ignite/ignition.h" #include "ignite/test_utils.h" +#include "teamcity_messages.h" using namespace boost::unit_test; @@ -963,6 +964,9 @@ BOOST_AUTO_TEST_CASE(TestSqlQuery) */ BOOST_AUTO_TEST_CASE(TestSqlQueryDistributedJoins) { + if (JetBrains::underTeamcity()) + return; + Cache cache1 = GetPersonCache(); Cache cache2 = GetRelationCache(); @@ -1241,6 +1245,9 @@ BOOST_AUTO_TEST_CASE(TestSqlFieldsQueryBasic) */ BOOST_AUTO_TEST_CASE(TestSqlFieldsQueryDistributedJoins) { + if (JetBrains::underTeamcity()) + return; + Cache cache1 = GetPersonCache(); Cache cache2 = GetRelationCache(); diff --git a/modules/platforms/cpp/core/include/ignite/cache/mutable_cache_entry.h b/modules/platforms/cpp/core/include/ignite/cache/mutable_cache_entry.h index 0481a5eb4223d..543e3cd25662f 100644 --- a/modules/platforms/cpp/core/include/ignite/cache/mutable_cache_entry.h +++ b/modules/platforms/cpp/core/include/ignite/cache/mutable_cache_entry.h @@ -15,12 +15,14 @@ * limitations under the License. */ +/** + * @file + * Declares ignite::cache::MutableCacheEntry class template. + */ + #ifndef _IGNITE_CACHE_MUTABLE_CACHE_ENTRY #define _IGNITE_CACHE_MUTABLE_CACHE_ENTRY -#include -#include - namespace ignite { namespace cache diff --git a/modules/platforms/cpp/core/include/ignite/ignite_binding.h b/modules/platforms/cpp/core/include/ignite/ignite_binding.h index fe060c1d95f3a..8ff5ad973c475 100644 --- a/modules/platforms/cpp/core/include/ignite/ignite_binding.h +++ b/modules/platforms/cpp/core/include/ignite/ignite_binding.h @@ -15,6 +15,11 @@ * limitations under the License. */ +/** + * @file + * Declares ignite::IgniteBinding class. + */ + #ifndef _IGNITE_IGNITE_BINDING #define _IGNITE_IGNITE_BINDING diff --git a/modules/platforms/cpp/core/include/ignite/impl/interop/interop_external_memory.h b/modules/platforms/cpp/core/include/ignite/impl/interop/interop_external_memory.h index 04d2e988a8a35..7daed35e50614 100644 --- a/modules/platforms/cpp/core/include/ignite/impl/interop/interop_external_memory.h +++ b/modules/platforms/cpp/core/include/ignite/impl/interop/interop_external_memory.h @@ -33,7 +33,7 @@ namespace ignite /** * Interop external memory. */ - class IGNITE_IMPORT_EXPORT InteropExternalMemory : public interop::InteropMemory + class IGNITE_IMPORT_EXPORT InteropExternalMemory : public InteropMemory { public: /** diff --git a/modules/platforms/cpp/core/namespaces.dox b/modules/platforms/cpp/core/namespaces.dox index 49379e642fb9e..60e8cf743a108 100644 --- a/modules/platforms/cpp/core/namespaces.dox +++ b/modules/platforms/cpp/core/namespaces.dox @@ -26,29 +26,29 @@ /** * Apache %Ignite API. */ - namespace ignite - { - /** - * %Ignite Binary Objects API. - */ - namespace binary - { - // Empty. - } +namespace ignite +{ + /** + * %Ignite Binary Objects API. + */ + namespace binary + { + // Empty. + } - /** - * %Ignite %Transaction API. - */ - namespace transactions - { - // Empty. - } + /** + * %Ignite %Transaction API. + */ + namespace transactions + { + // Empty. + } - /** - * %Ignite %Cache API. - */ - namespace cache - { + /** + * %Ignite %Cache API. + */ + namespace cache + { /** * Contains APIs for cache events. */ @@ -57,11 +57,11 @@ // Empty. } - /** - * Contains APIs for creating and executing cache queries. - */ - namespace query - { + /** + * Contains APIs for creating and executing cache queries. + */ + namespace query + { /** * Contains APIs for continuous queries. */ @@ -69,7 +69,21 @@ { // Empty. } - } - } - } + } + } + + /** + * %Ignite Thin Client API. + */ + namespace thin + { + /** + * %Ignite Thin Client Cache API. + */ + namespace cache + { + // Empty. + } + } +} diff --git a/modules/platforms/cpp/cpp.dxg b/modules/platforms/cpp/cpp.dxg index d25d9781c5f0e..2f571f86b7f70 100644 --- a/modules/platforms/cpp/cpp.dxg +++ b/modules/platforms/cpp/cpp.dxg @@ -1715,8 +1715,8 @@ GENERATE_LEGEND = YES DOT_CLEANUP = YES -;INPUT=core binary common -;EXCLUDE=core/include/ignite/impl core/os/linux/include/ignite/impl core/os/linux/src/impl core/os/win/include/ignite/impl core/os/win/src/impl core/src/impl binary/include/ignite/impl binary/src/impl common/include/ignite/common common/os -;STRIP_FROM_PATH=core/include/ignite core/src binary/include/ignite binary/src +;INPUT=core binary common thin-client +;EXCLUDE=core/include/ignite/impl core/os/linux/include/ignite/impl core/os/linux/src/impl core/os/win/include/ignite/impl core/os/win/src/impl core/src/impl binary/include/ignite/impl binary/src/impl common/include/ignite/common common/os thin-client/include/ignite/impl thin-client/src +;STRIP_FROM_PATH=core/include/ignite core/src binary/include/ignite binary/src thin-client/src ;OUTPUT_DIRECTORY=../../clients/target/cppdoc ;PROJECT_LOGO=../../../assembly/docfiles/ignite_logo.png diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am index 89e27ddb4bc55..87e3c893398c8 100644 --- a/modules/platforms/cpp/odbc-test/Makefile.am +++ b/modules/platforms/cpp/odbc-test/Makefile.am @@ -84,6 +84,7 @@ ignite_odbc_tests_SOURCES = \ src/errors_test.cpp \ src/odbc_test_suite.cpp \ src/types_test.cpp \ + src/authentication_test.cpp \ ../odbc/src/log.cpp \ ../odbc/src/cursor.cpp \ ../odbc/src/diagnostic/diagnostic_record.cpp \ diff --git a/modules/platforms/cpp/odbc-test/include/test_utils.h b/modules/platforms/cpp/odbc-test/include/test_utils.h index 786c2be873057..9faf89d9cb9f0 100644 --- a/modules/platforms/cpp/odbc-test/include/test_utils.h +++ b/modules/platforms/cpp/odbc-test/include/test_utils.h @@ -108,6 +108,11 @@ namespace ignite_test * @return New node. */ ignite::Ignite StartNode(const char* cfgFile, const char* name); + + /** + * Remove all the LFS artifacts. + */ + void ClearLfs(); } #endif // _IGNITE_ODBC_TEST_TEST_UTILS \ No newline at end of file diff --git a/modules/platforms/cpp/odbc-test/src/authentication_test.cpp b/modules/platforms/cpp/odbc-test/src/authentication_test.cpp index 0f11d44b89ae3..c746d5e92437a 100644 --- a/modules/platforms/cpp/odbc-test/src/authentication_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/authentication_test.cpp @@ -32,7 +32,7 @@ #include "test_utils.h" #include "odbc_test_suite.h" -#include "../../core-test/include/ignite/test_type.h" +#include "test_type.h" using namespace ignite; using namespace ignite::common; @@ -65,11 +65,11 @@ struct AuthenticationTestSuiteFixture : odbc::OdbcTestSuite AuthenticationTestSuiteFixture() : OdbcTestSuite() { + ClearLfs(); + grid = StartAdditionalNode("NodeMain"); grid.SetActive(true); - - grid.GetCache("cache").Clear(); } /** @@ -77,9 +77,7 @@ struct AuthenticationTestSuiteFixture : odbc::OdbcTestSuite */ virtual ~AuthenticationTestSuiteFixture() { - ExecQuery("DROP USER test"); - - grid.GetCache("cache").Clear(); + // No-op. } /** @@ -139,8 +137,6 @@ BOOST_AUTO_TEST_CASE(TestConnectionUserOperationsQuery) { Connect(MakeDefaultConnectionString()); - ExecQuery("DROP USER \"test\""); - SQLRETURN ret = ExecQuery("CREATE USER \"test\" WITH PASSWORD 'somePass'"); ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc); diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp index 7561e32919810..73727dd945fc8 100644 --- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp @@ -39,6 +39,7 @@ #include "ignite/impl/binary/binary_utils.h" #include "ignite/binary/binary_object.h" +#include "teamcity/teamcity_messages.h" #include "test_type.h" #include "complex_type.h" #include "test_utils.h" @@ -803,6 +804,9 @@ BOOST_AUTO_TEST_CASE(TestNullFields) BOOST_AUTO_TEST_CASE(TestDistributedJoins) { + if (JetBrains::underTeamcity()) + return; + // Starting additional node. Ignite node1 = StartAdditionalNode("Node1"); Ignite node2 = StartAdditionalNode("Node2"); diff --git a/modules/platforms/cpp/odbc-test/src/test_utils.cpp b/modules/platforms/cpp/odbc-test/src/test_utils.cpp index 6d9ed6b6484a8..9c1aca903f97d 100644 --- a/modules/platforms/cpp/odbc-test/src/test_utils.cpp +++ b/modules/platforms/cpp/odbc-test/src/test_utils.cpp @@ -121,4 +121,21 @@ namespace ignite_test return Ignition::Start(cfg, name); } + + std::string AppendPath(const std::string& base, const std::string& toAdd) + { + std::stringstream stream; + + stream << base << ignite::common::Fs << toAdd; + + return stream.str(); + } + + void ClearLfs() + { + std::string home = ignite::jni::ResolveIgniteHome(); + std::string workDir = AppendPath(home, "work"); + + ignite::common::DeletePath(workDir); + } } diff --git a/modules/platforms/cpp/odbc-test/src/utility_test.cpp b/modules/platforms/cpp/odbc-test/src/utility_test.cpp index a66860f7525b3..f2955b7f9a34f 100644 --- a/modules/platforms/cpp/odbc-test/src/utility_test.cpp +++ b/modules/platforms/cpp/odbc-test/src/utility_test.cpp @@ -24,6 +24,7 @@ #include #include +#include using namespace ignite::utility; @@ -34,7 +35,7 @@ BOOST_AUTO_TEST_CASE(TestUtilityRemoveSurroundingSpaces) std::string inStr(" \r \n \t some meaningfull data \n\n \t \r "); std::string expectedOutStr("some meaningfull data"); - std::string realOutStr(RemoveSurroundingSpaces(inStr.begin(), inStr.end())); + std::string realOutStr(ignite::common::StripSurroundingWhitespaces(inStr.begin(), inStr.end())); BOOST_REQUIRE(expectedOutStr == realOutStr); } diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/config/config_tools.h b/modules/platforms/cpp/odbc/include/ignite/odbc/config/config_tools.h index 573119145a318..9c376d8fe0cec 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/config/config_tools.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/config/config_tools.h @@ -20,7 +20,6 @@ #include #include -#include #include "ignite/odbc/end_point.h" diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h b/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h index d58479c5c7bd5..152da66357eac 100644 --- a/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h +++ b/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h @@ -23,18 +23,14 @@ #endif //min #include -#include #include -#include -#include #include #include #include "ignite/impl/binary/binary_reader_impl.h" #include "ignite/impl/binary/binary_writer_impl.h" -#include namespace ignite { @@ -43,67 +39,6 @@ namespace ignite /** Using common version of the util. */ using common::IntoLower; - /** - * Skip leading spaces. - * - * @param begin Iterator to the beginning of the character sequence. - * @param end Iterator to the end of the character sequence. - * @return Iterator to first non-blanc character. - */ - template - Iterator SkipLeadingSpaces(Iterator begin, Iterator end) - { - Iterator res = begin; - - while (isspace(*res) && res != end) - ++res; - - return res; - } - - /** - * Skip trailing spaces. - * - * @param begin Iterator to the beginning of the character sequence. - * @param end Iterator to the end of the character sequence. - * @return Iterator to last non-blanc character. - */ - template - Iterator SkipTrailingSpaces(Iterator begin, Iterator end) - { - Iterator res = end - 1; - - while (isspace(*res) && res != begin - 1) - --res; - - return res + 1; - } - - /** - * Remove leading and trailing spaces. - * - * @param begin Iterator to the beginning of the character sequence. - * @param end Iterator to the end of the character sequence. - * @return String without leading and trailing spaces. - */ - template - std::string RemoveSurroundingSpaces(Iterator begin, Iterator end) - { - std::string res; - - if (begin >= end) - return res; - - Iterator skipped_leading = SkipLeadingSpaces(begin, end); - Iterator skipped_trailing = SkipTrailingSpaces(skipped_leading, end); - - res.reserve(skipped_trailing - skipped_leading); - - std::copy(skipped_leading, skipped_trailing, std::back_inserter(res)); - - return res; - } - template T* GetPointerWithOffset(T* ptr, size_t offset) { diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj index 374695be2738a..352913efae505 100644 --- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj +++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj @@ -94,13 +94,12 @@ Disabled false $(OPENSSL_HOME_X86)\include;$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;$(ProjectDir)\..\..\src - _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;TARGET_MODULE_FULL_NAME="$(TargetFileName)";_DEBUG;ODBC_DEBUG;ODBC_LOG_PATH="D:\\odbc.log";%(PreprocessorDefinitions) + _CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;TARGET_MODULE_FULL_NAME="$(TargetFileName)";_DEBUG;%(PreprocessorDefinitions) true module.def Ws2_32.lib;Mswsock.lib;Advapi32.lib;Shlwapi.lib;%(AdditionalDependencies) - $(OPENSSL_HOME)\lib\VC @@ -117,7 +116,6 @@ true module.def Ws2_32.lib;Mswsock.lib;Advapi32.lib;Shlwapi.lib;%(AdditionalDependencies) - $(OPENSSL_HOME)\lib\VC @@ -138,7 +136,6 @@ true module.def Ws2_32.lib;Mswsock.lib;Advapi32.lib;Shlwapi.lib;%(AdditionalDependencies) - $(OPENSSL_HOME)\lib\VC @@ -157,7 +154,6 @@ true module.def Ws2_32.lib;Mswsock.lib;Advapi32.lib;Shlwapi.lib;%(AdditionalDependencies) - $(OPENSSL_HOME)\lib\VC diff --git a/modules/platforms/cpp/odbc/src/config/config_tools.cpp b/modules/platforms/cpp/odbc/src/config/config_tools.cpp index f0d956f5b9067..f53f49d77090b 100644 --- a/modules/platforms/cpp/odbc/src/config/config_tools.cpp +++ b/modules/platforms/cpp/odbc/src/config/config_tools.cpp @@ -17,10 +17,13 @@ #include #include +#include -#include "ignite/odbc/utility.h" -#include "ignite/odbc/config/config_tools.h" -#include "ignite/odbc/config/configuration.h" +#include + +#include +#include +#include namespace ignite { @@ -69,7 +72,7 @@ namespace ignite const char* addrBegin = parsedAddr.data() + addrBeginPos; const char* addrEnd = parsedAddr.data() + parsedAddr.size(); - std::string addr = utility::RemoveSurroundingSpaces(addrBegin, addrEnd); + std::string addr = common::StripSurroundingWhitespaces(addrBegin, addrEnd); if (!addr.empty()) { @@ -198,7 +201,7 @@ namespace ignite uint16_t ParsePort(const std::string& value, diagnostic::DiagnosticRecordStorage* diag) { - std::string port = utility::RemoveSurroundingSpaces(value.begin(), value.end()); + std::string port = common::StripSurroundingWhitespaces(value.begin(), value.end()); if (!common::AllOf(port.begin(), port.end(), &isdigit)) { 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 b32c2ec2c1fcc..d23c2cb6052e7 100644 --- a/modules/platforms/cpp/odbc/src/config/connection_string_parser.cpp +++ b/modules/platforms/cpp/odbc/src/config/connection_string_parser.cpp @@ -93,8 +93,8 @@ namespace ignite const char* value_begin = connect_str.data() + attr_eq_pos + 1; const char* value_end = connect_str.data() + connect_str.size(); - std::string key = utility::RemoveSurroundingSpaces(key_begin, key_end); - std::string value = utility::RemoveSurroundingSpaces(value_begin, value_end); + std::string key = common::StripSurroundingWhitespaces(key_begin, key_end); + std::string value = common::StripSurroundingWhitespaces(value_begin, value_end); if (value[0] == '{' && value[value.size() - 1] == '}') value = value.substr(1, value.size() - 2); diff --git a/modules/platforms/cpp/project/vs/ignite.sln b/modules/platforms/cpp/project/vs/ignite.sln index 01f12f959f210..cf2e48707839e 100644 --- a/modules/platforms/cpp/project/vs/ignite.sln +++ b/modules/platforms/cpp/project/vs/ignite.sln @@ -26,6 +26,10 @@ Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "common", "..\..\common\proj EndProject Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "jni", "..\..\jni\project\vs\jni.vcxproj", "{4F7E4917-4612-4B96-9838-025711ADE391}" EndProject +Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "thin-client", "..\..\thin-client\project\vs\thin-client.vcxproj", "{5C037386-B5F5-4A58-9EE2-3D3A508AA866}" +EndProject +Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "thin-client-test", "..\..\thin-client-test\project\vs\thin-client-test.vcxproj", "{5662F10A-9C40-45D6-AFF8-E93573FEAABA}" +EndProject Global GlobalSection(SolutionConfigurationPlatforms) = preSolution Debug|Win32 = Debug|Win32 @@ -98,8 +102,27 @@ Global {4F7E4917-4612-4B96-9838-025711ADE391}.Release|Win32.Build.0 = Release|Win32 {4F7E4917-4612-4B96-9838-025711ADE391}.Release|x64.ActiveCfg = Release|x64 {4F7E4917-4612-4B96-9838-025711ADE391}.Release|x64.Build.0 = Release|x64 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Debug|Win32.ActiveCfg = Debug|Win32 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Debug|Win32.Build.0 = Debug|Win32 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Debug|x64.ActiveCfg = Debug|x64 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Debug|x64.Build.0 = Debug|x64 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Release|Win32.ActiveCfg = Release|Win32 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Release|Win32.Build.0 = Release|Win32 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Release|x64.ActiveCfg = Release|x64 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Release|x64.Build.0 = Release|x64 + {5662F10A-9C40-45D6-AFF8-E93573FEAABA}.Debug|Win32.ActiveCfg = Debug|Win32 + {5662F10A-9C40-45D6-AFF8-E93573FEAABA}.Debug|Win32.Build.0 = Debug|Win32 + {5662F10A-9C40-45D6-AFF8-E93573FEAABA}.Debug|x64.ActiveCfg = Debug|x64 + {5662F10A-9C40-45D6-AFF8-E93573FEAABA}.Debug|x64.Build.0 = Debug|x64 + {5662F10A-9C40-45D6-AFF8-E93573FEAABA}.Release|Win32.ActiveCfg = Release|Win32 + {5662F10A-9C40-45D6-AFF8-E93573FEAABA}.Release|Win32.Build.0 = Release|Win32 + {5662F10A-9C40-45D6-AFF8-E93573FEAABA}.Release|x64.ActiveCfg = Release|x64 + {5662F10A-9C40-45D6-AFF8-E93573FEAABA}.Release|x64.Build.0 = Release|x64 EndGlobalSection GlobalSection(SolutionProperties) = preSolution HideSolutionNode = FALSE EndGlobalSection + GlobalSection(ExtensibilityGlobals) = postSolution + SolutionGuid = {3761BFB8-4345-4AC4-BE04-5ACB1DCA842C} + EndGlobalSection EndGlobal diff --git a/modules/platforms/cpp/project/vs/ignite.slnrel b/modules/platforms/cpp/project/vs/ignite.slnrel index 6a37bf417cf72..187ff6ae93eca 100644 --- a/modules/platforms/cpp/project/vs/ignite.slnrel +++ b/modules/platforms/cpp/project/vs/ignite.slnrel @@ -19,6 +19,8 @@ Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "common", "..\..\common\proj EndProject Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "jni", "..\..\jni\project\vs\jni.vcxproj", "{4F7E4917-4612-4B96-9838-025711ADE391}" EndProject +Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "thin-client", "..\..\thin-client\project\vs\thin-client.vcxproj", "{5C037386-B5F5-4A58-9EE2-3D3A508AA866}" +EndProject Global GlobalSection(SolutionConfigurationPlatforms) = preSolution Release|x64 = Release|x64 @@ -68,8 +70,19 @@ Global {4F7E4917-4612-4B96-9838-025711ADE391}.Release|Win32.Build.0 = Release|Win32 {4F7E4917-4612-4B96-9838-025711ADE391}.Release|x64.ActiveCfg = Release|x64 {4F7E4917-4612-4B96-9838-025711ADE391}.Release|x64.Build.0 = Release|x64 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Debug|Win32.ActiveCfg = Debug|Win32 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Debug|Win32.Build.0 = Debug|Win32 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Debug|x64.ActiveCfg = Debug|x64 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Debug|x64.Build.0 = Debug|x64 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Release|Win32.ActiveCfg = Release|Win32 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Release|Win32.Build.0 = Release|Win32 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Release|x64.ActiveCfg = Release|x64 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Release|x64.Build.0 = Release|x64 EndGlobalSection GlobalSection(SolutionProperties) = preSolution HideSolutionNode = FALSE EndGlobalSection + GlobalSection(ExtensibilityGlobals) = postSolution + SolutionGuid = {3761BFB8-4345-4AC4-BE04-5ACB1DCA842C} + EndGlobalSection EndGlobal diff --git a/modules/platforms/cpp/project/vs/ignite_x86.slnrel b/modules/platforms/cpp/project/vs/ignite_x86.slnrel index abd0ad1654146..44308aa07c1e6 100644 --- a/modules/platforms/cpp/project/vs/ignite_x86.slnrel +++ b/modules/platforms/cpp/project/vs/ignite_x86.slnrel @@ -19,6 +19,8 @@ Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "common", "..\..\common\proj EndProject Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "jni", "..\..\jni\project\vs\jni.vcxproj", "{4F7E4917-4612-4B96-9838-025711ADE391}" EndProject +Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "thin-client", "..\..\thin-client\project\vs\thin-client.vcxproj", "{5C037386-B5F5-4A58-9EE2-3D3A508AA866}" +EndProject Global GlobalSection(SolutionConfigurationPlatforms) = preSolution Release|Win32 = Release|Win32 @@ -68,8 +70,19 @@ Global {4F7E4917-4612-4B96-9838-025711ADE391}.Release|Win32.Build.0 = Release|Win32 {4F7E4917-4612-4B96-9838-025711ADE391}.Release|x64.ActiveCfg = Release|x64 {4F7E4917-4612-4B96-9838-025711ADE391}.Release|x64.Build.0 = Release|x64 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Debug|Win32.ActiveCfg = Debug|Win32 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Debug|Win32.Build.0 = Debug|Win32 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Debug|x64.ActiveCfg = Debug|x64 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Debug|x64.Build.0 = Debug|x64 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Release|Win32.ActiveCfg = Release|Win32 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Release|Win32.Build.0 = Release|Win32 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Release|x64.ActiveCfg = Release|x64 + {5C037386-B5F5-4A58-9EE2-3D3A508AA866}.Release|x64.Build.0 = Release|x64 EndGlobalSection GlobalSection(SolutionProperties) = preSolution HideSolutionNode = FALSE EndGlobalSection + GlobalSection(ExtensibilityGlobals) = postSolution + SolutionGuid = {3761BFB8-4345-4AC4-BE04-5ACB1DCA842C} + EndGlobalSection EndGlobal diff --git a/modules/platforms/cpp/thin-client-test/Makefile.am b/modules/platforms/cpp/thin-client-test/Makefile.am new file mode 100644 index 0000000000000..e4ab9a4ab2407 --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/Makefile.am @@ -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. +## + +ACLOCAL_AMFLAGS =-I m4 + +noinst_PROGRAMS = ignite-thin-client-tests + +SUBDIRS = \ + include + +AM_CPPFLAGS = \ + -I$(srcdir)/include \ + -I@top_srcdir@/core/include \ + -I@top_srcdir@/core/os/linux/include \ + -I@top_srcdir@/common/include \ + -I@top_srcdir@/common/os/linux/include \ + -I@top_srcdir@/binary/include \ + -I@top_srcdir@/jni/include \ + -I@top_srcdir@/jni/os/linux/include \ + -I@top_srcdir@/thin-client/include \ + -I@top_srcdir@/thin-client/os/linux/include \ + -I$(JAVA_HOME)/include \ + -I$(JAVA_HOME)/include/linux \ + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS + +AM_CXXFLAGS = \ + -Wall \ + -std=c++03 + +ignite_thin_client_tests_LDADD = \ + @top_srcdir@/core/libignite.la \ + @top_srcdir@/thin-client/libignite-thin-client.la \ + -lpthread \ + -lboost_thread \ + -lboost_system \ + -lboost_chrono + +ignite_thin_client_tests_LDFLAGS = \ + -static-libtool-libs \ + -rdynamic + +ignite_thin_client_tests_SOURCES = \ + src/cache_client_test.cpp \ + src/teamcity/teamcity_boost.cpp \ + src/teamcity/teamcity_messages.cpp \ + src/test_utils.cpp \ + src/ignite_client_test.cpp \ + src/auth_test.cpp \ + src/ssl_test.cpp + +run-check: check + ./ignite-thin-client-tests -p + +clean-local: clean-check + $(RM) *.gcno *.gcda + +clean-check: + $(RM) $(ignite_thin_client_tests_OBJECTS) diff --git a/modules/platforms/cpp/thin-client-test/config/auth-32.xml b/modules/platforms/cpp/thin-client-test/config/auth-32.xml new file mode 100644 index 0000000000000..c4ba75cde77bc --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/config/auth-32.xml @@ -0,0 +1,48 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/modules/platforms/cpp/thin-client-test/config/auth-default.xml b/modules/platforms/cpp/thin-client-test/config/auth-default.xml new file mode 100644 index 0000000000000..3d197e7ff88ea --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/config/auth-default.xml @@ -0,0 +1,77 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 127.0.0.1:47500 + + + + + + + + + diff --git a/modules/platforms/cpp/thin-client-test/config/auth.xml b/modules/platforms/cpp/thin-client-test/config/auth.xml new file mode 100644 index 0000000000000..f5af82902dfce --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/config/auth.xml @@ -0,0 +1,31 @@ + + + + + + + + + + + diff --git a/modules/platforms/cpp/thin-client-test/config/cache-32.xml b/modules/platforms/cpp/thin-client-test/config/cache-32.xml new file mode 100644 index 0000000000000..ecccd0da3ebd6 --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/config/cache-32.xml @@ -0,0 +1,52 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/modules/platforms/cpp/thin-client-test/config/cache-default.xml b/modules/platforms/cpp/thin-client-test/config/cache-default.xml new file mode 100644 index 0000000000000..8ed4437bfcfcf --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/config/cache-default.xml @@ -0,0 +1,141 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 127.0.0.1:47500..47503 + + + + + + + + + + + + + + + + + + + + + + diff --git a/modules/platforms/cpp/thin-client-test/config/cache.xml b/modules/platforms/cpp/thin-client-test/config/cache.xml new file mode 100644 index 0000000000000..54df16ea6cc61 --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/config/cache.xml @@ -0,0 +1,35 @@ + + + + + + + + + + + diff --git a/modules/platforms/cpp/thin-client-test/config/ssl-32.xml b/modules/platforms/cpp/thin-client-test/config/ssl-32.xml new file mode 100644 index 0000000000000..0750580e4b364 --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/config/ssl-32.xml @@ -0,0 +1,48 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/modules/platforms/cpp/thin-client-test/config/ssl-default.xml b/modules/platforms/cpp/thin-client-test/config/ssl-default.xml new file mode 100644 index 0000000000000..bbd0dd8628026 --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/config/ssl-default.xml @@ -0,0 +1,84 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 127.0.0.1:47500 + + + + + + + + + diff --git a/modules/platforms/cpp/thin-client-test/config/ssl.xml b/modules/platforms/cpp/thin-client-test/config/ssl.xml new file mode 100644 index 0000000000000..2b560644aea69 --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/config/ssl.xml @@ -0,0 +1,31 @@ + + + + + + + + + + + diff --git a/modules/platforms/cpp/thin-client-test/config/ssl/ca.pem b/modules/platforms/cpp/thin-client-test/config/ssl/ca.pem new file mode 100644 index 0000000000000..10bf7af8488aa --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/config/ssl/ca.pem @@ -0,0 +1,24 @@ +-----BEGIN TRUSTED CERTIFICATE----- +MIID8DCCAtgCCQCo436SJYMUcjANBgkqhkiG9w0BAQsFADCBuTELMAkGA1UEBhMC +UlUxGTAXBgNVBAgMEFNhaW50LVBldGVyc2J1cmcxGTAXBgNVBAcMEFNhaW50LVBl +dGVyc2J1cmcxIzAhBgNVBAoMGkFwYWNoZSBTcGZ0d2FyZSBGb3VuZGF0aW9uMRYw +FAYDVQQLDA1BcGFjaGUgSWduaXRlMRQwEgYDVQQDDAtJZ29yIFNhcGVnbzEhMB8G +CSqGSIb3DQEJARYSaXNhcGVnb0BhcGFjaGUub3JnMB4XDTE3MTEyODE3MzExNloX +DTI3MTEyNjE3MzExNlowgbkxCzAJBgNVBAYTAlJVMRkwFwYDVQQIDBBTYWludC1Q +ZXRlcnNidXJnMRkwFwYDVQQHDBBTYWludC1QZXRlcnNidXJnMSMwIQYDVQQKDBpB +cGFjaGUgU3BmdHdhcmUgRm91bmRhdGlvbjEWMBQGA1UECwwNQXBhY2hlIElnbml0 +ZTEUMBIGA1UEAwwLSWdvciBTYXBlZ28xITAfBgkqhkiG9w0BCQEWEmlzYXBlZ29A +YXBhY2hlLm9yZzCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEBALMcDfYN +Ixc0o4bLX/3T5MQE4pk+Bv9Dfr7vBYNPJKSr/GKQJN+5QA/tr1uxTsMSBoE19y5e +N1vXFtlWMJ2um3ojNbGeqSGuzuDKk0htbgmisyctvEFTqtiYI7D+f7dalEM2KnLj +f0jIV6NJVilkFKmgsfuZpbZFRkqJDEx74ZqNAYQQ0qJ+zGv7diEak8FwWa4n6xe7 +VHt7VZIbKIkMgTljJLULbExxCRvTHpSeXPP5IMr5x1RGuSavCu4GDl+HmrXac7ot +L7sqIFHL9JGXTWO16accOQnQIdLQmhj6qh2Em8z41udabzUyIQmOSP6mmwnJEIdz +jNyWLA8XtYZOgcsCAwEAATANBgkqhkiG9w0BAQsFAAOCAQEAT6jBgiQjCdQ02cxe +H9YDFw5+cfb6YQJcjJW94BspySftUGfJ6GQm/Ybcc/ZqW7lhEILKfifdpHPfWby/ +sqhwA0nsLt5hNCjMsDcBq5onggy9ymZYak0VVWT/XkHiK27sQLK6BXo6wqRDwT4h +F9CgUWNcLGDVAaVpjLaR8itZZgx98q9MAZ680oERUZWTsj17oO0RK/x9TBWUm7OR +f9g9VNRz3mwT5dTtuqrSq4NlF1nVD7BN3lRddZUW4KnU6et2mXtIKMnUT/+XyIW3 +g+f6LuhR+Q6yhmVdfBVa8opJQIR5trb0eKnQf4pEQcvT/EX+vWkThNJkdZ4qm0mY +eIl/jQ== +-----END TRUSTED CERTIFICATE----- diff --git a/modules/platforms/cpp/thin-client-test/config/ssl/client_full.pem b/modules/platforms/cpp/thin-client-test/config/ssl/client_full.pem new file mode 100644 index 0000000000000..9a54f43286474 --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/config/ssl/client_full.pem @@ -0,0 +1,84 @@ +Bag Attributes + friendlyName: client + localKeyID: 54 69 6D 65 20 31 35 31 31 39 36 35 31 38 31 32 33 37 +Key Attributes: +-----BEGIN RSA PRIVATE KEY----- +MIIEowIBAAKCAQEAmm+QSwc1yvJWjTG8L4rAiHOg38rM1AtOFInzYizT/B6wzsmx +3Ql4fb+aHm3dPt5kjhQLZToq1QCUsS9B/47ZFd0waDHGgLiSNKn1WdX7Q+4GgplY +wznc5j+GM65F2aI3tRnn2Jorfss9OElzU5vLh0aD3e5cEpgYSAGEFl1O9t8b6dHq +YMERtdTCNQVeBUQPiaNNTwOXixtdsPvGKNgMZtVpvV+AMlgVmV+VTYUqQhlX+ro4 +9E4n00+iDIk/REwAiQi/Kb08kovLY9sF2pqC24zU5swyIopEijoa6Z2WYekq1fyL +P9NknOiMOmWaKqu5ThVbP6c2xGeCdAGbNi+74wIDAQABAoIBAH2oALPLbg1vGNVR +flkAgJ+F1YPBst9lQ2aayBk9eE5PenUGz12V8x/94hyYL8iTkyW3UX5P0cH18wYn ++X9Cb4fGrThaJ7VceDm5gBRUc1lWNp3Uv9A5KG9/iGZMijEGOGV2appm9rT4ERvX +R6rjvLqYuXq+EtReRINyGFNKIJHkuwM7Ycy9Rk7O6SrcjjUEj1Z5gEUP0B7Z0jVI +bsOGqghRflRVqnFT61+koFWBn8taemWmXNa835beN2lRIWezNqfd+9kL72UUt6sl +VmkqBydDQ+IXCOgLZDYUu3N0bfrKuYmilsRg9XuKTUv+jvwtfGUWTO1+6eSrFD+B +3j0YeKECgYEA0XrsUOGVl2O4JWBOa36TRHMz0tRBrZUJMK7apEEiGx7jc7q22PI+ +RyPrf9TR7JOLN1AdXoHrJsKCadnTr6g/CAOZSV1V+lUeQkcDmG4OkY814fESxlBA +jHg7FnJdjzs6VfA1Cv0eT2KxU0ld461AN77WBgG21E2Gw3AGSddbflcCgYEAvLtV +fhy3+WOUutFsLv1uTz/y+Emih6pmgwsuRG7WgQD9s2KZS+ejh5TQAq4sD+9MSkKk +XnvdbwGQkHppdtVP5wMeSfY6tG/9YmY7X5rTG9PTsrzAzNJwrnE9hFv7Xnj3SMnC +0D9eBnA8XWq49vsair7gxBWwhGYR/5WDicGn31UCgYEAgJb6lbPEhkgadCcoPGhY +sn4dY+AT6SBBofbUFFw/OXK+oP4O3CHoBPeRysizjIwNu4icXrVVyldAgwXKIlwf +RSYgZJ689oxlpL2/AUUeAFBu/SFlKwN8fD4Z12+g6xmqd7KafQJgJs8olz3EOszR +TOcCrnFC8BQfE72ivpVKfsUCgYBBZw3maR9cHiUxZFeAJTTv47JYVnQrXWKu8NX0 +wxA0nlpMqtOC2V/zIR/Afgko8H1YkqWRVI1U5Y7qcnFPy/YnkQ4TBXAsjU6apeDX +bbjQwORw4TZTBG4mFXoC/zkp8POI1tpa/kqgIo7YtJ4qgLQe9qy/ZuDH0PNVxcNS +nriFDQKBgF70vrmuYEobLpBD/whcOqMDjn3Zj+xx3TSRmjmjc4w5eRR4+9iquKs5 +lhY1NsAcq3SYA8P5rNDV39FpiPErFqDTjsf0AaOXbpaqo2jOANmvyT15CKW8325R +yEcxaA8sSbBaz8QRkydzpoKT+4WFJnYzCZtG+kUU+Pgu6uahAs4N +-----END RSA PRIVATE KEY----- +Bag Attributes + friendlyName: client + localKeyID: 54 69 6D 65 20 31 35 31 31 39 36 35 31 38 31 32 33 37 +subject=/CN=ODBC +issuer=/C=RU/ST=Saint-Petersburg/L=Saint-Petersburg/O=Apache Spftware Foundation/OU=Apache Ignite/CN=Igor Sapego/emailAddress=isapego@apache.org +-----BEGIN CERTIFICATE----- +MIIDPTCCAiUCAQIwDQYJKoZIhvcNAQEFBQAwgbkxCzAJBgNVBAYTAlJVMRkwFwYD +VQQIDBBTYWludC1QZXRlcnNidXJnMRkwFwYDVQQHDBBTYWludC1QZXRlcnNidXJn +MSMwIQYDVQQKDBpBcGFjaGUgU3BmdHdhcmUgRm91bmRhdGlvbjEWMBQGA1UECwwN +QXBhY2hlIElnbml0ZTEUMBIGA1UEAwwLSWdvciBTYXBlZ28xITAfBgkqhkiG9w0B +CQEWEmlzYXBlZ29AYXBhY2hlLm9yZzAeFw0xNzExMjkxNDAwNDhaFw0yNzExMjcx +NDAwNDhaMA8xDTALBgNVBAMTBE9EQkMwggEiMA0GCSqGSIb3DQEBAQUAA4IBDwAw +ggEKAoIBAQCab5BLBzXK8laNMbwvisCIc6DfyszUC04UifNiLNP8HrDOybHdCXh9 +v5oebd0+3mSOFAtlOirVAJSxL0H/jtkV3TBoMcaAuJI0qfVZ1ftD7gaCmVjDOdzm +P4YzrkXZoje1GefYmit+yz04SXNTm8uHRoPd7lwSmBhIAYQWXU723xvp0epgwRG1 +1MI1BV4FRA+Jo01PA5eLG12w+8Yo2Axm1Wm9X4AyWBWZX5VNhSpCGVf6ujj0TifT +T6IMiT9ETACJCL8pvTySi8tj2wXamoLbjNTmzDIiikSKOhrpnZZh6SrV/Is/02Sc +6Iw6ZZoqq7lOFVs/pzbEZ4J0AZs2L7vjAgMBAAEwDQYJKoZIhvcNAQEFBQADggEB +AH8heMPYkC/abqg9xtC5WFgYxnXWA6jwjKVuCSnrEhZGQem9HFvFXmZckJr25RXc +9i8WIehN5oeJgmPioK6j8ylwCJxo9lRii4NFBVpS6IjmqMdYHa+4K1R9Y7XdDEBL +nVzlL8Hjlv4ESjg+1LW4nSt8f4oBdAvfds4uNTtAwABqmfqXWaOKYdh9OJs84QJa +pA4PQvxqUz6c4tyPi/VucVAV8uKxR7P+uoy6qKKZK6NkY3RuXwHXxJ5lDr6ET1mb +St0BbAt4KSktPonCu0qoHWqcynw79KcoPhmQWpzbijVs+y/SlHmZje+H4R7j7xt7 +aw7dWxUa096x4vsi1WvlMOY= +-----END CERTIFICATE----- +Bag Attributes + friendlyName: 1.2.840.113549.1.9.1=#16126973617065676f406170616368652e6f7267,CN=Igor Sapego,OU=Apache Ignite,O=Apache Spftware Foundation,L=Saint-Petersburg,ST=Saint-Petersburg,C=RU +subject=/C=RU/ST=Saint-Petersburg/L=Saint-Petersburg/O=Apache Spftware Foundation/OU=Apache Ignite/CN=Igor Sapego/emailAddress=isapego@apache.org +issuer=/C=RU/ST=Saint-Petersburg/L=Saint-Petersburg/O=Apache Spftware Foundation/OU=Apache Ignite/CN=Igor Sapego/emailAddress=isapego@apache.org +-----BEGIN CERTIFICATE----- +MIID8DCCAtgCCQCo436SJYMUcjANBgkqhkiG9w0BAQsFADCBuTELMAkGA1UEBhMC +UlUxGTAXBgNVBAgMEFNhaW50LVBldGVyc2J1cmcxGTAXBgNVBAcMEFNhaW50LVBl +dGVyc2J1cmcxIzAhBgNVBAoMGkFwYWNoZSBTcGZ0d2FyZSBGb3VuZGF0aW9uMRYw +FAYDVQQLDA1BcGFjaGUgSWduaXRlMRQwEgYDVQQDDAtJZ29yIFNhcGVnbzEhMB8G +CSqGSIb3DQEJARYSaXNhcGVnb0BhcGFjaGUub3JnMB4XDTE3MTEyODE3MzExNloX +DTI3MTEyNjE3MzExNlowgbkxCzAJBgNVBAYTAlJVMRkwFwYDVQQIDBBTYWludC1Q +ZXRlcnNidXJnMRkwFwYDVQQHDBBTYWludC1QZXRlcnNidXJnMSMwIQYDVQQKDBpB +cGFjaGUgU3BmdHdhcmUgRm91bmRhdGlvbjEWMBQGA1UECwwNQXBhY2hlIElnbml0 +ZTEUMBIGA1UEAwwLSWdvciBTYXBlZ28xITAfBgkqhkiG9w0BCQEWEmlzYXBlZ29A +YXBhY2hlLm9yZzCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEBALMcDfYN +Ixc0o4bLX/3T5MQE4pk+Bv9Dfr7vBYNPJKSr/GKQJN+5QA/tr1uxTsMSBoE19y5e +N1vXFtlWMJ2um3ojNbGeqSGuzuDKk0htbgmisyctvEFTqtiYI7D+f7dalEM2KnLj +f0jIV6NJVilkFKmgsfuZpbZFRkqJDEx74ZqNAYQQ0qJ+zGv7diEak8FwWa4n6xe7 +VHt7VZIbKIkMgTljJLULbExxCRvTHpSeXPP5IMr5x1RGuSavCu4GDl+HmrXac7ot +L7sqIFHL9JGXTWO16accOQnQIdLQmhj6qh2Em8z41udabzUyIQmOSP6mmwnJEIdz +jNyWLA8XtYZOgcsCAwEAATANBgkqhkiG9w0BAQsFAAOCAQEAT6jBgiQjCdQ02cxe +H9YDFw5+cfb6YQJcjJW94BspySftUGfJ6GQm/Ybcc/ZqW7lhEILKfifdpHPfWby/ +sqhwA0nsLt5hNCjMsDcBq5onggy9ymZYak0VVWT/XkHiK27sQLK6BXo6wqRDwT4h +F9CgUWNcLGDVAaVpjLaR8itZZgx98q9MAZ680oERUZWTsj17oO0RK/x9TBWUm7OR +f9g9VNRz3mwT5dTtuqrSq4NlF1nVD7BN3lRddZUW4KnU6et2mXtIKMnUT/+XyIW3 +g+f6LuhR+Q6yhmVdfBVa8opJQIR5trb0eKnQf4pEQcvT/EX+vWkThNJkdZ4qm0mY +eIl/jQ== +-----END CERTIFICATE----- diff --git a/modules/platforms/cpp/thin-client-test/config/ssl/client_unknown.pem b/modules/platforms/cpp/thin-client-test/config/ssl/client_unknown.pem new file mode 100644 index 0000000000000..b9a8e43828ed4 --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/config/ssl/client_unknown.pem @@ -0,0 +1,50 @@ +-----BEGIN PRIVATE KEY----- +MIIEvwIBADANBgkqhkiG9w0BAQEFAASCBKkwggSlAgEAAoIBAQC0HaDP3KkkgcT7 +3FWWxXEgiT5vAsW+ZByiWg5Yh5XtU7rAjiXT9VmOm1h/5GNdngF7ofhT+qal1k8W +yiLsbqWL6qYXkc0e8aKw9ciBaWDWDkgXNnV03butu70hNvm/fzYp/6/WQ5SXUJdm +mJZyeVjgU4NXnOe0lv2PlzUr5q1yjJCSnTbsHBvs0qfoy7DYqPQFbrwiuIOeYtVg +mjjL/y6SUCY/uNY8McLG0YFubNsnOCEIyZOgAM1RdmARt+iYZuLVZkicgkaxYtLB +pbUB8K2Y/A+0WjL7JceZVWbUaQKkDncegccMmHzEOBsJyLRWu+/5ztan7xh+jIGd +vtlgoz8/AgMBAAECggEBAKx4TnucHW7ywFA3/NEBwUDZiqo8KmQUcwLXXzNQqbT2 +3zXxJ7HeF2KhyE22wZtsaJkdULABrL/SBiuZZXWmwejHyUF6gMMnuJTg1Z1t7acr +R8bKhBBjHorO4dzDSsUfKBtxVcvDLK70IfXBGRcGsZ8k+PjVK15W0vhDX0vpeJ7c +9E6FS+Kfz31d8/xLETF3QdevfPTTMliakL0Tr37NKefdGXU8O07tvL/Chq40VWzl +KqdU0ti+mI18fNsTgpe16JLvEpQBQ1ZRm/ip5ZJPkPloZn7sP7KIpmZOuMplbYlJ +ZyINe+OKOLB8/65XK8eqwgEyQQi9Iq7/2bWgMbZKP8ECgYEA54ysuNUEVtwhmcXK +CGiTfuCqguvKmaFraenoCxvf+aY3TU7dguA8soI+yQraqt9nW1fqrMEOnzr/mEFI +3NE81k3uWLPdcbk/sPKKI2iQqkmlCR7XQYch0Kq2KZ3A4kXvT0M6njBb/8UFtSai +dWiit9/9c40PGUA28+dYCSnnn2ECgYEAxyKT/+hrCHauj7Enz2jqIBsIJGRq6bc0 +rZZf4yoQO4wJTs7zTUa3REYHQOGYM/zb6BkmO5SaHiUxwrAvWi+3UvSwzSC7SZEw +2Ym8FRFZLcjG2n9CeHK4jcp8kXnoeoKnD9K9ztQszmjl+YJ5LLAXEQuXlwG0ZSld +Nz2OHuHYgp8CgYEAnrJequtsZu/Go7LYzjyGYp4unz5mBDeK+/NRk1OOhXczTevw +i8OONjZoO7gRuK0XFJbEA5TppZ51EXAlNRq/HI4a/naAHnJ9wZPps2z2QY/p6b9W +ftXUCkUGRq2VRj/yTZvhMAwq8biiOc7554hIhceh2TFjmMXXJAF56AXtduECgYA6 +Y1kh3/QCN0pXt4oEtq5vW323rfnrLVy0yTK0WNEJpLXV9bhBIEr8KcxWqc8G701L +cJRVQ+0siZtdimtX9kW6fSSaAM7pczjvkAMl6kBLvGMSDd5fksOqT4TW4vfXdBen +/BLLYksA+qDz193PzJH23X9FN6q9PrOmTtYtF+tz4QKBgQDeUhgpylS0sjTzS36b +8V86DJ4mj22B7H6jp826Y696V79a2/jSacs1KKExOYXeJBUdhDoutVS0PiT10br9 +lSIKiExbrddmBUd5OEtZKz1bXqRoMZlAvjuOhqadU0DciSQjiZ8CSmSzCX31++7G +velIMG6xcMV3LKtY4dr7B6KuiA== +-----END PRIVATE KEY----- +-----BEGIN CERTIFICATE----- +MIIDoTCCAomgAwIBAgIJAJy64S/DycmkMA0GCSqGSIb3DQEBCwUAMGcxCzAJBgNV +BAYTAlJVMRkwFwYDVQQIDBBTYWludCBQZXRlcnNidXJnMRkwFwYDVQQHDBBTYWlu +dCBQZXRlcnNidXJnMRAwDgYDVQQKDAdVbmtub3duMRAwDgYDVQQDDAdVbmtub3du +MB4XDTE4MDEyNDEwMTUwNFoXDTE5MDEyNDEwMTUwNFowZzELMAkGA1UEBhMCUlUx +GTAXBgNVBAgMEFNhaW50IFBldGVyc2J1cmcxGTAXBgNVBAcMEFNhaW50IFBldGVy +c2J1cmcxEDAOBgNVBAoMB1Vua25vd24xEDAOBgNVBAMMB1Vua25vd24wggEiMA0G +CSqGSIb3DQEBAQUAA4IBDwAwggEKAoIBAQC0HaDP3KkkgcT73FWWxXEgiT5vAsW+ +ZByiWg5Yh5XtU7rAjiXT9VmOm1h/5GNdngF7ofhT+qal1k8WyiLsbqWL6qYXkc0e +8aKw9ciBaWDWDkgXNnV03butu70hNvm/fzYp/6/WQ5SXUJdmmJZyeVjgU4NXnOe0 +lv2PlzUr5q1yjJCSnTbsHBvs0qfoy7DYqPQFbrwiuIOeYtVgmjjL/y6SUCY/uNY8 +McLG0YFubNsnOCEIyZOgAM1RdmARt+iYZuLVZkicgkaxYtLBpbUB8K2Y/A+0WjL7 +JceZVWbUaQKkDncegccMmHzEOBsJyLRWu+/5ztan7xh+jIGdvtlgoz8/AgMBAAGj +UDBOMB0GA1UdDgQWBBSmehz8ReTNQ6IQmMBXhns97I3U6jAfBgNVHSMEGDAWgBSm +ehz8ReTNQ6IQmMBXhns97I3U6jAMBgNVHRMEBTADAQH/MA0GCSqGSIb3DQEBCwUA +A4IBAQAjtzZtXSqIrWJfCJCp0Kal/swR9EDIhvdZNpH3VJMUKENRPKZF2AdlGLwx +sSAL6oUkCjew3NC4UM7CH1LZR+RZOQXhLxh+xmlIUDJukjftPcEAXAymCJyEShjD +qr/YH3SrrVfzf/ViZOa3YrlWXuP9WqEYYAMJcNmgl2A1vhpIwrF0+GLrvnUV2E6a +qHqBmA8GforiiOye+TQKNuFCgd+m8nr3AUCBaY2aVgFBg5Z2uumMzx5qmejQIdDr +tiKsr7+QiaY97lXIThMCDxPt63Jk2uO9yUJZSzDJlY2JHuRnMmyO+TnVrUAyCr60 +dBkPpbvH/kMPPFkOhDk6e40YvqWY +-----END CERTIFICATE----- diff --git a/modules/platforms/cpp/thin-client-test/config/ssl/server.jks b/modules/platforms/cpp/thin-client-test/config/ssl/server.jks new file mode 100644 index 0000000000000000000000000000000000000000..2632662c175974e8c31fdaf377fd4b9393854e83 GIT binary patch literal 4256 zcmeH~cU03^7RU2TNF*3~wbF|~LINZd1wjOXrARL_KxmPcgoKWO1jItI&_NI>Dlj0$ z&=CTPG#vzW03m>=bQqA1z@j_jnLWE_|KHvDYU~?}n0-6>R;01Dv`(}90u5=V=ETg8Sr ztWCzdUUV0+Aba=+;VJGSdZYlN8{XfOM8pW7`MxWN5#ir)8d?ee4)amjRzKmT}L^pT=-RB&>EdxmK7?Ft;i`7EqgWAm89=W zgy&UBBOA3XiU-og%XcGd?2~nHvXqHP{SKRa1M3rRd_}nx+Zlz`x_XA!5k_I-nF&A) zS6^O4kN0+<=;0&=*{)1_RRI|g7jFEsBSO>00DoF(0(%SM+g9!K4Q=$#Nlt02YVca-$4X9 zBK+32@L-4dl%}yGCpx9)&l5VQ+@yA5hpB5`7wYg_w5|y0kpk+dU1M|QBQnHb4#mVP z%k-410415yG(=OEhpm?}zm?mbqxJ_m;+$4xJv3CkwLpiVDJs~TYv$@8@B9`h^o*~* zy)0+vfe2rIYy{*q_C;}0-uO!-C z(C#f~xiQo7r+^fJv7$HAvw<05`ZAqwO!v|{uGBa1cjh! zEPH4U7@HgnU;$*o+`xWHvCvp%d&XPLqU4Zp8diS-+3Mr+FnSZyIIAlhnWQnhn(2HR z!?YFB>iuPoE`2HQ-TUs!;#Z?h#!EG@O2gMAI&GWpzWHTmqy*C>!kH$?xdN)Ir+)GE zon-P7Q&tydAGF_&==F$O2lwl{5O*`~ND9r`d*1VeafW)DYA(A5l`+ak#);sm8;ZOe z8rN~V%=k1@1Oz*BSDGiW{KAB`^a$2BQ8?X1)j<-bx)>w;G>!O7hTPg%&+o_Ke@RBO zY6S>yG)_Sr_BZq7rgwOK;vQmr9u@QY{1y^U0Z zU1$9ZrAS0nT>tXzwu(i1d!w=I@dX>V`Cf{*V;t&3?44_-9)oVTyxXZoA}2S{t7oKx z&vDz-1sPAt9Wr_?Ln!WIyn3>-1mNVGJCYB)=SO~iaEfv-;pPh@uJcu~r`AdF$U{wW zVUo>@D{V`jzAw|0_$|EK+cN3-wgu@rb@l2y;eD0~5nA09_VIZwo3yTrXK@Yj(h9ho z;&@F+Q+vi9{nL!#Gqrl&_YGjXc|Avm%i73(^$kn>>k6FQ2jTU1r9Z#a?x5kwHoD`6&A(Yoa&8J+COZ-yJbW@Xp?+GHQKWQ@c-6ARh35!E-F^Plo-*{; z;ktLuwy3e_b7|?w32%+pCc}y`O81nQINefbuf>&*Pfs&H2SN@ZLZyO83WeUjmah(% zWs7=tM#g64juMp-N{-h}&-&nqIIUiIks?4qg$# zNmft4n0jhEE(-5V91NWe2)c{t$}0Wjqd{gGD$4nc-3@3M=(iz&rgqYEJg|;yq$%hb z?%L3@Rr{7|xWMqn-fqjPRd6vl-UjyN7~4XGa)sUW;ToP%w+|tzoHJGP2_d1*FG4aD z9{Yr}``xOl?GQm7@#-CKbxDUW^wxOnqJzzws=j2ivi3R6ByOM6jDB>8MZP%`|3pl? z{g)bH*Oe9gTKtyo=Af&gsk@adf>_#3q9zZ&+Qoi)%5EI%lyZ3rj|k_@HHCBt5#uQX z*6Ujyj}`*I)-tHdd-t=NJ8DNZT?xsx2gveeIQ@}ck;aY4xW6IGnq$_#W;oJ_O6S!i zUvL?{6)!HE)~VjoSfc$j7?j(&(jfNp&q_8F?9g*?l^a2Lu1~kwE`P zs7MW?gjQ0*{y{}5ze}{h58&ecGjQ#b#eXJ_rod7u4%R$aN9`F`N6*You^qh7%-LSE z3q^5vco&_92hW);o6C5dkn^#2mb!Ahj=?f1r<~{E{dR)8mQ*WjYCki>gK|tP$ySu8 z2IoKnjath4p$*vjq1?JHryB~-@#}>r%qQ8r@&(%yjPoTd06*1SmB+2mnq=$fGFN&b zd{$uKjE}9g3$K{bw!tcF_cVM$#CT*`40qI+i~cTg@S62%!KM3&**7Z?8uNjjHwYKw zUw$wiZm%{hY4>VR9TLLzNo#h>GfUH%ajCP9n!8&p7Rc^bD^dP2IwwnaMh05d+Q3Q6 zCrygu5`Hw{ukodPn7S(fgK2o<__zAIRRZCOmK70LhUxfko%eLBXqo%0UBGOpW70Jf!_-)NI z3w`93t|w_h$FE!v&rLR*=1^{XT~MfHkT7a_PFHSifs$Knjh%gF_{++HNtZ>wt*(r$ zsmn=N=`Q1$Y2G1QLh;j&^fha03Cl}6H#XWNIM@nd3TO%(oe+!TQ)@CmESuq&;toAC zDmI&!c=5UPz58m)JV)EaS?-i@;~wnjnZ2b&n@9VFh2xR1=kw~j!bYXcifY&=JXBBB z-2Njf_EnpP$&wXQX^4NDIn;i-|Ms>~HS@uE>$|@ij=vg?zZ#DJ+lIpf^ESbNu+xpu P>lk0NnWR0ZSn~WGo?zNW literal 0 HcmV?d00001 diff --git a/modules/platforms/cpp/thin-client-test/config/ssl/trust.jks b/modules/platforms/cpp/thin-client-test/config/ssl/trust.jks new file mode 100644 index 0000000000000000000000000000000000000000..a0f3bcfc30787fbe400e3c794f039dbbdd3161e3 GIT binary patch literal 1089 zcmezO_TO6u1_mY|W(3nB$*DypnQ58Hi6yCtr6n2pMVTd)K*_C*_cCF*o2va zLJcJi#6cVm9)aM*%)AoafYg%IqT-~|qI8%PI~FNr14WQ7E*>ezg2d#ERE6Mzw370~ zqErR9{L;LX#FEVXJVP-95s)Ho9$u&-&-A>^l2k(x10j$&GY_|CdVY~YaAHAfdcL8e zfjr!$oQz^ZnZ;mnheWUidih1^26E!OhUSKbMiz$V#)gJwQR2KtAg&pdJEZ;A#HfTE zRgA0*%uS5^3NNJ01V`@{=Phk7nAj{dcb0_nx)cUuDVa zKS>i*?(cNqf4e?&`tmHQ6IKk8{yx zb=^IV!K-e}P~PybetXmuXEUv$$Mqg3!WVmnX{Ly*T(I%?%%$60-Ml(^e5xPL>Sb&Z zxU{J5O!n_GMXAXL3nJI4zZTyeQe7Q7Nm`?mr_nN5Wh-}%Pa&uDWw|NyVm|*=IQ8>* zh}%xJ^<3}R_~P4VZM{{zOILrlmO|j^FB7NxCU1SYT*i{~g5sqMvm}14l5LrN=Et?? zQTe7uiky8O|CY_>JSos#+;eA|4!`);HowNxOw5c7jL0DbOw7O#Vr1}Naj;26ne&Rt z%`I%R8N@R-ZnY#CZwB||mw*l!VU!|2W4-+IeprHtk}qvO2WH`JkPm z_=N?5$uT+!R~eUP_H3K@Njox)r}oo&AI5omE;R}UPMy5TwtB%^LG3@aKB7}*Z=P6x z!#3nf@x2`3r&r$YT6Jl4bECjTSZQGZMl^5!}TpUkd{^R;@Z>Dg|rIgZnTC+W8RCLz& aPIr=dbfVr#$@Hq`rNH^hlc!vtsRsa3Nt9gx literal 0 HcmV?d00001 diff --git a/modules/platforms/cpp/thin-client-test/configure.ac b/modules/platforms/cpp/thin-client-test/configure.ac new file mode 100644 index 0000000000000..61aa5039876be --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/configure.ac @@ -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. +# + +# -*- Autoconf -*- +# Process this file with autoconf to produce a configure script. + +AC_PREREQ([2.69]) +AC_INIT([Apache Ignite C++ Thin Client Test], [2.5.0.26265], [dev@ignite.apache.org], [ignite], [ignite.apache.org]) +AC_CONFIG_SRCDIR(src) + +AC_CANONICAL_SYSTEM +AC_CONFIG_MACRO_DIR([m4]) +AC_LANG([C++]) + +# Initialize automake +AM_INIT_AUTOMAKE([-Wall foreign subdir-objects]) +AC_CONFIG_HEADER(config.h) + +AM_PROG_AR + +# Checks for programs. +GXX="-g -O2" + +AC_PROG_CXX + +# Initialize Libtool +LT_INIT + +# Checks for libraries. +AC_CHECK_LIB([pthread], [pthread_mutex_lock]) + +# Checks for header files. + +# Checks for typedefs, structures, and compiler characteristics. +AC_C_INLINE +AC_TYPE_INT16_T +AC_TYPE_INT32_T +AC_TYPE_INT64_T +AC_TYPE_INT8_T +AC_TYPE_PID_T +AC_TYPE_SIZE_T + +# Checks for library functions. +AC_FUNC_ERROR_AT_LINE + +AC_CONFIG_FILES(Makefile include/Makefile) + +AC_OUTPUT diff --git a/modules/platforms/cpp/thin-client-test/include/Makefile.am b/modules/platforms/cpp/thin-client-test/include/Makefile.am new file mode 100644 index 0000000000000..46a18253d6cfb --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/include/Makefile.am @@ -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. +## + +ACLOCAL_AMFLAGS =-I m4 + +noinst_HEADERS = \ + teamcity/teamcity_messages.h \ + ignite/complex_type.h \ + test_utils.h + diff --git a/modules/platforms/cpp/thin-client-test/include/ignite/complex_type.h b/modules/platforms/cpp/thin-client-test/include/ignite/complex_type.h new file mode 100644 index 0000000000000..6bd0b7f5ecaf7 --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/include/ignite/complex_type.h @@ -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. + */ + +#ifndef _IGNITE_ODBC_TEST_COMPLEX_TYPE +#define _IGNITE_ODBC_TEST_COMPLEX_TYPE + +#include +#include + +namespace ignite +{ + struct InnerObject + { + InnerObject() : + f1(412), + f2("Lorem ipsum") + { + // No-op. + } + + friend bool operator==(const InnerObject& one, const InnerObject& two) + { + return one.f1 == two.f1 && + one.f2 == two.f2; + } + + int32_t f1; + std::string f2; + }; + + struct ComplexType + { + ComplexType() : + i32Field(0) + { + // No-op. + } + + friend bool operator==(const ComplexType& one, const ComplexType& two) + { + return one.i32Field == two.i32Field && + one.objField == two.objField && + one.strField == two.strField; + } + + int32_t i32Field; + InnerObject objField; + std::string strField; + }; +} + +namespace ignite +{ + namespace binary + { + + IGNITE_BINARY_TYPE_START(ignite::InnerObject) + + typedef ignite::InnerObject InnerObject; + + IGNITE_BINARY_GET_TYPE_ID_AS_HASH(InnerObject) + IGNITE_BINARY_GET_TYPE_NAME_AS_IS(InnerObject) + IGNITE_BINARY_GET_FIELD_ID_AS_HASH + IGNITE_BINARY_IS_NULL_FALSE(InnerObject) + IGNITE_BINARY_GET_NULL_DEFAULT_CTOR(InnerObject) + + static void Write(BinaryWriter& writer, const InnerObject& obj) + { + writer.WriteInt32("f1", obj.f1); + writer.WriteString("f2", obj.f2); + } + + static void Read(BinaryReader& reader, InnerObject& dst) + { + dst.f1 = reader.ReadInt32("f1"); + dst.f2 = reader.ReadString("f2"); + } + + IGNITE_BINARY_TYPE_END + + IGNITE_BINARY_TYPE_START(ignite::ComplexType) + + typedef ignite::ComplexType ComplexType; + + IGNITE_BINARY_GET_TYPE_ID_AS_HASH(ComplexType) + IGNITE_BINARY_GET_TYPE_NAME_AS_IS(ComplexType) + IGNITE_BINARY_GET_FIELD_ID_AS_HASH + IGNITE_BINARY_IS_NULL_FALSE(ComplexType) + IGNITE_BINARY_GET_NULL_DEFAULT_CTOR(ComplexType) + + static void Write(BinaryWriter& writer, const ComplexType& obj) + { + writer.WriteInt32("i32Field", obj.i32Field); + writer.WriteObject("objField", obj.objField); + writer.WriteString("strField", obj.strField); + } + + static void Read(BinaryReader& reader, ComplexType& dst) + { + dst.i32Field = reader.ReadInt32("i32Field"); + dst.objField = reader.ReadObject("objField"); + dst.strField = reader.ReadString("strField"); + } + + IGNITE_BINARY_TYPE_END + } +}; + +#endif // _IGNITE_ODBC_TEST_COMPLEX_TYPE diff --git a/modules/platforms/cpp/thin-client-test/include/teamcity/teamcity_messages.h b/modules/platforms/cpp/thin-client-test/include/teamcity/teamcity_messages.h new file mode 100644 index 0000000000000..81a915b89d704 --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/include/teamcity/teamcity_messages.h @@ -0,0 +1,55 @@ +/* Copyright 2011 JetBrains s.r.o. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * $Revision: 88625 $ +*/ + +#ifndef H_TEAMCITY_MESSAGES +#define H_TEAMCITY_MESSAGES + +#include +#include + +namespace JetBrains { + +std::string getFlowIdFromEnvironment(); +bool underTeamcity(); + +class TeamcityMessages { + std::ostream *m_out; + +protected: + std::string escape(std::string s); + + void openMsg(const std::string &name); + void writeProperty(std::string name, std::string value); + void closeMsg(); + +public: + TeamcityMessages(); + + void setOutput(std::ostream &); + + void suiteStarted(std::string name, std::string flowid = ""); + void suiteFinished(std::string name, std::string flowid = ""); + + void testStarted(std::string name, std::string flowid = ""); + void testFailed(std::string name, std::string message, std::string details, std::string flowid = ""); + void testIgnored(std::string name, std::string message, std::string flowid = ""); + void testFinished(std::string name, int durationMs = -1, std::string flowid = ""); +}; + +} + +#endif /* H_TEAMCITY_MESSAGES */ diff --git a/modules/platforms/cpp/thin-client-test/include/test_utils.h b/modules/platforms/cpp/thin-client-test/include/test_utils.h new file mode 100644 index 0000000000000..653e7da19f905 --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/include/test_utils.h @@ -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. + */ + +#ifndef _IGNITE_THIN_CLIENT_TEST_TEST_UTILS +#define _IGNITE_THIN_CLIENT_TEST_TEST_UTILS + +#include + +#include + +namespace ignite_test +{ + /** + * @return Test config directory path. + */ + std::string GetTestConfigDir(); + + /** + * Initialize configuration for a node. + * + * Inits Ignite node configuration from specified config file. + * Config file is searched in path specified by IGNITE_NATIVE_TEST_CPP_THIN_CONFIG_PATH + * environmental variable. + * + * @param cfg Ignite config. + * @param cfgFile Ignite node config file name without path. + */ + void InitConfig(ignite::IgniteConfiguration& cfg, const char* cfgFile); + + /** + * Start Ignite node. + * + * Starts new Ignite node with the specified name and from specified config file. + * Config file is searched in path specified by IGNITE_NATIVE_TEST_CPP_THIN_CONFIG_PATH + * environmental variable. + * + * @param cfgFile Ignite node config file name without path. + * @param name Node name. + * @return New node. + */ + ignite::Ignite StartServerNode(const char* cfgFile, const char* name); + + /** + * Start Ignite node with config path corrected for specific platform. + * + * @param cfgFile Ignite node config file name without path. + * @param name Node name. + * @return New node. + */ + ignite::Ignite StartCrossPlatformServerNode(const char* cfgFile, const char* name); + + /** + * Remove all the LFS artifacts. + */ + void ClearLfs(); +} + +#endif // _IGNITE_THIN_CLIENT_TEST_TEST_UTILS \ No newline at end of file diff --git a/modules/platforms/cpp/thin-client-test/project/vs/thin-client-test.vcxproj b/modules/platforms/cpp/thin-client-test/project/vs/thin-client-test.vcxproj new file mode 100644 index 0000000000000..5d5729f9be9f7 --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/project/vs/thin-client-test.vcxproj @@ -0,0 +1,186 @@ + + + + + Debug + Win32 + + + Debug + x64 + + + Release + Win32 + + + Release + x64 + + + + + + + + + + + + + + + + + + + {4f15669b-92eb-49f0-b774-8f19bae0b960} + + + {b63f2e01-5157-4719-8491-0e1c7cd3b701} + + + {e2dea693-f2ea-43c2-a813-053378f6e4db} + + + {4f7e4917-4612-4b96-9838-025711ade391} + + + {5c037386-b5f5-4a58-9ee2-3d3a508aa866} + + + + + + + + + + + + + + + {5662F10A-9C40-45D6-AFF8-E93573FEAABA} + thinclienttest + + + + Application + true + v100 + Unicode + + + Application + true + v100 + MultiByte + + + Application + false + v100 + true + MultiByte + + + Application + false + v100 + true + MultiByte + + + + + + + + + + + + + + + + + + + $(SolutionDir)$(Platform)\$(Configuration)\ + $(Platform)\$(Configuration)\ + + + $(SolutionDir)$(Platform)\$(Configuration)\ + $(Platform)\$(Configuration)\ + + + + Level3 + Disabled + true + $(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\jni\include;$(ProjectDir)\..\..\..\jni\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\..\core\include;$(ProjectDir)\..\..\..\core\os\win\include;$(ProjectDir)\..\..\..\thin-client\include;$(ProjectDir)\..\..\..\thin-client\os\win\include;$(ProjectDir)\..\..\include;$(BOOST_HOME) + _DEBUG;IGNITE_IMPL;BOOST_DATE_TIME_NO_LIB;BOOST_REGEX_NO_LIB;_CRT_SECURE_NO_WARNINGS;IGNITE_FRIEND;_CRTDBG_MAP_ALLOC;%(PreprocessorDefinitions) + Async + + + true + $(BOOST_HOME)\lib64-msvc-10.0\libboost_unit_test_framework-vc100-mt-gd-1_58.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_thread-vc100-mt-gd-1_58.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_system-vc100-mt-gd-1_58.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_chrono-vc100-mt-gd-1_58.lib;%(AdditionalDependencies) + Console + + + + + Level3 + Disabled + true + $(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\jni\include;$(ProjectDir)\..\..\..\jni\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\..\core\include;$(ProjectDir)\..\..\..\core\os\win\include;$(ProjectDir)\..\..\..\thin-client\include;$(ProjectDir)\..\..\..\thin-client\os\win\include;$(ProjectDir)\..\..\include;$(BOOST_HOME) + _DEBUG;IGNITE_IMPL;BOOST_DATE_TIME_NO_LIB;BOOST_REGEX_NO_LIB;_CRT_SECURE_NO_WARNINGS;IGNITE_FRIEND;IGNITE_TESTS_32;%(PreprocessorDefinitions) + Async + + + true + $(BOOST_HOME)\lib32-msvc-10.0\libboost_unit_test_framework-vc100-mt-gd-1_58.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_thread-vc100-mt-gd-1_58.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_system-vc100-mt-gd-1_58.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_chrono-vc100-mt-gd-1_58.lib;%(AdditionalDependencies) + + + + + Level3 + MaxSpeed + true + true + true + $(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\jni\include;$(ProjectDir)\..\..\..\jni\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\..\core\include;$(ProjectDir)\..\..\..\core\os\win\include;$(ProjectDir)\..\..\..\thin-client\include;$(ProjectDir)\..\..\..\thin-client\os\win\include;$(ProjectDir)\..\..\include;$(BOOST_HOME) + NDEBUG;IGNITE_IMPL;BOOST_DATE_TIME_NO_LIB;BOOST_REGEX_NO_LIB;_CRT_SECURE_NO_WARNINGS;IGNITE_FRIEND;%(PreprocessorDefinitions) + Async + + + true + true + true + $(BOOST_HOME)\lib64-msvc-10.0\libboost_unit_test_framework-vc100-mt-1_58.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_thread-vc100-mt-1_58.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_system-vc100-mt-1_58.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_chrono-vc100-mt-1_58.lib;%(AdditionalDependencies) + + + + + Level3 + MaxSpeed + true + true + true + $(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\jni\include;$(ProjectDir)\..\..\..\jni\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\..\core\include;$(ProjectDir)\..\..\..\core\os\win\include;$(ProjectDir)\..\..\..\thin-client\include;$(ProjectDir)\..\..\..\thin-client\os\win\include;$(ProjectDir)\..\..\include;$(BOOST_HOME) + NDEBUG;IGNITE_IMPL;BOOST_DATE_TIME_NO_LIB;BOOST_REGEX_NO_LIB;_CRT_SECURE_NO_WARNINGS;IGNITE_FRIEND;IGNITE_TESTS_32;%(PreprocessorDefinitions) + Async + + + true + true + true + $(BOOST_HOME)\lib32-msvc-10.0\libboost_unit_test_framework-vc100-mt-1_58.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_thread-vc100-mt-1_58.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_system-vc100-mt-1_58.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_chrono-vc100-mt-1_58.lib;%(AdditionalDependencies) + + + + + + diff --git a/modules/platforms/cpp/thin-client-test/project/vs/thin-client-test.vcxproj.filters b/modules/platforms/cpp/thin-client-test/project/vs/thin-client-test.vcxproj.filters new file mode 100644 index 0000000000000..e8aee85229aae --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/project/vs/thin-client-test.vcxproj.filters @@ -0,0 +1,82 @@ + + + + + {93995380-89BD-4b04-88EB-625FBE52EBFB} + h;hh;hpp;hxx;hm;inl;inc;ipp;xsd + + + {4FC737F1-C7A5-4376-A066-2A32D752A2FF} + cpp;c;cc;cxx;def;odl;idl;hpj;bat;asm;asmx + + + {0ed957ce-09b9-495f-8f11-7ed2b6a98c9b} + + + {1a71111d-792b-48d8-b1af-044ebf8cec0e} + + + + + Code + + + TeamCity + + + TeamCity + + + Code + + + Code + + + Code + + + Code + + + + + TeamCity + + + Code + + + Code\Types + + + + + Configs + + + Configs + + + Configs + + + Configs + + + Configs + + + Configs + + + Configs + + + Configs + + + Configs + + + diff --git a/modules/platforms/cpp/thin-client-test/src/auth_test.cpp b/modules/platforms/cpp/thin-client-test/src/auth_test.cpp new file mode 100644 index 0000000000000..d241a9d303792 --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/src/auth_test.cpp @@ -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. + */ + +#ifndef _MSC_VER +# define BOOST_TEST_DYN_LINK +#endif + +#include + +#include + +#include +#include + +#include + +using namespace ignite::thin; +using namespace boost::unit_test; + +class AuthTestSuiteFixture +{ +public: + AuthTestSuiteFixture() + { + ignite_test::ClearLfs(); + + serverNode = ignite_test::StartCrossPlatformServerNode("auth.xml", "ServerNode"); + } + + ~AuthTestSuiteFixture() + { + ignite::Ignition::StopAll(false); + } + +private: + /** Server node. */ + ignite::Ignite serverNode; +}; + +BOOST_FIXTURE_TEST_SUITE(AuthTestSuite, AuthTestSuiteFixture) + +BOOST_AUTO_TEST_CASE(AuthSuccess) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + cfg.SetUser("ignite"); + cfg.SetPassword("ignite"); + + IgniteClient::Start(cfg); +} + +BOOST_AUTO_TEST_CASE(AuthReject) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + + BOOST_CHECK_THROW(IgniteClient::Start(cfg), ignite::IgniteError); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/thin-client-test/src/cache_client_test.cpp b/modules/platforms/cpp/thin-client-test/src/cache_client_test.cpp new file mode 100644 index 0000000000000..5594134b4c498 --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/src/cache_client_test.cpp @@ -0,0 +1,743 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 _MSC_VER +# define BOOST_TEST_DYN_LINK +#endif + +#include +#include + +#include + +#include +#include + +#include + +#include +#include + +using namespace ignite::thin; +using namespace boost::unit_test; + +class CacheClientTestSuiteFixture +{ +public: + static ignite::Ignite StartNode(const char* name) + { + return ignite_test::StartCrossPlatformServerNode("cache.xml", name); + } + + CacheClientTestSuiteFixture() + { + serverNode = StartNode("ServerNode"); + } + + ~CacheClientTestSuiteFixture() + { + ignite::Ignition::StopAll(false); + } + + template + void LocalPeek(cache::CacheClient& cache, const K& key, V& value) + { + using namespace ignite::impl::thin; + using namespace ignite::impl::thin::cache; + + CacheClientProxy& proxy = CacheClientProxy::GetFromCacheClient(cache); + + WritableKeyImpl wkey(key); + ReadableImpl rvalue(value); + + proxy.LocalPeek(wkey, rvalue); + } + + template + void NumPartitionTest(int64_t num) + { + StartNode("node1"); + + boost::this_thread::sleep_for(boost::chrono::seconds(2)); + + IgniteClientConfiguration cfg; + cfg.SetEndPoints("127.0.0.1:11110..11120"); + + IgniteClient client = IgniteClient::Start(cfg); + + cache::CacheClient cache = + client.GetCache("partitioned"); + + cache.RefreshAffinityMapping(); + + for (int64_t i = 1; i < num; ++i) + cache.Put(static_cast(i * 39916801), i * 5039); + + for (int64_t i = 1; i < num; ++i) + { + int64_t val; + LocalPeek(cache, static_cast(i * 39916801), val); + + BOOST_CHECK_EQUAL(val, i * 5039); + } + } + + void SizeTest(int32_t peekMode) + { + IgniteClientConfiguration cfg; + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient client = IgniteClient::Start(cfg); + + cache::CacheClient cache = + client.GetCache("partitioned"); + + for (int32_t i = 0; i < 1000; ++i) + cache.Put(i, i * 5039); + + int64_t size = cache.GetSize(peekMode); + BOOST_CHECK_EQUAL(size, 1000); + } + +private: + /** Server node. */ + ignite::Ignite serverNode; +}; + +BOOST_FIXTURE_TEST_SUITE(CacheClientTestSuite, CacheClientTestSuiteFixture) + +BOOST_AUTO_TEST_CASE(CacheClientGetCacheExisting) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient client = IgniteClient::Start(cfg); + + client.GetCache("local"); +} + +BOOST_AUTO_TEST_CASE(CacheClientGetCacheNonxisting) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient client = IgniteClient::Start(cfg); + + BOOST_REQUIRE_THROW((client.GetCache("unknown")), ignite::IgniteError); +} + +BOOST_AUTO_TEST_CASE(CacheClientGetOrCreateCacheExisting) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient client = IgniteClient::Start(cfg); + + client.GetOrCreateCache("local"); +} + +BOOST_AUTO_TEST_CASE(CacheClientGetOrCreateCacheNonexisting) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient client = IgniteClient::Start(cfg); + + client.GetOrCreateCache("unknown"); +} + +BOOST_AUTO_TEST_CASE(CacheClientCreateCacheExisting) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient client = IgniteClient::Start(cfg); + + BOOST_REQUIRE_THROW((client.CreateCache("local")), ignite::IgniteError); +} + +BOOST_AUTO_TEST_CASE(CacheClientCreateCacheNonexisting) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient client = IgniteClient::Start(cfg); + + client.CreateCache("unknown"); +} + +BOOST_AUTO_TEST_CASE(CacheClientDestroyCacheExisting) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient client = IgniteClient::Start(cfg); + + client.DestroyCache("local"); +} + +BOOST_AUTO_TEST_CASE(CacheClientDestroyCacheNonexisting) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient client = IgniteClient::Start(cfg); + + BOOST_REQUIRE_THROW(client.DestroyCache("unknown"), ignite::IgniteError); +} + +BOOST_AUTO_TEST_CASE(CacheClientGetCacheNames) +{ + std::set expectedNames; + + expectedNames.insert("local_atomic"); + expectedNames.insert("partitioned_atomic_near"); + expectedNames.insert("partitioned_near"); + expectedNames.insert("partitioned2"); + expectedNames.insert("partitioned"); + expectedNames.insert("replicated"); + expectedNames.insert("replicated_atomic"); + expectedNames.insert("local"); + expectedNames.insert("partitioned_atomic"); + + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient client = IgniteClient::Start(cfg); + + std::vector caches; + + client.GetCacheNames(caches); + + BOOST_CHECK_EQUAL(expectedNames.size(), caches.size()); + + for (std::vector::const_iterator it = caches.begin(); it != caches.end(); ++it) + { + BOOST_CHECK_EQUAL(expectedNames.count(*it), 1); + } +} + +BOOST_AUTO_TEST_CASE(CacheClientPutGetBasicKeyValue) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient client = IgniteClient::Start(cfg); + + cache::CacheClient cache = client.GetCache("local"); + + int32_t key = 42; + std::string valIn = "Lorem ipsum"; + + cache.Put(key, valIn); + + std::string valOut = cache.Get(key); + + BOOST_CHECK_EQUAL(valOut, valIn); +} + +BOOST_AUTO_TEST_CASE(CacheClientPutGetComplexValue) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient client = IgniteClient::Start(cfg); + + cache::CacheClient cache = client.GetCache("local"); + + ignite::ComplexType valIn; + + int32_t key = 42; + + valIn.i32Field = 123; + valIn.strField = "Test value"; + valIn.objField.f1 = 42; + valIn.objField.f2 = "Inner value"; + + cache.Put(key, valIn); + + ignite::ComplexType valOut = cache.Get(key); + + BOOST_CHECK_EQUAL(valIn.i32Field, valOut.i32Field); + BOOST_CHECK_EQUAL(valIn.strField, valOut.strField); + BOOST_CHECK_EQUAL(valIn.objField.f1, valOut.objField.f1); + BOOST_CHECK_EQUAL(valIn.objField.f2, valOut.objField.f2); +} + +BOOST_AUTO_TEST_CASE(CacheClientPutGetComplexKey) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient client = IgniteClient::Start(cfg); + + cache::CacheClient cache = client.GetCache("local"); + + ignite::ComplexType key; + + key.i32Field = 123; + key.strField = "Test value"; + key.objField.f1 = 42; + key.objField.f2 = "Inner value"; + + int32_t valIn = 42; + + cache.Put(key, valIn); + + int32_t valOut = cache.Get(key); + + BOOST_CHECK_EQUAL(valIn, valOut); +} + +BOOST_AUTO_TEST_CASE(CacheClientContainsBasicKey) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient client = IgniteClient::Start(cfg); + + cache::CacheClient cache = client.GetCache("local"); + + int32_t key = 42; + std::string valIn = "Lorem ipsum"; + + BOOST_CHECK(!cache.ContainsKey(key)); + + cache.Put(key, valIn); + + BOOST_CHECK(cache.ContainsKey(key)); +} + +BOOST_AUTO_TEST_CASE(CacheClientContainsComplexKey) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient client = IgniteClient::Start(cfg); + + cache::CacheClient cache = client.GetCache("local"); + + ignite::ComplexType key; + + key.i32Field = 123; + key.strField = "Test value"; + key.objField.f1 = 42; + key.objField.f2 = "Inner value"; + + int32_t valIn = 42; + + BOOST_CHECK(!cache.ContainsKey(key)); + + cache.Put(key, valIn); + + BOOST_CHECK(cache.ContainsKey(key)); +} + +BOOST_AUTO_TEST_CASE(CacheClientPartitionsInt8) +{ + NumPartitionTest(100); +} + +BOOST_AUTO_TEST_CASE(CacheClientPartitionsInt16) +{ + NumPartitionTest(2000); +} + +BOOST_AUTO_TEST_CASE(CacheClientPartitionsInt32) +{ + NumPartitionTest(1050); +} + +BOOST_AUTO_TEST_CASE(CacheClientPartitionsInt64) +{ + NumPartitionTest(2000); +} + +BOOST_AUTO_TEST_CASE(CacheClientPartitionsUint16) +{ + NumPartitionTest(1500); +} + +BOOST_AUTO_TEST_CASE(CacheClientPartitionsFloat) +{ + NumPartitionTest(1500); +} + +BOOST_AUTO_TEST_CASE(CacheClientPartitionsDouble) +{ + NumPartitionTest(500); +} + +BOOST_AUTO_TEST_CASE(CacheClientPartitionsString) +{ + StartNode("node1"); + + boost::this_thread::sleep_for(boost::chrono::seconds(2)); + + IgniteClientConfiguration cfg; + cfg.SetEndPoints("127.0.0.1:11110..11120"); + + IgniteClient client = IgniteClient::Start(cfg); + + cache::CacheClient cache = + client.GetCache("partitioned"); + + cache.RefreshAffinityMapping(); + + for (int64_t i = 1; i < 1000; ++i) + cache.Put(ignite::common::LexicalCast(i * 39916801), i * 5039); + + for (int64_t i = 1; i < 1000; ++i) + { + int64_t val; + LocalPeek(cache, ignite::common::LexicalCast(i * 39916801), val); + + BOOST_CHECK_EQUAL(val, i * 5039); + } +} + +BOOST_AUTO_TEST_CASE(CacheClientPartitionsGuid) +{ + StartNode("node1"); + + boost::this_thread::sleep_for(boost::chrono::seconds(2)); + + IgniteClientConfiguration cfg; + cfg.SetEndPoints("127.0.0.1:11110..11120"); + + IgniteClient client = IgniteClient::Start(cfg); + + cache::CacheClient cache = + client.GetCache("partitioned"); + + cache.RefreshAffinityMapping(); + + for (int64_t i = 1; i < 1000; ++i) + cache.Put(ignite::Guid(i * 406586897, i * 87178291199), i * 5039); + + for (int64_t i = 1; i < 1000; ++i) + { + int64_t val; + LocalPeek(cache, ignite::Guid(i * 406586897, i * 87178291199), val); + + BOOST_CHECK_EQUAL(val, i * 5039); + } +} + +BOOST_AUTO_TEST_CASE(CacheClientPartitionsComplexType) +{ + StartNode("node1"); + + boost::this_thread::sleep_for(boost::chrono::seconds(2)); + + IgniteClientConfiguration cfg; + cfg.SetEndPoints("127.0.0.1:11110..11120"); + + IgniteClient client = IgniteClient::Start(cfg); + + cache::CacheClient cache = + client.GetCache("partitioned"); + + cache.RefreshAffinityMapping(); + + for (int64_t i = 1; i < 1000; ++i) + { + ignite::ComplexType key; + + key.i32Field = static_cast(i * 406586897); + key.strField = ignite::common::LexicalCast(i * 39916801); + key.objField.f1 = static_cast(i * 87178291199); + key.objField.f2 = ignite::common::LexicalCast(i * 59969537); + + cache.Put(key, i * 5039); + } + + for (int64_t i = 1; i < 1000; ++i) + { + ignite::ComplexType key; + + key.i32Field = static_cast(i * 406586897); + key.strField = ignite::common::LexicalCast(i * 39916801); + key.objField.f1 = static_cast(i * 87178291199); + key.objField.f2 = ignite::common::LexicalCast(i * 59969537); + + int64_t val; + LocalPeek(cache, key, val); + + BOOST_CHECK_EQUAL(val, i * 5039); + } +} + +BOOST_AUTO_TEST_CASE(CacheClientPartitionsDate) +{ + StartNode("node1"); + + boost::this_thread::sleep_for(boost::chrono::seconds(2)); + + IgniteClientConfiguration cfg; + cfg.SetEndPoints("127.0.0.1:11110..11120"); + + IgniteClient client = IgniteClient::Start(cfg); + + cache::CacheClient cache = + client.GetCache("partitioned"); + + cache.RefreshAffinityMapping(); + + for (int64_t i = 1; i < 1000; ++i) + cache.Put(ignite::common::MakeDateGmt( + static_cast(1990 + i), + std::abs(static_cast((i * 87178291199) % 11) + 1), + std::abs(static_cast((i * 39916801) % 27) + 1), + std::abs(static_cast(9834497 * i) % 24), + std::abs(static_cast(i * 87178291199) % 60), + std::abs(static_cast(i * 39916801) % 60)), + i * 5039); + + for (int64_t i = 1; i < 1000; ++i) + { + int64_t val; + LocalPeek(cache, ignite::common::MakeDateGmt( + static_cast(1990 + i), + std::abs(static_cast((i * 87178291199) % 11) + 1), + std::abs(static_cast((i * 39916801) % 27) + 1), + std::abs(static_cast(9834497 * i) % 24), + std::abs(static_cast(i * 87178291199) % 60), + std::abs(static_cast(i * 39916801) % 60)), + val); + + BOOST_CHECK_EQUAL(val, i * 5039); + } +} + +BOOST_AUTO_TEST_CASE(CacheClientPartitionsTime) +{ + StartNode("node1"); + + boost::this_thread::sleep_for(boost::chrono::seconds(2)); + + IgniteClientConfiguration cfg; + cfg.SetEndPoints("127.0.0.1:11110..11120"); + + IgniteClient client = IgniteClient::Start(cfg); + + cache::CacheClient cache = + client.GetCache("partitioned"); + + cache.RefreshAffinityMapping(); + + for (int64_t i = 1; i < 100; ++i) + cache.Put(ignite::common::MakeTimeGmt( + std::abs(static_cast(9834497 * i) % 24), + std::abs(static_cast(i * 87178291199) % 60), + std::abs(static_cast(i * 39916801) % 60)), + i * 5039); + + for (int64_t i = 1; i < 100; ++i) + { + int64_t val; + LocalPeek(cache, ignite::common::MakeTimeGmt( + std::abs(static_cast(9834497 * i) % 24), + std::abs(static_cast(i * 87178291199) % 60), + std::abs(static_cast(i * 39916801) % 60)), + val); + + BOOST_CHECK_EQUAL(val, i * 5039); + } +} + +BOOST_AUTO_TEST_CASE(CacheClientPartitionsTimestamp) +{ + StartNode("node1"); + + boost::this_thread::sleep_for(boost::chrono::seconds(2)); + + IgniteClientConfiguration cfg; + cfg.SetEndPoints("127.0.0.1:11110..11120"); + + IgniteClient client = IgniteClient::Start(cfg); + + cache::CacheClient cache = + client.GetCache("partitioned"); + + cache.RefreshAffinityMapping(); + + for (int64_t i = 1; i < 1000; ++i) + cache.Put(ignite::common::MakeTimestampGmt( + static_cast(1990 + i), + std::abs(static_cast(i * 87178291199) % 11 + 1), + std::abs(static_cast(i * 39916801) % 28), + std::abs(static_cast(9834497 * i) % 24), + std::abs(static_cast(i * 87178291199) % 60), + std::abs(static_cast(i * 39916801) % 60), + std::abs(static_cast((i * 303595777) % 1000000000))), + i * 5039); + + for (int64_t i = 1; i < 1000; ++i) + { + int64_t val; + LocalPeek(cache, ignite::common::MakeTimestampGmt( + static_cast(1990 + i), + std::abs(static_cast(i * 87178291199) % 11 + 1), + std::abs(static_cast(i * 39916801) % 28), + std::abs(static_cast(9834497 * i) % 24), + std::abs(static_cast(i * 87178291199) % 60), + std::abs(static_cast(i * 39916801) % 60), + std::abs(static_cast((i * 303595777) % 1000000000))), + val); + + BOOST_CHECK_EQUAL(val, i * 5039); + } +} + +BOOST_AUTO_TEST_CASE(CacheClientGetSizeAll) +{ + SizeTest(cache::CachePeekMode::ALL); +} + +BOOST_AUTO_TEST_CASE(CacheClientGetSizePrimary) +{ + SizeTest(cache::CachePeekMode::PRIMARY); +} + +BOOST_AUTO_TEST_CASE(CacheClientGetSizeOnheap) +{ + SizeTest(cache::CachePeekMode::ONHEAP); +} + +BOOST_AUTO_TEST_CASE(CacheClientGetSizeSeveral) +{ + using cache::CachePeekMode; + + SizeTest( + CachePeekMode::NEAR_CACHE | + CachePeekMode::PRIMARY | + CachePeekMode::BACKUP | + CachePeekMode::ONHEAP | + CachePeekMode::OFFHEAP + ); +} + +BOOST_AUTO_TEST_CASE(CacheClientRemoveAll) +{ + IgniteClientConfiguration cfg; + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient client = IgniteClient::Start(cfg); + + cache::CacheClient cache = + client.GetCache("partitioned"); + + for (int32_t i = 0; i < 1000; ++i) + cache.Put(i, i * 5039); + + int64_t size = cache.GetSize(cache::CachePeekMode::ALL); + BOOST_CHECK_EQUAL(size, 1000); + + cache.RemoveAll(); + + size = cache.GetSize(cache::CachePeekMode::ALL); + BOOST_CHECK_EQUAL(size, 0); +} + +BOOST_AUTO_TEST_CASE(CacheClientClearAll) +{ + IgniteClientConfiguration cfg; + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient client = IgniteClient::Start(cfg); + + cache::CacheClient cache = + client.GetCache("partitioned"); + + for (int32_t i = 0; i < 1000; ++i) + cache.Put(i, i * 5039); + + int64_t size = cache.GetSize(cache::CachePeekMode::ALL); + BOOST_CHECK_EQUAL(size, 1000); + + cache.Clear(); + + size = cache.GetSize(cache::CachePeekMode::ALL); + BOOST_CHECK_EQUAL(size, 0); +} + +BOOST_AUTO_TEST_CASE(CacheClientRemove) +{ + IgniteClientConfiguration cfg; + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient client = IgniteClient::Start(cfg); + + cache::CacheClient cache = + client.GetCache("partitioned"); + + for (int32_t i = 0; i < 1000; ++i) + cache.Put(i, i * 5039); + + int64_t size = cache.GetSize(cache::CachePeekMode::ALL); + BOOST_CHECK_EQUAL(size, 1000); + + for (int32_t i = 0; i < 1000; ++i) + { + BOOST_CHECK(cache.Remove(i)); + + size = cache.GetSize(cache::CachePeekMode::ALL); + BOOST_CHECK_EQUAL(size, 1000 - i - 1); + } +} + +BOOST_AUTO_TEST_CASE(CacheClientClear) +{ + IgniteClientConfiguration cfg; + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient client = IgniteClient::Start(cfg); + + cache::CacheClient cache = + client.GetCache("partitioned"); + + for (int32_t i = 0; i < 1000; ++i) + cache.Put(i, i * 5039); + + int64_t size = cache.GetSize(cache::CachePeekMode::ALL); + BOOST_CHECK_EQUAL(size, 1000); + + for (int32_t i = 0; i < 1000; ++i) + { + cache.Clear(i); + + size = cache.GetSize(cache::CachePeekMode::ALL); + BOOST_CHECK_EQUAL(size, 1000 - i - 1); + } +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/thin-client-test/src/ignite_client_test.cpp b/modules/platforms/cpp/thin-client-test/src/ignite_client_test.cpp new file mode 100644 index 0000000000000..4dbc7f76104d3 --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/src/ignite_client_test.cpp @@ -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. + */ + +#ifndef _MSC_VER +# define BOOST_TEST_DYN_LINK +#endif + +#include + +#include + +#include +#include + +#include + +using namespace ignite::thin; +using namespace boost::unit_test; + +class IgniteClientTestSuiteFixture +{ +public: + IgniteClientTestSuiteFixture() + { + serverNode = ignite_test::StartCrossPlatformServerNode("cache.xml", "ServerNode"); + } + + ~IgniteClientTestSuiteFixture() + { + ignite::Ignition::StopAll(false); + } + +private: + /** Server node. */ + ignite::Ignite serverNode; +}; + +BOOST_FIXTURE_TEST_SUITE(IgniteClientTestSuite, IgniteClientTestSuiteFixture) + +BOOST_AUTO_TEST_CASE(IgniteClientConnection) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + + IgniteClient::Start(cfg); +} + +BOOST_AUTO_TEST_CASE(IgniteClientConnectionFailover) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11109..11111"); + + IgniteClient::Start(cfg); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/thin-client-test/src/ssl_test.cpp b/modules/platforms/cpp/thin-client-test/src/ssl_test.cpp new file mode 100644 index 0000000000000..6bfda493a3005 --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/src/ssl_test.cpp @@ -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. + */ + +#ifndef _MSC_VER +# define BOOST_TEST_DYN_LINK +#endif + +#include + +#include + +#include +#include + +#include + +using namespace ignite::thin; +using namespace boost::unit_test; + +class SslTestSuiteFixture +{ +public: + SslTestSuiteFixture() + { + serverNode = ignite_test::StartCrossPlatformServerNode("ssl.xml", "ServerNode"); + } + + ~SslTestSuiteFixture() + { + ignite::Ignition::StopAll(false); + } + + std::string GetConfigFile(const std::string& file) + { + using namespace ignite::common; + std::stringstream pathBuilder; + + pathBuilder << ignite_test::GetTestConfigDir() << Fs << "ssl" << Fs << file; + + return pathBuilder.str(); + } + +private: + /** Server node. */ + ignite::Ignite serverNode; +}; + +BOOST_FIXTURE_TEST_SUITE(SslTestSuite, SslTestSuiteFixture) + +BOOST_AUTO_TEST_CASE(SslConnectionSuccess) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + + cfg.SetSslMode(SslMode::REQUIRE); + cfg.SetSslCertFile(GetConfigFile("client_full.pem")); + cfg.SetSslKeyFile(GetConfigFile("client_full.pem")); + cfg.SetSslCaFile(GetConfigFile("ca.pem")); + + IgniteClient::Start(cfg); +} + +BOOST_AUTO_TEST_CASE(SslConnectionReject) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + + cfg.SetSslMode(SslMode::REQUIRE); + cfg.SetSslCertFile(GetConfigFile("client_unknown.pem")); + cfg.SetSslKeyFile(GetConfigFile("client_unknown.pem")); + cfg.SetSslCaFile(GetConfigFile("ca.pem")); + + BOOST_CHECK_THROW(IgniteClient::Start(cfg), ignite::IgniteError); +} + +BOOST_AUTO_TEST_CASE(SslConnectionReject2) +{ + IgniteClientConfiguration cfg; + + cfg.SetEndPoints("127.0.0.1:11110"); + + cfg.SetSslMode(SslMode::DISABLE); + + BOOST_CHECK_THROW(IgniteClient::Start(cfg), ignite::IgniteError); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/modules/platforms/cpp/thin-client-test/src/teamcity/teamcity_boost.cpp b/modules/platforms/cpp/thin-client-test/src/teamcity/teamcity_boost.cpp new file mode 100644 index 0000000000000..2f3afe9ad4032 --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/src/teamcity/teamcity_boost.cpp @@ -0,0 +1,159 @@ +/* Copyright 2011 JetBrains s.r.o. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * $Revision: 88625 $ +*/ + +#define BOOST_TEST_MODULE IgniteThinClientTest + +#include + +#include +#include +#include +#include +#include + +#include "teamcity/teamcity_messages.h" + +using namespace boost::unit_test; +using namespace std; + +namespace JetBrains { + +// Custom formatter for TeamCity messages +class TeamcityBoostLogFormatter: public boost::unit_test::unit_test_log_formatter { + TeamcityMessages messages; + std::string currentDetails; + std::string flowId; + +public: + TeamcityBoostLogFormatter(const std::string &_flowId); + TeamcityBoostLogFormatter(); + + void log_start(std::ostream&, boost::unit_test::counter_t test_cases_amount); + void log_finish(std::ostream&); + void log_build_info(std::ostream&); + + void test_unit_start(std::ostream&, boost::unit_test::test_unit const& tu); + void test_unit_finish(std::ostream&, + boost::unit_test::test_unit const& tu, + unsigned long elapsed); + void test_unit_skipped(std::ostream&, boost::unit_test::test_unit const& tu); + + void log_exception(std::ostream&, + boost::unit_test::log_checkpoint_data const&, + boost::unit_test::const_string explanation); + + void log_entry_start(std::ostream&, + boost::unit_test::log_entry_data const&, + log_entry_types let); + void log_entry_value(std::ostream&, boost::unit_test::const_string value); + void log_entry_finish(std::ostream&); +}; + +// Fake fixture to register formatter +struct TeamcityFormatterRegistrar { + TeamcityFormatterRegistrar() { + if (JetBrains::underTeamcity()) { + boost::unit_test::unit_test_log.set_formatter(new JetBrains::TeamcityBoostLogFormatter()); + boost::unit_test::unit_test_log.set_threshold_level(boost::unit_test::log_successful_tests); + } + } +}; +BOOST_GLOBAL_FIXTURE(TeamcityFormatterRegistrar); + +// Formatter implementation +string toString(const_string bstr) { + stringstream ss; + + ss << bstr; + + return ss.str(); +} + +TeamcityBoostLogFormatter::TeamcityBoostLogFormatter(const std::string &_flowId) +: flowId(_flowId) +{} + +TeamcityBoostLogFormatter::TeamcityBoostLogFormatter() +: flowId(getFlowIdFromEnvironment()) +{} + +void TeamcityBoostLogFormatter::log_start(ostream &out, counter_t test_cases_amount) +{} + +void TeamcityBoostLogFormatter::log_finish(ostream &out) +{} + +void TeamcityBoostLogFormatter::log_build_info(ostream &out) +{} + +void TeamcityBoostLogFormatter::test_unit_start(ostream &out, test_unit const& tu) { + messages.setOutput(out); + + if (tu.p_type == tut_case) { + messages.testStarted(tu.p_name, flowId); + } else { + messages.suiteStarted(tu.p_name, flowId); + } + + currentDetails.clear(); +} + +void TeamcityBoostLogFormatter::test_unit_finish(ostream &out, test_unit const& tu, unsigned long elapsed) { + messages.setOutput(out); + + test_results const& tr = results_collector.results(tu.p_id); + if (tu.p_type == tut_case) { + if(!tr.passed()) { + if(tr.p_skipped) { + messages.testIgnored(tu.p_name, "ignored", flowId); + } else if (tr.p_aborted) { + messages.testFailed(tu.p_name, "aborted", currentDetails, flowId); + } else { + messages.testFailed(tu.p_name, "failed", currentDetails, flowId); + } + } + + messages.testFinished(tu.p_name, elapsed / 1000, flowId); + } else { + messages.suiteFinished(tu.p_name, flowId); + } +} + +void TeamcityBoostLogFormatter::test_unit_skipped(ostream &out, test_unit const& tu) +{} + +void TeamcityBoostLogFormatter::log_exception(ostream &out, log_checkpoint_data const&, const_string explanation) { + string what = toString(explanation); + + out << what << endl; + currentDetails += what + "\n"; +} + +void TeamcityBoostLogFormatter::log_entry_start(ostream&, log_entry_data const&, log_entry_types let) +{} + +void TeamcityBoostLogFormatter::log_entry_value(ostream &out, const_string value) { + out << value; + currentDetails += toString(value); +} + +void TeamcityBoostLogFormatter::log_entry_finish(ostream &out) { + out << endl; + currentDetails += "\n"; +} + +} diff --git a/modules/platforms/cpp/thin-client-test/src/teamcity/teamcity_messages.cpp b/modules/platforms/cpp/thin-client-test/src/teamcity/teamcity_messages.cpp new file mode 100644 index 0000000000000..2837542c42df3 --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/src/teamcity/teamcity_messages.cpp @@ -0,0 +1,150 @@ +/* Copyright 2011 JetBrains s.r.o. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * $Revision: 88625 $ +*/ + +#include +#include + +#include "teamcity/teamcity_messages.h" + +using namespace std; + +namespace JetBrains { + +std::string getFlowIdFromEnvironment() { + const char *flowId = getenv("TEAMCITY_PROCESS_FLOW_ID"); + return flowId == NULL ? "" : flowId; +} + +bool underTeamcity() { + return getenv("TEAMCITY_PROJECT_NAME") != NULL; +} + +TeamcityMessages::TeamcityMessages() +: m_out(&cout) +{} + +void TeamcityMessages::setOutput(ostream &out) { + m_out = &out; +} + +string TeamcityMessages::escape(string s) { + string result; + + for (size_t i = 0; i < s.length(); i++) { + char c = s[i]; + + switch (c) { + case '\n': result.append("|n"); break; + case '\r': result.append("|r"); break; + case '\'': result.append("|'"); break; + case '|': result.append("||"); break; + case ']': result.append("|]"); break; + default: result.append(&c, 1); + } + } + + return result; +} + +void TeamcityMessages::openMsg(const string &name) { + // endl for http://jetbrains.net/tracker/issue/TW-4412 + *m_out << endl << "##teamcity[" << name; +} + +void TeamcityMessages::closeMsg() { + *m_out << "]"; + // endl for http://jetbrains.net/tracker/issue/TW-4412 + *m_out << endl; + m_out->flush(); +} + +void TeamcityMessages::writeProperty(string name, string value) { + *m_out << " " << name << "='" << escape(value) << "'"; +} + +void TeamcityMessages::suiteStarted(string name, string flowid) { + openMsg("testSuiteStarted"); + writeProperty("name", name); + if(flowid.length() > 0) { + writeProperty("flowId", flowid); + } + + closeMsg(); +} + +void TeamcityMessages::suiteFinished(string name, string flowid) { + openMsg("testSuiteFinished"); + writeProperty("name", name); + if(flowid.length() > 0) { + writeProperty("flowId", flowid); + } + + closeMsg(); +} + +void TeamcityMessages::testStarted(string name, string flowid) { + openMsg("testStarted"); + writeProperty("name", name); + if(flowid.length() > 0) { + writeProperty("flowId", flowid); + } + + closeMsg(); +} + +void TeamcityMessages::testFinished(string name, int durationMs, string flowid) { + openMsg("testFinished"); + + writeProperty("name", name); + + if(flowid.length() > 0) { + writeProperty("flowId", flowid); + } + + if(durationMs >= 0) { + stringstream out; + out << durationMs; + writeProperty("duration", out.str()); + } + + closeMsg(); +} + +void TeamcityMessages::testFailed(string name, string message, string details, string flowid) { + openMsg("testFailed"); + writeProperty("name", name); + writeProperty("message", message); + writeProperty("details", details); + if(flowid.length() > 0) { + writeProperty("flowId", flowid); + } + + closeMsg(); +} + +void TeamcityMessages::testIgnored(std::string name, std::string message, string flowid) { + openMsg("testIgnored"); + writeProperty("name", name); + writeProperty("message", message); + if(flowid.length() > 0) { + writeProperty("flowId", flowid); + } + + closeMsg(); +} + +} diff --git a/modules/platforms/cpp/thin-client-test/src/test_utils.cpp b/modules/platforms/cpp/thin-client-test/src/test_utils.cpp new file mode 100644 index 0000000000000..a60e39221af36 --- /dev/null +++ b/modules/platforms/cpp/thin-client-test/src/test_utils.cpp @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 + +#include + +#include "test_utils.h" + +namespace ignite_test +{ + std::string GetTestConfigDir() + { + return ignite::common::GetEnv("IGNITE_NATIVE_TEST_CPP_THIN_CONFIG_PATH"); + } + + void InitConfig(ignite::IgniteConfiguration& cfg, const char* cfgFile) + { + using namespace ignite; + + assert(cfgFile != 0); + + cfg.jvmOpts.push_back("-Xdebug"); + cfg.jvmOpts.push_back("-Xnoagent"); + cfg.jvmOpts.push_back("-Djava.compiler=NONE"); + cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005"); + cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError"); + cfg.jvmOpts.push_back("-Duser.timezone=GMT"); + cfg.jvmOpts.push_back("-DIGNITE_QUIET=false"); + cfg.jvmOpts.push_back("-DIGNITE_CONSOLE_APPENDER=false"); + cfg.jvmOpts.push_back("-DIGNITE_UPDATE_NOTIFIER=false"); + cfg.jvmOpts.push_back("-Duser.language=en"); + // Un-comment to debug SSL + //cfg.jvmOpts.push_back("-Djavax.net.debug=ssl"); + + cfg.igniteHome = jni::ResolveIgniteHome(); + cfg.jvmClassPath = jni::CreateIgniteHomeClasspath(cfg.igniteHome, true); + +#ifdef IGNITE_TESTS_32 + cfg.jvmInitMem = 256; + cfg.jvmMaxMem = 768; +#else + cfg.jvmInitMem = 1024; + cfg.jvmMaxMem = 4096; +#endif + + cfg.springCfgPath = GetTestConfigDir().append("/").append(cfgFile); + } + + ignite::Ignite StartServerNode(const char* cfgFile, const char* name) + { + using namespace ignite; + + assert(name != 0); + + IgniteConfiguration cfg; + + InitConfig(cfg, cfgFile); + + return Ignition::Start(cfg, name); + } + + ignite::Ignite StartCrossPlatformServerNode(const char* cfgFile, const char* name) + { + std::string config(cfgFile); + +#ifdef IGNITE_TESTS_32 + // Cutting off the ".xml" part. + config.resize(config.size() - 4); + config += "-32.xml"; +#endif //IGNITE_TESTS_32 + + return StartServerNode(config.c_str(), name); + } + + std::string AppendPath(const std::string& base, const std::string& toAdd) + { + std::stringstream stream; + + stream << base << ignite::common::Fs << toAdd; + + return stream.str(); + } + + void ClearLfs() + { + std::string home = ignite::jni::ResolveIgniteHome(); + std::string workDir = AppendPath(home, "work"); + + ignite::common::DeletePath(workDir); + } +} diff --git a/modules/platforms/cpp/thin-client/Makefile.am b/modules/platforms/cpp/thin-client/Makefile.am new file mode 100644 index 0000000000000..f8d919012fade --- /dev/null +++ b/modules/platforms/cpp/thin-client/Makefile.am @@ -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. +## + +ACLOCAL_AMFLAGS =-I m4 + +lib_LTLIBRARIES = libignite-thin-client.la + +SUBDIRS = \ + include + +AM_CPPFLAGS = \ + -I$(srcdir)/include \ + -I$(srcdir)/src \ + -I@top_srcdir@/common/include \ + -I@top_srcdir@/common/os/linux/include \ + -I@top_srcdir@/binary/include \ + -DIGNITE_IMPL \ + -D__STDC_LIMIT_MACROS \ + -D__STDC_CONSTANT_MACROS + +AM_CXXFLAGS = \ + -Wall \ + -std=c++03 + +libignite_thin_client_la_LIBADD = \ + @top_srcdir@/binary/libignite-binary.la + +libignite_thin_client_la_LDFLAGS = \ + -no-undefined \ + -ldl \ + -version-info 0:0:0 \ + -release $(PACKAGE_VERSION) + +libignite_thin_client_la_DEPENDENCIES = \ + @top_srcdir@/binary/libignite-binary.la + +libignite_thin_client_la_SOURCES = \ + os/linux/src/net/net_utils.cpp \ + os/linux/src/net/tcp_socket_client.cpp \ + src/impl/utility.cpp \ + src/impl/protocol_version.cpp \ + src/impl/data_channel.cpp \ + src/impl/net/remote_type_updater.cpp \ + src/impl/message.cpp \ + src/impl/ignite_client_impl.cpp \ + src/impl/cache/cache_client_proxy.cpp \ + src/impl/cache/cache_client_impl.cpp \ + src/impl/cache/cache_affinity_info.cpp \ + src/impl/data_router.cpp \ + src/impl/ssl/ssl_gateway.cpp \ + src/impl/ssl/secure_socket_client.cpp \ + src/ignite_client.cpp + +clean-local: + $(RM) *.gcno *.gcda + +clean-docs: + $(RM) $(DX_CLEANFILES) diff --git a/modules/platforms/cpp/thin-client/README.md b/modules/platforms/cpp/thin-client/README.md new file mode 100644 index 0000000000000..523524bf129f7 --- /dev/null +++ b/modules/platforms/cpp/thin-client/README.md @@ -0,0 +1,4 @@ +Apache Ignite Thin C++ client +======================================= + +TBD \ No newline at end of file diff --git a/modules/platforms/cpp/thin-client/include/Makefile.am b/modules/platforms/cpp/thin-client/include/Makefile.am new file mode 100644 index 0000000000000..79816fe8f8d73 --- /dev/null +++ b/modules/platforms/cpp/thin-client/include/Makefile.am @@ -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. +## + +ACLOCAL_AMFLAGS =-I m4 + +noinst_HEADERS = \ + ignite/thin/ssl_mode.h \ + ignite/thin/ignite_client.h \ + ignite/thin/ignite_client_configuration.h \ + ignite/thin/cache/cache_client.h \ + ignite/impl/thin/writable_key.h \ + ignite/impl/thin/readable.h \ + ignite/impl/thin/writable.h \ + ignite/impl/thin/cache/cache_client_proxy.h + +uninstall-hook: + if [ -d ${includedir}/ignite ]; then find ${includedir}/ignite -type d -empty -delete; fi diff --git a/modules/platforms/cpp/thin-client/include/ignite/impl/thin/cache/cache_client_proxy.h b/modules/platforms/cpp/thin-client/include/ignite/impl/thin/cache/cache_client_proxy.h new file mode 100644 index 0000000000000..00c04f61e44ca --- /dev/null +++ b/modules/platforms/cpp/thin-client/include/ignite/impl/thin/cache/cache_client_proxy.h @@ -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. + */ + +#ifndef _IGNITE_IMPL_THIN_CACHE_CACHE_CLIENT_PROXY +#define _IGNITE_IMPL_THIN_CACHE_CACHE_CLIENT_PROXY + +#include + +namespace ignite +{ + namespace impl + { + namespace thin + { + /* Forward declaration. */ + class Writable; + + /* Forward declaration. */ + class WritableKey; + + /* Forward declaration. */ + class Readable; + + namespace cache + { + /** + * Ignite client class proxy. + */ + class IGNITE_IMPORT_EXPORT CacheClientProxy + { + public: + /** + * Constructor. + */ + CacheClientProxy(const common::concurrent::SharedPointer& impl) : + impl(impl) + { + // No-op. + } + + /** + * Destructor. + */ + ~CacheClientProxy() + { + // No-op. + } + + /** + * Put value to cache. + * + * @param key Key. + * @param value Value. + */ + void Put(const WritableKey& key, const Writable& value); + + /** + * Get value from cache. + * + * @param key Key. + * @param value Value. + */ + void Get(const WritableKey& key, Readable& value); + + /** + * Check if the cache contains a value for the specified key. + * + * @param key Key whose presence in this cache is to be tested. + * @return @c true if the cache contains specified key. + */ + bool ContainsKey(const WritableKey& key); + + /** + * Gets the number of all entries cached across all nodes. + * @note This operation is distributed and will query all participating nodes for their cache sizes. + * + * @param peekModes Peek modes mask. + * @return Cache size across all nodes. + */ + int64_t GetSize(int32_t peekModes); + + /** + * Peeks at in-memory cached value using default optional peek mode. This method will not load value + * from any persistent store or from a remote node. + * + * Use for testing purposes only. + * + * @param key Key whose presence in this cache is to be tested. + * @param value Value. + */ + void LocalPeek(const WritableKey& key, Readable& value); + + /** + * Update cache partitions info. + */ + void RefreshAffinityMapping(); + + /** + * Removes given key mapping from cache. If cache previously contained value for the given key, + * then this value is returned. In case of PARTITIONED or REPLICATED caches, the value will be + * loaded from the primary node, which in its turn may load the value from the disk-based swap + * storage, and consecutively, if it's not in swap, from the underlying persistent storage. + * If the returned value is not needed, method removex() should always be used instead of this + * one to avoid the overhead associated with returning of the previous value. + * If write-through is enabled, the value will be removed from store. + * This method is transactional and will enlist the entry into ongoing transaction if there is one. + * + * @param key Key whose mapping is to be removed from cache. + * @return False if there was no matching key. + */ + bool Remove(const WritableKey& key); + + /** + * Removes all mappings from cache. + * If write-through is enabled, the value will be removed from store. + * This method is transactional and will enlist the entry into ongoing transaction if there is one. + */ + void RemoveAll(); + + /** + * Clear entry from the cache and swap storage, without notifying listeners or CacheWriters. + * Entry is cleared only if it is not currently locked, and is not participating in a transaction. + * + * @param key Key to clear. + */ + void Clear(const WritableKey& key); + + /** + * Clear cache. + */ + void Clear(); + + /** + * Get from CacheClient. + * Use for testing purposes only. + */ + template + static CacheClientProxy& GetFromCacheClient(CacheClientT& instance) + { + return instance.proxy; + } + + private: + /** Implementation. */ + common::concurrent::SharedPointer impl; + }; + } + } + } +} +#endif // _IGNITE_IMPL_THIN_CACHE_CACHE_CLIENT_PROXY diff --git a/modules/platforms/cpp/thin-client/include/ignite/impl/thin/readable.h b/modules/platforms/cpp/thin-client/include/ignite/impl/thin/readable.h new file mode 100644 index 0000000000000..8c3e337d04f75 --- /dev/null +++ b/modules/platforms/cpp/thin-client/include/ignite/impl/thin/readable.h @@ -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. + */ + +#ifndef _IGNITE_IMPL_THIN_READABLE +#define _IGNITE_IMPL_THIN_READABLE + +#include + +namespace ignite +{ + namespace impl + { + namespace thin + { + /** + * Abstraction to any type that can be read from a binary stream. + */ + class Readable + { + public: + /** + * Destructor. + */ + virtual ~Readable() + { + // No-op. + } + + /** + * Read value using reader. + * + * @param reader Reader to use. + */ + virtual void Read(binary::BinaryReaderImpl& reader) = 0; + }; + + /** + * Implementation of the Readable class for a concrete type. + */ + template + class ReadableImpl : public Readable + { + public: + /** Value type. */ + typedef T ValueType; + + /** + * Constructor. + * + * @param value Value. + */ + ReadableImpl(ValueType& value) : + value(value) + { + // No-op. + } + + /** + * Destructor. + */ + virtual ~ReadableImpl() + { + // No-op. + } + + /** + * Read value using reader. + * + * @param reader Reader to use. + */ + virtual void Read(binary::BinaryReaderImpl& reader) + { + reader.ReadTopObject0(value); + } + + private: + /** Data router. */ + ValueType& value; + }; + } + } +} + +#endif // _IGNITE_IMPL_THIN_READABLE diff --git a/modules/platforms/cpp/thin-client/include/ignite/impl/thin/writable.h b/modules/platforms/cpp/thin-client/include/ignite/impl/thin/writable.h new file mode 100644 index 0000000000000..5d5eefe203515 --- /dev/null +++ b/modules/platforms/cpp/thin-client/include/ignite/impl/thin/writable.h @@ -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. + */ + +#ifndef _IGNITE_IMPL_THIN_WRITABLE +#define _IGNITE_IMPL_THIN_WRITABLE + +#include + +namespace ignite +{ + namespace impl + { + namespace thin + { + /** + * Abstraction to any type that can be written to a binary stream. + */ + class Writable + { + public: + /** + * Destructor. + */ + virtual ~Writable() + { + // No-op. + } + + /** + * Write value using writer. + * + * @param writer Writer to use. + */ + virtual void Write(binary::BinaryWriterImpl& writer) const = 0; + }; + + /** + * Implementation of the Writable class for a concrete type. + */ + template + class WritableImpl : public Writable + { + public: + /** Value type. */ + typedef T ValueType; + + /** + * Constructor. + * + * @param value Value. + */ + WritableImpl(const ValueType& value) : + value(value) + { + // No-op. + } + + /** + * Destructor. + */ + virtual ~WritableImpl() + { + // No-op. + } + + /** + * Write value using writer. + * + * @param writer Writer to use. + */ + virtual void Write(binary::BinaryWriterImpl& writer) const + { + writer.WriteObject(value); + } + + private: + /** Value. */ + const ValueType& value; + }; + } + } +} + +#endif // _IGNITE_IMPL_THIN_WRITABLE diff --git a/modules/platforms/cpp/thin-client/include/ignite/impl/thin/writable_key.h b/modules/platforms/cpp/thin-client/include/ignite/impl/thin/writable_key.h new file mode 100644 index 0000000000000..bf16fd19b03d3 --- /dev/null +++ b/modules/platforms/cpp/thin-client/include/ignite/impl/thin/writable_key.h @@ -0,0 +1,681 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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_IMPL_THIN_WRITABLE_KEY +#define _IGNITE_IMPL_THIN_WRITABLE_KEY + +#include + +#include + +#include + +namespace ignite +{ + namespace impl + { + namespace thin + { + /** + * Abstraction to any type that can be written to a binary stream. + */ + class WritableKey : public Writable + { + public: + /** + * Destructor. + */ + virtual ~WritableKey() + { + // No-op. + } + + /** + * Get hash code of the value. + * + * @return Hash code of the value. + */ + virtual int32_t GetHashCode() const = 0; + }; + + /** + * Implementation of the Writable class for a user type. + */ + template + class WritableKeyImpl : public WritableKey + { + public: + enum { BUFFER_SIZE = 1024 }; + + /** Value type. */ + typedef T ValueType; + + /** + * Constructor. + * + * @param value Value. + */ + WritableKeyImpl(const ValueType& value) : + value(value), + mem(BUFFER_SIZE) + { + // No-op. + } + + /** + * Destructor. + */ + virtual ~WritableKeyImpl() + { + // No-op. + } + + /** + * Write value using writer. + * + * @param writer Writer to use. + */ + virtual void Write(binary::BinaryWriterImpl& writer) const + { + if (mem.Length() == 0) + { + writer.WriteObject(value); + } + else + { + interop::InteropOutputStream *stream = writer.GetStream(); + + stream->WriteInt8Array(mem.Data(), mem.Length()); + } + } + + /** + * Get hash code of the value. + * + * @return Hash code of the value. + */ + virtual int32_t GetHashCode() const + { + if (mem.Length() == 0) + { + interop::InteropOutputStream stream(&mem); + binary::BinaryWriterImpl writer(&stream, 0); + + writer.WriteObject(value); + + stream.Synchronize(); + } + + binary::BinaryObjectImpl binaryKey(mem, 0, 0, 0); + + return binaryKey.GetHashCode(); + } + + private: + /** Value. */ + const ValueType& value; + + /** Memory. */ + mutable interop::InteropUnpooledMemory mem; + }; + + /** + * Implementation of the Writable class for an basic type. + */ + template + class WritableBasicKeyImpl : public WritableKey + { + public: + /** Value type. */ + typedef T ValueType; + + /** + * Constructor. + * + * @param value Value. + */ + WritableBasicKeyImpl(const ValueType& value) : + value(value) + { + // No-op. + } + + /** + * Destructor. + */ + virtual ~WritableBasicKeyImpl() + { + // No-op. + } + + /** + * Write value using writer. + * + * @param writer Writer to use. + */ + virtual void Write(binary::BinaryWriterImpl& writer) const + { + writer.WriteObject(value); + } + + protected: + /** Value. */ + const ValueType& value; + }; + + /** + * Specializatoin for int8_t type. + */ + template<> + class WritableKeyImpl : public WritableBasicKeyImpl + { + public: + /** + * Constructor. + * + * @param value Value. + */ + WritableKeyImpl(const int8_t& value) : + WritableBasicKeyImpl(value) + { + // No-op. + } + + /** + * Destructor. + */ + virtual ~WritableKeyImpl() + { + // No-op. + } + + /** + * Get hash code of the value. + * + * @return Hash code of the value. + */ + virtual int32_t GetHashCode() const + { + return static_cast(value); + } + }; + + /** + * Specializatoin for int16_t type. + */ + template<> + class WritableKeyImpl : public WritableBasicKeyImpl + { + public: + /** + * Constructor. + * + * @param value Value. + */ + WritableKeyImpl(const int16_t& value) : + WritableBasicKeyImpl(value) + { + // No-op. + } + + /** + * Destructor. + */ + virtual ~WritableKeyImpl() + { + // No-op. + } + + /** + * Get hash code of the value. + * + * @return Hash code of the value. + */ + virtual int32_t GetHashCode() const + { + return static_cast(value); + } + }; + + /** + * Specializatoin for int32_t type. + */ + template<> + class WritableKeyImpl : public WritableBasicKeyImpl + { + public: + /** + * Constructor. + * + * @param value Value. + */ + WritableKeyImpl(const int32_t& value) : + WritableBasicKeyImpl(value) + { + // No-op. + } + + /** + * Destructor. + */ + virtual ~WritableKeyImpl() + { + // No-op. + } + + /** + * Get hash code of the value. + * + * @return Hash code of the value. + */ + virtual int32_t GetHashCode() const + { + return value; + } + }; + + /** + * Specializatoin for uint16_t type. + */ + template<> + class WritableKeyImpl : public WritableBasicKeyImpl + { + public: + /** + * Constructor. + * + * @param value Value. + */ + WritableKeyImpl(const uint16_t& value) : + WritableBasicKeyImpl(value) + { + // No-op. + } + + /** + * Destructor. + */ + virtual ~WritableKeyImpl() + { + // No-op. + } + + /** + * Get hash code of the value. + * + * @return Hash code of the value. + */ + virtual int32_t GetHashCode() const + { + return static_cast(value); + } + }; + + /** + * Specializatoin for boolean type. + */ + template<> + class WritableKeyImpl : public WritableBasicKeyImpl + { + public: + /** + * Constructor. + * + * @param value Value. + */ + WritableKeyImpl(const bool& value) : + WritableBasicKeyImpl(value) + { + // No-op. + } + + /** + * Destructor. + */ + virtual ~WritableKeyImpl() + { + // No-op. + } + + /** + * Get hash code of the value. + * + * @return Hash code of the value. + */ + virtual int32_t GetHashCode() const + { + return value ? 1231 : 1237; + } + }; + + /** + * Specializatoin for int64_t type. + */ + template<> + class WritableKeyImpl : public WritableBasicKeyImpl + { + public: + /** + * Constructor. + * + * @param value Value. + */ + WritableKeyImpl(const int64_t& value) : + WritableBasicKeyImpl(value) + { + // No-op. + } + + /** + * Destructor. + */ + virtual ~WritableKeyImpl() + { + // No-op. + } + + /** + * Get hash code of the value. + * + * @return Hash code of the value. + */ + virtual int32_t GetHashCode() const + { + return static_cast(value ^ ((value >> 32) & 0xFFFFFFFF)); + } + }; + + /** + * Specializatoin for float type. + */ + template<> + class WritableKeyImpl : public WritableBasicKeyImpl + { + public: + /** + * Constructor. + * + * @param value Value. + */ + WritableKeyImpl(const float& value) : + WritableBasicKeyImpl(value) + { + // No-op. + } + + /** + * Destructor. + */ + virtual ~WritableKeyImpl() + { + // No-op. + } + + /** + * Get hash code of the value. + * + * @return Hash code of the value. + */ + virtual int32_t GetHashCode() const + { + const int32_t *res = reinterpret_cast(&value); + + return *res; + } + }; + + /** + * Specializatoin for double type. + */ + template<> + class WritableKeyImpl : public WritableBasicKeyImpl + { + public: + /** + * Constructor. + * + * @param value Value. + */ + WritableKeyImpl(const double& value) : + WritableBasicKeyImpl(value) + { + // No-op. + } + + /** + * Destructor. + */ + virtual ~WritableKeyImpl() + { + // No-op. + } + + /** + * Get hash code of the value. + * + * @return Hash code of the value. + */ + virtual int32_t GetHashCode() const + { + const int64_t *res = reinterpret_cast(&value); + + return static_cast(*res ^ ((*res >> 32) & 0xFFFFFFFF)); + } + }; + + /** + * Specializatoin for std::string type. + */ + template<> + class WritableKeyImpl : public WritableBasicKeyImpl + { + public: + /** + * Constructor. + * + * @param value Value. + */ + WritableKeyImpl(const std::string& value) : + WritableBasicKeyImpl(value) + { + // No-op. + } + + /** + * Destructor. + */ + virtual ~WritableKeyImpl() + { + // No-op. + } + + /** + * Get hash code of the value. + * + * @return Hash code of the value. + */ + virtual int32_t GetHashCode() const + { + int32_t hash = 0; + + for (size_t i = 0; i < value.size(); ++i) + hash = 31 * hash + value[i]; + + return hash; + } + }; + + /** + * Specializatoin for Guid type. + */ + template<> + class WritableKeyImpl : public WritableBasicKeyImpl + { + public: + /** + * Constructor. + * + * @param value Value. + */ + WritableKeyImpl(const Guid& value) : + WritableBasicKeyImpl(value) + { + // No-op. + } + + /** + * Destructor. + */ + virtual ~WritableKeyImpl() + { + // No-op. + } + + /** + * Get hash code of the value. + * + * @return Hash code of the value. + */ + virtual int32_t GetHashCode() const + { + int64_t hilo = value.GetMostSignificantBits() ^ value.GetLeastSignificantBits(); + + return static_cast(hilo ^ ((hilo >> 32) & 0xFFFFFFFF)); + } + }; + + /** + * Specializatoin for Date type. + */ + template<> + class WritableKeyImpl : public WritableBasicKeyImpl + { + public: + /** + * Constructor. + * + * @param value Value. + */ + WritableKeyImpl(const Date& value) : + WritableBasicKeyImpl(value) + { + // No-op. + } + + /** + * Destructor. + */ + virtual ~WritableKeyImpl() + { + // No-op. + } + + /** + * Get hash code of the value. + * + * @return Hash code of the value. + */ + virtual int32_t GetHashCode() const + { + int64_t ht = value.GetMilliseconds(); + + return static_cast(ht ^ ((ht >> 32) & 0xFFFFFFFF)); + } + }; + + /** + * Specializatoin for Timestamp type. + */ + template<> + class WritableKeyImpl : public WritableBasicKeyImpl + { + public: + /** + * Constructor. + * + * @param value Value. + */ + WritableKeyImpl(const Timestamp& value) : + WritableBasicKeyImpl(value) + { + // No-op. + } + + /** + * Destructor. + */ + virtual ~WritableKeyImpl() + { + // No-op. + } + + /** + * Get hash code of the value. + * + * @return Hash code of the value. + */ + virtual int32_t GetHashCode() const + { + int64_t ht = value.GetMilliseconds(); + + return static_cast(ht ^ ((ht >> 32) & 0xFFFFFFFF)); + } + }; + + /** + * Specializatoin for Time type. + */ + template<> + class WritableKeyImpl