Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id D5117200CCA for ; Tue, 4 Jul 2017 12:02:15 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id D3957160C59; Tue, 4 Jul 2017 10:02:15 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id BD8A6160C56 for ; Tue, 4 Jul 2017 12:02:11 +0200 (CEST) Received: (qmail 52606 invoked by uid 500); 4 Jul 2017 10:02:10 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 51711 invoked by uid 99); 4 Jul 2017 10:02:10 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 04 Jul 2017 10:02:10 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 96351F551E; Tue, 4 Jul 2017 10:02:07 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: vozerov@apache.org To: commits@ignite.apache.org Date: Tue, 04 Jul 2017 10:02:27 -0000 Message-Id: <31ae94028b06404e95715b264755f3b7@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [21/50] ignite git commit: IGNITE-5461 Support for Memory metrics and fixed cache metrics. archived-at: Tue, 04 Jul 2017 10:02:16 -0000 IGNITE-5461 Support for Memory metrics and fixed cache metrics. Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/3cd30f06 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/3cd30f06 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/3cd30f06 Branch: refs/heads/master Commit: 3cd30f0640fb9b77d4db739a8291055c2207aa4f Parents: 1cb9772 Author: anovikov Authored: Wed Jun 28 11:47:25 2017 +0700 Committer: Alexey Kuznetsov Committed: Wed Jun 28 11:47:25 2017 +0700 ---------------------------------------------------------------------- .../IgniteCacheDatabaseSharedManager.java | 1 - .../ignite/internal/visor/cache/VisorCache.java | 20 ++- .../cache/VisorCacheAggregatedMetrics.java | 6 +- .../internal/visor/cache/VisorCacheMetrics.java | 28 +++ .../visor/cache/VisorMemoryMetrics.java | 10 +- .../node/VisorMemoryPolicyConfiguration.java | 10 +- .../resources/META-INF/classnames.properties | 46 ++--- .../cluster-select/cluster-select.controller.js | 51 +++--- .../cluster-select/cluster-select.pug | 16 +- .../app/modules/agent/AgentManager.service.js | 174 +++++++++++-------- .../frontend/app/modules/cluster/Cache.js | 19 +- .../modules/configuration/Version.service.js | 9 +- .../configuration/generator/Maven.service.js | 159 +++++++++-------- .../views/templates/agent-download.tpl.pug | 9 +- 14 files changed, 308 insertions(+), 250 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/3cd30f06/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java ---------------------------------------------------------------------- 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 e1f1c9a..ec0e895 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 @@ -41,7 +41,6 @@ import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider; import org.apache.ignite.internal.pagemem.PageMemory; import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl; import org.apache.ignite.internal.processors.cache.CacheGroupContext; -import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; import org.apache.ignite.internal.processors.cache.persistence.evict.FairFifoPageEvictionTracker; http://git-wip-us.apache.org/repos/asf/ignite/blob/3cd30f06/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java index aa4f271..5d7dfd3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java @@ -33,6 +33,10 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.visor.VisorDataTransferObject; import org.apache.ignite.lang.IgniteUuid; +import static org.apache.ignite.cache.CachePeekMode.ONHEAP; +import static org.apache.ignite.cache.CachePeekMode.PRIMARY; +import static org.apache.ignite.cache.CachePeekMode.BACKUP; + /** * Data transfer object for {@link IgniteCache}. */ @@ -41,8 +45,12 @@ public class VisorCache extends VisorDataTransferObject { private static final long serialVersionUID = 0L; /** */ - private static final CachePeekMode[] PEEK_NO_NEAR = - new CachePeekMode[] {CachePeekMode.PRIMARY, CachePeekMode.BACKUP}; + private static final CachePeekMode[] PEEK_ONHEAP_PRIMARY = + new CachePeekMode[] {ONHEAP, PRIMARY}; + + /** */ + private static final CachePeekMode[] PEEK_ONHEAP_BACKUP = + new CachePeekMode[] {ONHEAP, BACKUP}; /** Cache name. */ private String name; @@ -102,10 +110,12 @@ public class VisorCache extends VisorDataTransferObject { name = ca.name(); dynamicDeploymentId = cctx.dynamicDeploymentId(); mode = cfg.getCacheMode(); - size = ca.localSizeLong(PEEK_NO_NEAR); - primarySize = ca.primarySizeLong(); - backupSize = size - primarySize; + + primarySize = ca.localSizeLong(PEEK_ONHEAP_PRIMARY); + backupSize = ca.localSizeLong(PEEK_ONHEAP_BACKUP); nearSize = ca.nearSize(); + size = primarySize + backupSize + nearSize; + partitions = ca.affinity().partitions(); near = cctx.isNear(); http://git-wip-us.apache.org/repos/asf/ignite/blob/3cd30f06/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java index 6812274..ccefa67 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java @@ -170,7 +170,7 @@ public class VisorCacheAggregatedMetrics extends VisorDataTransferObject { minHeapSize = Long.MAX_VALUE; for (VisorCacheMetrics metric : metrics.values()) - minHeapSize = Math.min(minHeapSize, metric.getKeySize()); + minHeapSize = Math.min(minHeapSize, metric.getHeapEntriesCount()); } return minHeapSize; @@ -184,7 +184,7 @@ public class VisorCacheAggregatedMetrics extends VisorDataTransferObject { avgHeapSize = 0.0d; for (VisorCacheMetrics metric : metrics.values()) - avgHeapSize += metric.getKeySize(); + avgHeapSize += metric.getHeapEntriesCount(); avgHeapSize /= metrics.size(); } @@ -200,7 +200,7 @@ public class VisorCacheAggregatedMetrics extends VisorDataTransferObject { maxHeapSize = Long.MIN_VALUE; for (VisorCacheMetrics metric : metrics.values()) - maxHeapSize = Math.max(maxHeapSize, metric.getKeySize()); + maxHeapSize = Math.max(maxHeapSize, metric.getHeapEntriesCount()); } return maxHeapSize; http://git-wip-us.apache.org/repos/asf/ignite/blob/3cd30f06/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java ---------------------------------------------------------------------- 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 e67f5f8..d99e0c9 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 @@ -166,6 +166,9 @@ public class VisorCacheMetrics extends VisorDataTransferObject { /** Number of cache entries stored in off-heap memory. */ private long offHeapEntriesCnt; + /** Number of primary cache entries stored in off-heap memory. */ + private long offHeapPrimaryEntriesCnt; + /** Total number of partitions on current node. */ private int totalPartsCnt; @@ -265,6 +268,7 @@ public class VisorCacheMetrics extends VisorDataTransferObject { heapEntriesCnt = m.getHeapEntriesCount(); offHeapAllocatedSize = m.getOffHeapAllocatedSize(); offHeapEntriesCnt = m.getOffHeapEntriesCount(); + offHeapPrimaryEntriesCnt = m.getOffHeapPrimaryEntriesCount(); totalPartsCnt = m.getTotalPartitionsCount(); rebalancingPartsCnt = m.getRebalancingPartitionsCount(); @@ -586,6 +590,20 @@ public class VisorCacheMetrics extends VisorDataTransferObject { } /** + * @return Number of primary cache entries stored in off-heap memory. + */ + public long getOffHeapPrimaryEntriesCount() { + return offHeapPrimaryEntriesCnt; + } + + /** + * @return Number of backup cache entries stored in off-heap memory. + */ + public long getOffHeapBackupEntriesCount() { + return offHeapEntriesCnt - offHeapPrimaryEntriesCnt; + } + + /** * @return Total number of partitions on current node. */ public int getTotalPartitionsCount() { @@ -621,6 +639,11 @@ public class VisorCacheMetrics extends VisorDataTransferObject { } /** {@inheritDoc} */ + @Override public byte getProtocolVersion() { + return V2; + } + + /** {@inheritDoc} */ @Override protected void writeExternalData(ObjectOutput out) throws IOException { U.writeString(out, name); U.writeEnum(out, mode); @@ -675,6 +698,8 @@ public class VisorCacheMetrics extends VisorDataTransferObject { out.writeLong(rebalancingBytesRate); out.writeObject(qryMetrics); + + out.writeLong(offHeapPrimaryEntriesCnt); } /** {@inheritDoc} */ @@ -731,6 +756,9 @@ public class VisorCacheMetrics extends VisorDataTransferObject { rebalancingBytesRate = in.readLong(); qryMetrics = (VisorQueryMetrics)in.readObject(); + + if (protoVer >= V2) + offHeapPrimaryEntriesCnt = in.readLong(); } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/ignite/blob/3cd30f06/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorMemoryMetrics.java ---------------------------------------------------------------------- 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 8193db6..c6cdd5c 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 @@ -54,7 +54,7 @@ public class VisorMemoryMetrics extends VisorDataTransferObject { private long dirtyPages; /** */ - private float pageReplaceRate; + private float pagesReplaceRate; /** */ private long physicalMemoryPages; @@ -77,7 +77,7 @@ public class VisorMemoryMetrics extends VisorDataTransferObject { largeEntriesPagesPercentage = m.getLargeEntriesPagesPercentage(); pagesFillFactor = m.getPagesFillFactor(); dirtyPages = m.getDirtyPages(); - pageReplaceRate = m.getPagesReplaceRate(); + pagesReplaceRate = m.getPagesReplaceRate(); physicalMemoryPages = m.getPhysicalMemoryPages(); } @@ -134,7 +134,7 @@ public class VisorMemoryMetrics extends VisorDataTransferObject { * @return Pages per second replace rate. */ public float getPagesReplaceRate() { - return pageReplaceRate; + return pagesReplaceRate; } /** @@ -153,7 +153,7 @@ public class VisorMemoryMetrics extends VisorDataTransferObject { out.writeFloat(largeEntriesPagesPercentage); out.writeFloat(pagesFillFactor); out.writeLong(dirtyPages); - out.writeFloat(pageReplaceRate); + out.writeFloat(pagesReplaceRate); out.writeLong(physicalMemoryPages); } @@ -166,7 +166,7 @@ public class VisorMemoryMetrics extends VisorDataTransferObject { largeEntriesPagesPercentage = in.readFloat(); pagesFillFactor = in.readFloat(); dirtyPages = in.readLong(); - pageReplaceRate = in.readFloat(); + pagesReplaceRate = in.readFloat(); physicalMemoryPages = in.readLong(); } http://git-wip-us.apache.org/repos/asf/ignite/blob/3cd30f06/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java index 4f8d15e..ced3124 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java @@ -81,11 +81,6 @@ public class VisorMemoryPolicyConfiguration extends VisorDataTransferObject { emptyPagesPoolSize = plc.getEmptyPagesPoolSize(); } - /** {@inheritDoc} */ - @Override public byte getProtocolVersion() { - return V2; - } - /** * Unique name of MemoryPolicy. */ @@ -136,6 +131,11 @@ public class VisorMemoryPolicyConfiguration extends VisorDataTransferObject { } /** {@inheritDoc} */ + @Override public byte getProtocolVersion() { + return V2; + } + + /** {@inheritDoc} */ @Override protected void writeExternalData(ObjectOutput out) throws IOException { U.writeString(out, name); out.writeLong(maxSize); http://git-wip-us.apache.org/repos/asf/ignite/blob/3cd30f06/modules/core/src/main/resources/META-INF/classnames.properties ---------------------------------------------------------------------- diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties index 93382d7..1528040 100644 --- a/modules/core/src/main/resources/META-INF/classnames.properties +++ b/modules/core/src/main/resources/META-INF/classnames.properties @@ -408,20 +408,24 @@ org.apache.ignite.internal.processors.cache.CacheWeakQueryIteratorsHolder$WeakQu org.apache.ignite.internal.processors.cache.CacheWeakQueryIteratorsHolder$WeakQueryFutureIterator org.apache.ignite.internal.processors.cache.CacheWeakQueryIteratorsHolder$WeakReferenceCloseableIterator org.apache.ignite.internal.processors.cache.ChangeGlobalStateMessage +org.apache.ignite.internal.processors.cache.ClientCacheChangeDiscoveryMessage +org.apache.ignite.internal.processors.cache.ClientCacheChangeDummyDiscoveryMessage org.apache.ignite.internal.processors.cache.ClusterCachesInfo$1$1 org.apache.ignite.internal.processors.cache.ClusterState org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest org.apache.ignite.internal.processors.cache.EntryProcessorResourceInjectorProxy +org.apache.ignite.internal.processors.cache.ExchangeActions$1 +org.apache.ignite.internal.processors.cache.ExchangeActions$2 org.apache.ignite.internal.processors.cache.GridCacheAdapter org.apache.ignite.internal.processors.cache.GridCacheAdapter$10 org.apache.ignite.internal.processors.cache.GridCacheAdapter$11 org.apache.ignite.internal.processors.cache.GridCacheAdapter$12 org.apache.ignite.internal.processors.cache.GridCacheAdapter$13 org.apache.ignite.internal.processors.cache.GridCacheAdapter$14 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$15 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$15$1 org.apache.ignite.internal.processors.cache.GridCacheAdapter$16 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$17$1 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$17 org.apache.ignite.internal.processors.cache.GridCacheAdapter$2 org.apache.ignite.internal.processors.cache.GridCacheAdapter$25$1 org.apache.ignite.internal.processors.cache.GridCacheAdapter$27 @@ -489,7 +493,6 @@ org.apache.ignite.internal.processors.cache.GridCacheIdMessage org.apache.ignite.internal.processors.cache.GridCacheIndexUpdateException org.apache.ignite.internal.processors.cache.GridCacheIoManager$1$1 org.apache.ignite.internal.processors.cache.GridCacheIoManager$1$2 -org.apache.ignite.internal.processors.cache.GridCacheIoManager$2 org.apache.ignite.internal.processors.cache.GridCacheIterator org.apache.ignite.internal.processors.cache.GridCacheLoaderWriterStore org.apache.ignite.internal.processors.cache.GridCacheLoaderWriterStoreFactory @@ -513,10 +516,10 @@ org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$7 org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeFutureSet org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$MessageHandler org.apache.ignite.internal.processors.cache.GridCacheProcessor$1 -org.apache.ignite.internal.processors.cache.GridCacheProcessor$10 org.apache.ignite.internal.processors.cache.GridCacheProcessor$3 org.apache.ignite.internal.processors.cache.GridCacheProcessor$4 org.apache.ignite.internal.processors.cache.GridCacheProcessor$5 +org.apache.ignite.internal.processors.cache.GridCacheProcessor$7 org.apache.ignite.internal.processors.cache.GridCacheProcessor$8 org.apache.ignite.internal.processors.cache.GridCacheProcessor$9 org.apache.ignite.internal.processors.cache.GridCacheProcessor$LocalAffinityFunction @@ -556,7 +559,8 @@ org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl$3 org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl$4 org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl$5 org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl$6 -org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl$8 +org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl$7 +org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl$9 org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl$CacheDataStoreImpl$1 org.apache.ignite.internal.processors.cache.IgniteCacheProxy org.apache.ignite.internal.processors.cache.IgniteCacheProxy$1 @@ -758,9 +762,8 @@ org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPar 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 +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$6$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 @@ -883,13 +886,11 @@ org.apache.ignite.internal.processors.cache.persistence.wal.SegmentEofException org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDataIntegrityViolationException org.apache.ignite.internal.processors.cache.query.CacheQueryType org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryFuture$1 -org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryFuture$3 org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager$1 org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager$2 org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager$4 org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager$5 org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager$6 -org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager$7 org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$1 org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$ScanQueryFallbackClosableIterator org.apache.ignite.internal.processors.cache.query.GridCacheQueryDetailMetricsAdapter @@ -1071,39 +1072,22 @@ org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$IsolatedUpda org.apache.ignite.internal.processors.datastreamer.DataStreamerRequest org.apache.ignite.internal.processors.datastreamer.DataStreamerResponse org.apache.ignite.internal.processors.datastreamer.DataStreamerUpdateJob$1 +org.apache.ignite.internal.processors.datastructures.AtomicDataStructureValue org.apache.ignite.internal.processors.datastructures.DataStructureInfoKey +org.apache.ignite.internal.processors.datastructures.DataStructureType org.apache.ignite.internal.processors.datastructures.DataStructuresCacheKey -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$10 org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$11 -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$12 org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$13 -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$14 org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$15 org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$16 org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$17 -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$18 -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$19 -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$2 -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$20 -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$21 -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$22 -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$23 -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$24 -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$3 org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$4 org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$5 -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$6 -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$7 org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$8 org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$9 -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$AddDataCacheProcessor -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$CacheCollectionInfo -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$CollectionInfo -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$DataStructureInfo -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$DataStructureType org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$DataStructuresEntryFilter org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$DataStructuresEntryListener$2 -org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$QueueInfo +org.apache.ignite.internal.processors.datastructures.DistributedCollectionMetadata org.apache.ignite.internal.processors.datastructures.GridCacheAtomicLongImpl org.apache.ignite.internal.processors.datastructures.GridCacheAtomicLongImpl$AddAndGetProcessor org.apache.ignite.internal.processors.datastructures.GridCacheAtomicLongImpl$CompareAndSetProcessor @@ -1148,6 +1132,7 @@ org.apache.ignite.internal.processors.datastructures.GridCacheSetImpl$SumReducer org.apache.ignite.internal.processors.datastructures.GridCacheSetItemKey org.apache.ignite.internal.processors.datastructures.GridCacheSetProxy org.apache.ignite.internal.processors.datastructures.GridSetQueryPredicate +org.apache.ignite.internal.processors.datastructures.VolatileAtomicDataStructureValue org.apache.ignite.internal.processors.dr.GridDrType org.apache.ignite.internal.processors.dr.IgniteDrDataStreamerCacheUpdater org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo @@ -1322,7 +1307,6 @@ org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils$FutureL org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils$InternalFutureListenable$1 org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionLockProcessor org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionSetAndUnlockProcessor -org.apache.ignite.internal.processors.query.DynamicTableAffinityKeyMapper org.apache.ignite.internal.processors.query.GridQueryFieldMetadata org.apache.ignite.internal.processors.query.GridQueryProcessor$10 org.apache.ignite.internal.processors.query.GridQueryProcessor$4 @@ -1778,6 +1762,7 @@ org.apache.ignite.internal.visor.cache.VisorCacheStopTask org.apache.ignite.internal.visor.cache.VisorCacheStopTask$VisorCacheStopJob org.apache.ignite.internal.visor.cache.VisorCacheStopTaskArg org.apache.ignite.internal.visor.cache.VisorCacheStoreConfiguration +org.apache.ignite.internal.visor.cache.VisorMemoryMetrics org.apache.ignite.internal.visor.cache.VisorPartitionMap org.apache.ignite.internal.visor.compute.VisorComputeCancelSessionsTask org.apache.ignite.internal.visor.compute.VisorComputeCancelSessionsTask$VisorComputeCancelSessionsJob @@ -2001,6 +1986,7 @@ org.apache.ignite.spi.collision.priorityqueue.PriorityQueueCollisionSpi$Priority org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$1 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$12 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$13 +org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$1 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosure org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosure$1 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosureNew http://git-wip-us.apache.org/repos/asf/ignite/blob/3cd30f06/modules/web-console/frontend/app/components/cluster-select/cluster-select.controller.js ---------------------------------------------------------------------- diff --git a/modules/web-console/frontend/app/components/cluster-select/cluster-select.controller.js b/modules/web-console/frontend/app/components/cluster-select/cluster-select.controller.js index a318172..6d30e94 100644 --- a/modules/web-console/frontend/app/components/cluster-select/cluster-select.controller.js +++ b/modules/web-console/frontend/app/components/cluster-select/cluster-select.controller.js @@ -23,33 +23,44 @@ export default ['$scope', 'AgentManager', function($scope, agentMgr) { ctrl.cluster = null; ctrl.clusters = []; - $scope.$watchCollection(() => agentMgr.clusters, (clusters) => { - if (_.isEmpty(clusters)) - return ctrl.clusters.length = 0; + agentMgr.connectionSbj.subscribe({ + next: ({cluster, clusters}) => { + if (_.isEmpty(clusters)) + return ctrl.clusters.length = 0; - const removed = _.differenceBy(ctrl.clusters, clusters, 'id'); + const removed = _.differenceBy(ctrl.clusters, clusters, 'id'); - if (_.nonEmpty(removed)) - _.pullAll(ctrl.clusters, removed); + if (_.nonEmpty(removed)) + _.pullAll(ctrl.clusters, removed); - const added = _.differenceBy(clusters, ctrl.clusters, 'id'); + const added = _.differenceBy(clusters, ctrl.clusters, 'id'); - _.forEach(added, (cluster) => { - ctrl.clusters.push({ - id: cluster.id, - name: `Cluster ${cluster.id.substring(0, 8).toUpperCase()}`, - click: () => { - if (cluster.id === ctrl.cluster.id) - return; + _.forEach(added, (cluster) => { + ctrl.clusters.push({ + id: cluster.id, + name: `Cluster ${cluster.id.substring(0, 8).toUpperCase()}`, + connected: true, + click: () => { + if (cluster.id === _.get(ctrl, 'cluster.id')) + return; - agentMgr.saveToStorage(cluster); + if (_.get(ctrl, 'cluster.connected')) { + agentMgr.saveToStorage(cluster); - window.open(window.location.href, '_blank'); - } + window.open(window.location.href, '_blank'); + } + else + ctrl.cluster = _.find(ctrl.clusters, {id: cluster.id}); + } + }); }); - }); - if (_.isNil(ctrl.cluster)) - ctrl.cluster = _.find(ctrl.clusters, {id: agentMgr.cluster.id}); + const item = _.find(ctrl.clusters, {id: cluster.id}); + + if (_.isNil(ctrl.cluster)) + ctrl.cluster = item; + else + ctrl.cluster.connected = !!item; + } }); }]; http://git-wip-us.apache.org/repos/asf/ignite/blob/3cd30f06/modules/web-console/frontend/app/components/cluster-select/cluster-select.pug ---------------------------------------------------------------------- diff --git a/modules/web-console/frontend/app/components/cluster-select/cluster-select.pug b/modules/web-console/frontend/app/components/cluster-select/cluster-select.pug index a5fd08c..bb81415 100644 --- a/modules/web-console/frontend/app/components/cluster-select/cluster-select.pug +++ b/modules/web-console/frontend/app/components/cluster-select/cluster-select.pug @@ -14,22 +14,26 @@ See the License for the specific language governing permissions and limitations under the License. -ul.nav(ng-switch='ctrl.clusters.length') - li.disabled(ng-switch-when='0') - a +ul.nav + li.disabled(ng-if='ctrl.clusters.length === 0') + a(ng-if='!ctrl.cluster') i.icon-cluster label.padding-left-dflt(bs-tooltip='' data-placement='bottom' data-title='Check that Web Agent(s) started and connected to cluster(s)') No clusters available + a(ng-if='ctrl.cluster') + i.icon-danger + label.padding-left-dflt(bs-tooltip='' data-placement='bottom' data-title='Connection to cluster was lost') {{ctrl.cluster.name}} - li.disabled(ng-switch-when='1') + li(ng-if='ctrl.clusters.length === 1 && ctrl.cluster.connected') a i.icon-cluster label.padding-left-dflt {{ctrl.cluster.name}} - li(ng-switch-default) + li(ng-if='ctrl.clusters.length > 1 || ctrl.clusters.length === 1 && !ctrl.cluster.connected') a.dropdown-toggle(bs-dropdown='' data-placement='bottom-left' data-trigger='hover focus' data-container='self' ng-click='$event.stopPropagation()' aria-haspopup='true' aria-expanded='expanded') - i.icon-cluster + i(ng-class='{"icon-cluster": ctrl.cluster.connected, "icon-danger": !ctrl.cluster.connected}') label.padding-left-dflt {{ctrl.cluster.name}} span.caret + ul.dropdown-menu(role='menu') li(ng-repeat='item in ctrl.clusters' ng-class='{active: ctrl.cluster === item}') div(ng-click='item.click()') http://git-wip-us.apache.org/repos/asf/ignite/blob/3cd30f06/modules/web-console/frontend/app/modules/agent/AgentManager.service.js ---------------------------------------------------------------------- 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 eb4ebfc..3873567 100644 --- a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js +++ b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js @@ -20,12 +20,56 @@ import { BehaviorSubject } from 'rxjs/BehaviorSubject'; import maskNull from 'app/core/utils/maskNull'; const State = { - INIT: 'INIT', + DISCONNECTED: 'DISCONNECTED', AGENT_DISCONNECTED: 'AGENT_DISCONNECTED', CLUSTER_DISCONNECTED: 'CLUSTER_DISCONNECTED', CONNECTED: 'CONNECTED' }; +class ConnectionState { + constructor(cluster) { + this.agents = []; + this.cluster = cluster; + this.clusters = []; + this.state = State.DISCONNECTED; + } + + update(demo, count, clusters) { + this.clusters = clusters; + + if (_.isNil(this.cluster)) + this.cluster = _.head(clusters); + + if (_.nonNil(this.cluster)) + this.cluster.connected = !!_.find(clusters, {id: this.cluster.id}); + + if (count === 0) + this.state = State.AGENT_DISCONNECTED; + else if (demo || _.get(this.cluster, 'connected')) + this.state = State.CONNECTED; + else + this.state = State.CLUSTER_DISCONNECTED; + } + + useConnectedCluster() { + if (_.nonEmpty(this.clusters) && !this.cluster.connected) { + this.cluster = _.head(this.clusters); + + this.state = State.CONNECTED; + } + } + + disconnect() { + this.agents = []; + + if (this.cluster) + this.cluster.disconnect = true; + + this.clusters = []; + this.state = State.DISCONNECTED; + } +} + export default class IgniteAgentManager { static $inject = ['$rootScope', '$q', 'igniteSocketFactory', 'AgentModal', 'UserNotifications']; @@ -48,34 +92,41 @@ export default class IgniteAgentManager { $root.$on('$stateChangeSuccess', () => this.stopWatch()); - this.ignite2x = true; - this.ignite2_1 = true; - - if (!$root.IgniteDemoMode) { - $root.$watch(() => _.get(this, 'cluster.clusterVersion'), (ver) => { - if (_.isEmpty(ver)) - return; - - this.ignite2x = ver.startsWith('2.'); - this.ignite2_1 = ver.startsWith('2.1'); - }, true); - } - /** * Connection to backend. * @type {Socket} */ this.socket = null; - this.connectionState = new BehaviorSubject(State.INIT); + let cluster; - /** - * Has agent with enabled demo mode. - * @type {boolean} - */ - this.hasDemo = false; + try { + cluster = JSON.parse(localStorage.cluster); - this.clusters = []; + localStorage.removeItem('cluster'); + } + catch (ignore) { + // No-op. + } + + this.connectionSbj = new BehaviorSubject(new ConnectionState(cluster)); + + this.ignite2x = true; + this.ignite2_1 = true; + + if (!$root.IgniteDemoMode) { + this.connectionSbj.subscribe({ + next: ({cluster}) => { + const version = _.get(cluster, 'clusterVersion'); + + if (_.isEmpty(version)) + return; + + this.ignite2x = version.startsWith('2.'); + this.ignite2_1 = version.startsWith('2.1'); + } + }); + } } connect() { @@ -87,59 +138,28 @@ export default class IgniteAgentManager { self.socket = self.socketFactory(); const onDisconnect = () => { - self.connected = false; + const conn = self.connectionSbj.getValue(); + + conn.disconnect(); + + self.connectionSbj.next(conn); }; self.socket.on('connect_error', onDisconnect); self.socket.on('disconnect', onDisconnect); - self.connected = null; + self.socket.on('agents:stat', ({clusters, count}) => { + const conn = self.connectionSbj.getValue(); - try { - self.cluster = JSON.parse(localStorage.cluster); + conn.update(self.$root.IgniteDemoMode, count, clusters); - localStorage.removeItem('cluster'); - } - catch (ignore) { - // No-op. - } - - self.socket.on('agents:stat', ({count, hasDemo, clusters}) => { - self.hasDemo = hasDemo; - - const removed = _.differenceBy(self.clusters, clusters, 'id'); - - if (_.nonEmpty(removed)) { - _.pullAll(self.clusters, removed); - - if (self.cluster && _.find(removed, {id: self.cluster.id})) - self.cluster.disconnect = true; - } - - const added = _.differenceBy(clusters, self.clusters, 'id'); - - if (_.nonEmpty(added)) { - self.clusters.push(...added); - - if (_.isNil(self.cluster)) - self.cluster = _.head(added); - - if (self.cluster && _.find(added, {id: self.cluster.id})) - self.cluster.disconnect = false; - } - - if (count === 0) - self.connectionState.next(State.AGENT_DISCONNECTED); - else if (self.$root.IgniteDemoMode || _.get(self.cluster, 'disconnect') === false) - self.connectionState.next(State.CONNECTED); - else - self.connectionState.next(State.CLUSTER_DISCONNECTED); + self.connectionSbj.next(conn); }); self.socket.on('user:notifications', (notification) => this.UserNotifications.notification = notification); } - saveToStorage(cluster = this.cluster) { + saveToStorage(cluster = this.connectionSbj.getValue().cluster) { try { localStorage.cluster = JSON.stringify(cluster); } catch (ignore) { @@ -156,8 +176,8 @@ export default class IgniteAgentManager { this.promises.add(defer); - const subscription = this.connectionState.subscribe({ - next: (state) => { + const subscription = this.connectionSbj.subscribe({ + next: ({state}) => { if (_.includes(states, state)) defer.resolve(); } @@ -190,14 +210,14 @@ export default class IgniteAgentManager { self.backText = backText; self.backState = backState; - if (_.nonEmpty(self.clusters) && _.get(self.cluster, 'disconnect') === true) { - self.cluster = _.head(self.clusters); + const conn = self.connectionSbj.getValue(); - self.connectionState.next(State.CONNECTED); - } + conn.useConnectedCluster(); - self.modalSubscription = this.connectionState.subscribe({ - next: (state) => { + self.connectionSbj.next(conn); + + self.modalSubscription = this.connectionSbj.subscribe({ + next: ({state}) => { switch (state) { case State.CONNECTED: case State.CLUSTER_DISCONNECTED: @@ -211,7 +231,7 @@ export default class IgniteAgentManager { break; default: - // Connection to backend is not established yet. + // Connection to backend is not established yet. } } }); @@ -230,14 +250,14 @@ export default class IgniteAgentManager { self.backText = backText; self.backState = backState; - if (_.nonEmpty(self.clusters) && _.get(self.cluster, 'disconnect') === true) { - self.cluster = _.head(self.clusters); + const conn = self.connectionSbj.getValue(); - self.connectionState.next(State.CONNECTED); - } + conn.useConnectedCluster(); + + self.connectionSbj.next(conn); - self.modalSubscription = this.connectionState.subscribe({ - next: (state) => { + self.modalSubscription = this.connectionSbj.subscribe({ + next: ({state}) => { switch (state) { case State.CONNECTED: this.AgentModal.hide(); http://git-wip-us.apache.org/repos/asf/ignite/blob/3cd30f06/modules/web-console/frontend/app/modules/cluster/Cache.js ---------------------------------------------------------------------- diff --git a/modules/web-console/frontend/app/modules/cluster/Cache.js b/modules/web-console/frontend/app/modules/cluster/Cache.js index 62c0512..c5b1d00 100644 --- a/modules/web-console/frontend/app/modules/cluster/Cache.js +++ b/modules/web-console/frontend/app/modules/cluster/Cache.js @@ -25,27 +25,20 @@ export default class Cache { // Mode. this.mode = cache.mode; - // Memory Usage. - this.memorySize = cache.memorySize; - // Heap. this.size = cache.size; this.primarySize = cache.primarySize; this.backupSize = _.isNil(cache.backupSize) ? cache.dhtSize - cache.primarySize : cache.backupSize; this.nearSize = cache.nearSize; - // Off-heap. - this.offHeapAllocatedSize = cache.offHeapAllocatedSize; - this.offHeapSize = cache.offHeapEntriesCount; - this.offHeapPrimarySize = cache.offHeapPrimaryEntriesCount || 0; - this.offHeapBackupSize = cache.offHeapBackupEntriesCount || 0; - - // Swap. - this.swapSize = cache.swapSize; - this.swapKeys = cache.swapKeys; - const m = cache.metrics; + // Off-heap. + this.offHeapAllocatedSize = m.offHeapAllocatedSize; + this.offHeapSize = m.offHeapEntriesCount; + this.offHeapPrimarySize = m.offHeapPrimaryEntriesCount || 0; + this.offHeapBackupSize = this.offHeapSize - this.offHeapPrimarySize; + // Read/write metrics. this.hits = m.hits; this.misses = m.misses; http://git-wip-us.apache.org/repos/asf/ignite/blob/3cd30f06/modules/web-console/frontend/app/modules/configuration/Version.service.js ---------------------------------------------------------------------- diff --git a/modules/web-console/frontend/app/modules/configuration/Version.service.js b/modules/web-console/frontend/app/modules/configuration/Version.service.js index 5b29291..ad8916c 100644 --- a/modules/web-console/frontend/app/modules/configuration/Version.service.js +++ b/modules/web-console/frontend/app/modules/configuration/Version.service.js @@ -118,14 +118,15 @@ export default class IgniteVersion { } /** - * @return {String} Target Ignite version. + * @return {String} Current Ignite version. */ get current() { return this.currentSbj.getValue().ignite; } /** - * Check if version in range + * Check if version in range. + * * @param {String} target Target version. * @param {String | Array.} ranges Version ranges to compare with. * @returns {Boolean} `True` if version is equal or greater than specified range. @@ -147,6 +148,7 @@ export default class IgniteVersion { /** * Check whether version before than specified version. + * * @param {String} target Target version. * @param {String} ranges Version ranges to compare with. * @return {Boolean} `True` if version before than specified version. @@ -156,7 +158,8 @@ export default class IgniteVersion { } /** - * Check if configuration version in range + * Check if current version in specified range. + * * @param {String|Array.} ranges Version ranges to compare with. * @returns {Boolean} `True` if configuration version is equal or greater than specified range. */ http://git-wip-us.apache.org/repos/asf/ignite/blob/3cd30f06/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js ---------------------------------------------------------------------- diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js index f6f6a80..da098fc 100644 --- a/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js +++ b/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js @@ -32,18 +32,26 @@ export default class IgniteMavenGenerator { } addProperty(sb, tag, val) { - sb.append('<' + tag + '>' + val + ''); + sb.append(`<${tag}>${val}`); } addDependency(deps, groupId, artifactId, version, jar) { - if (!_.find(deps, (dep) => dep.groupId === groupId && dep.artifactId === artifactId)) - deps.push({groupId, artifactId, version, jar}); + deps.add({groupId, artifactId, version, jar}); + } + + pickDependency(deps, key, dfltVer) { + if (!_.has(POM_DEPENDENCIES, key)) + return; + + const {groupId, artifactId, version, jar} = POM_DEPENDENCIES[key]; + + this.addDependency(deps, groupId || 'org.apache.ignite', artifactId, version || dfltVer, jar); } addResource(sb, dir, exclude) { sb.startBlock(''); - if (dir) - this.addProperty(sb, 'directory', dir); + + this.addProperty(sb, 'directory', dir); if (exclude) { sb.startBlock(''); @@ -54,18 +62,16 @@ export default class IgniteMavenGenerator { sb.endBlock(''); } - artifact(sb, cluster, version) { + artifactSection(sb, cluster, targetVer) { this.addProperty(sb, 'groupId', 'org.apache.ignite'); this.addProperty(sb, 'artifactId', this.escapeId(cluster.name) + '-project'); - this.addProperty(sb, 'version', version); - - sb.emptyLine(); + this.addProperty(sb, 'version', targetVer.ignite); } - dependencies(sb, cluster, deps) { + dependenciesSection(sb, deps) { sb.startBlock(''); - _.forEach(deps, (dep) => { + deps.forEach((dep) => { sb.startBlock(''); this.addProperty(sb, 'groupId', dep.groupId); @@ -85,14 +91,16 @@ export default class IgniteMavenGenerator { return sb; } - build(sb = new StringBuilder(), cluster, excludeGroupIds) { + buildSection(sb = new StringBuilder(), excludeGroupIds) { sb.startBlock(''); + sb.startBlock(''); this.addResource(sb, 'src/main/java', '**/*.java'); this.addResource(sb, 'src/main/resources'); sb.endBlock(''); sb.startBlock(''); + sb.startBlock(''); this.addProperty(sb, 'artifactId', 'maven-dependency-plugin'); sb.startBlock(''); @@ -111,6 +119,7 @@ export default class IgniteMavenGenerator { sb.endBlock(''); sb.endBlock(''); sb.endBlock(''); + sb.startBlock(''); this.addProperty(sb, 'artifactId', 'maven-compiler-plugin'); this.addProperty(sb, 'version', '3.1'); @@ -119,67 +128,27 @@ export default class IgniteMavenGenerator { this.addProperty(sb, 'target', '1.7'); sb.endBlock(''); sb.endBlock(''); + sb.endBlock(''); - sb.endBlock(''); - sb.endBlock(''); + sb.endBlock(''); } /** * Add dependency for specified store factory if not exist. - * @param storeDeps Already added dependencies. + * @param deps Already added dependencies. * @param storeFactory Store factory to add dependency. */ - storeFactoryDependency(storeDeps, storeFactory) { - if (storeFactory.dialect && (!storeFactory.connectVia || storeFactory.connectVia === 'DataSource')) { - const dep = POM_DEPENDENCIES[storeFactory.dialect]; - - this.addDependency(storeDeps, dep.groupId, dep.artifactId, dep.version, dep.jar); - } + storeFactoryDependency(deps, storeFactory) { + if (storeFactory.dialect && (!storeFactory.connectVia || storeFactory.connectVia === 'DataSource')) + this.pickDependency(deps, storeFactory.dialect); } - /** - * Generate pom.xml. - * - * @param {Object} cluster Cluster to take info about dependencies. - * @param {Object} targetVer Target version for dependencies. - * @returns {String} Generated content. - */ - generate(cluster, targetVer) { - const caches = cluster.caches; - const deps = []; - const storeDeps = []; - const excludeGroupIds = ['org.apache.ignite']; - - const blobStoreFactory = {cacheStoreFactory: {kind: 'CacheHibernateBlobStoreFactory'}}; - + collectDependencies(cluster, targetVer) { const igniteVer = targetVer.ignite; - _.forEach(caches, (cache) => { - if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) - this.storeFactoryDependency(storeDeps, cache.cacheStoreFactory[cache.cacheStoreFactory.kind]); - - if (_.get(cache, 'nodeFilter.kind') === 'Exclude') - this.addDependency(deps, 'org.apache.ignite', 'ignite-extdata-p2p', igniteVer); - }); - - const sb = new StringBuilder(); - - sb.append(''); - - sb.emptyLine(); - - sb.append(``); - - sb.emptyLine(); - - sb.startBlock(''); - - sb.append('4.0.0'); - - sb.emptyLine(); - - this.artifact(sb, cluster, igniteVer); + const deps = new Set(); + const storeDeps = new Set(); this.addDependency(deps, 'org.apache.ignite', 'ignite-core', igniteVer); @@ -190,10 +159,19 @@ export default class IgniteMavenGenerator { if (_.get(cluster, 'deploymentSpi.kind') === 'URI') this.addDependency(deps, 'org.apache.ignite', 'ignite-urideploy', igniteVer); - let dep = POM_DEPENDENCIES[cluster.discovery.kind]; + this.pickDependency(deps, cluster.discovery.kind, igniteVer); + + const caches = cluster.caches; + + const blobStoreFactory = {cacheStoreFactory: {kind: 'CacheHibernateBlobStoreFactory'}}; + + _.forEach(caches, (cache) => { + if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) + this.storeFactoryDependency(storeDeps, cache.cacheStoreFactory[cache.cacheStoreFactory.kind]); - if (dep) - this.addDependency(deps, 'org.apache.ignite', dep.artifactId, igniteVer); + if (_.get(cache, 'nodeFilter.kind') === 'Exclude') + this.addDependency(deps, 'org.apache.ignite', 'ignite-extdata-p2p', igniteVer); + }); if (cluster.discovery.kind === 'Jdbc') { const store = cluster.discovery.Jdbc; @@ -203,12 +181,8 @@ export default class IgniteMavenGenerator { } _.forEach(cluster.checkpointSpi, (spi) => { - if (spi.kind === 'S3') { - dep = POM_DEPENDENCIES.S3; - - if (dep) - this.addDependency(deps, 'org.apache.ignite', dep.artifactId, igniteVer); - } + if (spi.kind === 'S3') + this.pickDependency(deps, spi.kind, igniteVer); else if (spi.kind === 'JDBC') this.storeFactoryDependency(storeDeps, spi.JDBC); }); @@ -220,18 +194,49 @@ export default class IgniteMavenGenerator { if (_.find(caches, blobStoreFactory)) this.addDependency(deps, 'org.apache.ignite', 'ignite-hibernate', igniteVer); - if (cluster.logger && cluster.logger.kind) { - dep = POM_DEPENDENCIES[cluster.logger.kind]; + if (cluster.logger && cluster.logger.kind) + this.pickDependency(deps, cluster.logger.kind, igniteVer); - if (dep) - this.addDependency(deps, 'org.apache.ignite', dep.artifactId, igniteVer); - } + return new Set([...deps, ...storeDeps]); + } + + /** + * Generate pom.xml. + * + * @param {Object} cluster Cluster to take info about dependencies. + * @param {Object} targetVer Target version for dependencies. + * @returns {String} Generated content. + */ + generate(cluster, targetVer) { + const sb = new StringBuilder(); + + sb.append(''); + + sb.emptyLine(); + + sb.append(``); + + sb.emptyLine(); - this.dependencies(sb, cluster, deps.concat(storeDeps)); + sb.startBlock(''); + + sb.append('4.0.0'); + + sb.emptyLine(); + + this.artifactSection(sb, cluster, targetVer); sb.emptyLine(); - this.build(sb, cluster, excludeGroupIds); + const deps = this.collectDependencies(cluster, targetVer); + + this.dependenciesSection(sb, deps); + + sb.emptyLine(); + + this.buildSection(sb, ['org.apache.ignite']); + + sb.endBlock(''); return sb.asString(); } http://git-wip-us.apache.org/repos/asf/ignite/blob/3cd30f06/modules/web-console/frontend/views/templates/agent-download.tpl.pug ---------------------------------------------------------------------- diff --git a/modules/web-console/frontend/views/templates/agent-download.tpl.pug b/modules/web-console/frontend/views/templates/agent-download.tpl.pug index baf2c6c..0aec7e5 100644 --- a/modules/web-console/frontend/views/templates/agent-download.tpl.pug +++ b/modules/web-console/frontend/views/templates/agent-download.tpl.pug @@ -24,11 +24,10 @@ button.close(type='button' aria-label='Close' ng-click='$hide()') svg(ignite-icon="cross") .modal-body.agent-download - p Please download and run #[a(href='/api/v1/downloads/agent' target='_self') ignite-web-agent] to use this functionality - p To start: - ul - li Download and unzip #[a(href='/api/v1/downloads/agent' target='_self') ignite-web-agent] archive - li Run shell file #[b ignite-web-agent.{sh|bat}] + p Please download and run #[a(href='/api/v1/downloads/agent' target='_self') ignite-web-agent] to use this functionality: + ul + li Download and unzip #[a(href='/api/v1/downloads/agent' target='_self') ignite-web-agent] archive + li Run shell file #[b ignite-web-agent.{sh|bat}] p Refer to #[b README.txt] in the ignite-web-agent folder for more information. .modal-advanced-options i.fa(ng-class='showToken ? "fa-chevron-circle-down" : "fa-chevron-circle-right"' ng-click='showToken = !showToken')