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 C82A6200C02 for ; Thu, 5 Jan 2017 09:58:07 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id C6F7E160B26; Thu, 5 Jan 2017 08:58:07 +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 0F4C3160B59 for ; Thu, 5 Jan 2017 09:58:05 +0100 (CET) Received: (qmail 11258 invoked by uid 500); 5 Jan 2017 08:58:05 -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 10702 invoked by uid 99); 5 Jan 2017 08:58:05 -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; Thu, 05 Jan 2017 08:58:04 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id CAC9DDFEED; Thu, 5 Jan 2017 08:58:04 +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: Thu, 05 Jan 2017 08:58:20 -0000 Message-Id: <55ffd135e3cc444d94bc6dfe27b0aa09@git.apache.org> In-Reply-To: <76c54cad1ab1489682d63371d771d299@git.apache.org> References: <76c54cad1ab1489682d63371d771d299@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [17/26] ignite git commit: Implemented Visor tasks for Services. archived-at: Thu, 05 Jan 2017 08:58:07 -0000 Implemented Visor tasks for Services. (cherry picked from commit fdf1f4b) Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/b252b441 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/b252b441 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/b252b441 Branch: refs/heads/master Commit: b252b441a9ada31c7200b385d75e0b3e7c0362dd Parents: 2b3a180 Author: Alexey Kuznetsov Authored: Fri Dec 23 18:20:44 2016 +0700 Committer: Alexey Kuznetsov Committed: Fri Dec 23 18:35:58 2016 +0700 ---------------------------------------------------------------------- .../visor/service/VisorCancelServiceTask.java | 70 ++++++++++ .../visor/service/VisorServiceDescriptor.java | 132 +++++++++++++++++++ .../visor/service/VisorServiceTask.java | 75 +++++++++++ .../internal/visor/util/VisorTaskUtils.java | 15 ++- .../resources/META-INF/classnames.properties | 65 +++++++-- 5 files changed, 342 insertions(+), 15 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/b252b441/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorCancelServiceTask.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorCancelServiceTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorCancelServiceTask.java new file mode 100644 index 0000000..64987e9 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorCancelServiceTask.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.visor.service; + +import org.apache.ignite.IgniteServices; +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; + +/** + * Task for cancel services with specified name. + */ +@GridInternal +public class VisorCancelServiceTask extends VisorOneNodeTask { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override protected VisorCancelServiceJob job(String arg) { + return new VisorCancelServiceJob(arg, debug); + } + + /** + * Job for cancel services with specified name. + */ + private static class VisorCancelServiceJob extends VisorJob { + /** */ + private static final long serialVersionUID = 0L; + + /** + * Create job with specified argument. + * + * @param arg Job argument. + * @param debug Debug flag. + */ + protected VisorCancelServiceJob(String arg, boolean debug) { + super(arg, debug); + } + + /** {@inheritDoc} */ + @Override protected Void run(final String arg) { + IgniteServices services = ignite.services(); + + services.cancel(arg); + + return null; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorCancelServiceJob.class, this); + } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/b252b441/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.java new file mode 100644 index 0000000..83ec77d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.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.visor.service; + +import java.io.Serializable; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.visor.util.VisorTaskUtils; +import org.apache.ignite.services.ServiceDescriptor; + +/** + * Data transfer object for {@link ServiceDescriptor} object. + */ +public class VisorServiceDescriptor implements Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** Service name. */ + private String name; + + /** Service class. */ + private String srvcCls; + + /** Maximum allowed total number of deployed services in the grid, {@code 0} for unlimited. */ + private int totalCnt; + + /** Maximum allowed number of deployed services on each node. */ + private int maxPerNodeCnt; + + /** Cache name used for key-to-node affinity calculation. */ + private String cacheName; + + /** ID of grid node that initiated the service deployment. */ + private UUID originNodeId; + + /** + * Service deployment topology snapshot. + * Number of service instances deployed on a node mapped to node ID. + */ + private Map topSnapshot; + + /** + * Default constructor. + */ + public VisorServiceDescriptor() { + // No-op. + } + + /** + * Create task result with given parameters + * + */ + public VisorServiceDescriptor(ServiceDescriptor srvc) { + name = srvc.name(); + srvcCls = VisorTaskUtils.compactClass(srvc.serviceClass()); + totalCnt = srvc.totalCount(); + maxPerNodeCnt = srvc.maxPerNodeCount(); + cacheName = srvc.cacheName(); + originNodeId = srvc.originNodeId(); + topSnapshot = srvc.topologySnapshot(); + } + + /** + * @return Service name. + */ + public String getName() { + return name; + } + + /** + * @return Service class. + */ + public String getServiceClass() { + return srvcCls; + } + + /** + * @return Maximum allowed total number of deployed services in the grid, 0 for unlimited. + */ + public int getTotalCnt() { + return totalCnt; + } + + /** + * @return Maximum allowed number of deployed services on each node. + */ + public int getMaxPerNodeCnt() { + return maxPerNodeCnt; + } + + /** + * @return Cache name used for key-to-node affinity calculation. + */ + public String getCacheName() { + return cacheName; + } + + /** + * @return ID of grid node that initiated the service deployment. + */ + public UUID getOriginNodeId() { + return originNodeId; + } + + /** + * @return Service deployment topology snapshot. Number of service instances deployed on a node mapped to node ID. + */ + public Map getTopologySnapshot() { + return topSnapshot; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorServiceDescriptor.class, this); + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/b252b441/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java new file mode 100644 index 0000000..1b3495c --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.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.service; + +import java.util.ArrayList; +import java.util.Collection; +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.services.ServiceDescriptor; + +/** + * Task for collect topology service configuration. + */ +@GridInternal +public class VisorServiceTask extends VisorOneNodeTask> { + /** */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override protected VisorServiceJob job(Void arg) { + return new VisorServiceJob(arg, debug); + } + + /** + * Job for collect topology service configuration. + */ + private static class VisorServiceJob extends VisorJob> { + /** */ + private static final long serialVersionUID = 0L; + + /** + * Create job with specified argument. + * + * @param arg Job argument. + * @param debug Debug flag. + */ + protected VisorServiceJob(Void arg, boolean debug) { + super(arg, debug); + } + + /** {@inheritDoc} */ + @Override protected Collection run(final Void arg) { + Collection services = ignite.services().serviceDescriptors(); + + Collection res = new ArrayList<>(services.size()); + + for (ServiceDescriptor srvc: services) + res.add(new VisorServiceDescriptor(srvc)); + + return res; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(VisorServiceJob.class, this); + } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/b252b441/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java index 1e9346c..3f5003a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java @@ -270,6 +270,19 @@ public class VisorTaskUtils { /** * Compact class names. * + * @param cls Class object for compact. + * @return Compacted string. + */ + @Nullable public static String compactClass(Class cls) { + if (cls == null) + return null; + + return U.compact(cls.getName()); + } + + /** + * Compact class names. + * * @param obj Object for compact. * @return Compacted string. */ @@ -277,7 +290,7 @@ public class VisorTaskUtils { if (obj == null) return null; - return U.compact(obj.getClass().getName()); + return compactClass(obj.getClass()); } /** http://git-wip-us.apache.org/repos/asf/ignite/blob/b252b441/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 4c9596c..4d0b931 100644 --- a/modules/core/src/main/resources/META-INF/classnames.properties +++ b/modules/core/src/main/resources/META-INF/classnames.properties @@ -294,12 +294,17 @@ org.apache.ignite.internal.jdbc2.JdbcDatabaseMetadata$UpdateMetadataTask org.apache.ignite.internal.jdbc2.JdbcQueryTask org.apache.ignite.internal.jdbc2.JdbcQueryTask$1 org.apache.ignite.internal.jdbc2.JdbcQueryTask$QueryResult +org.apache.ignite.internal.jdbc2.JdbcQueryTaskV2 +org.apache.ignite.internal.jdbc2.JdbcQueryTaskV2$1 +org.apache.ignite.internal.jdbc2.JdbcQueryTaskV2$QueryResult +org.apache.ignite.internal.jdbc2.JdbcSqlFieldsQuery org.apache.ignite.internal.managers.GridManagerAdapter$1$1 org.apache.ignite.internal.managers.checkpoint.GridCheckpointManager$CheckpointSet org.apache.ignite.internal.managers.checkpoint.GridCheckpointRequest org.apache.ignite.internal.managers.communication.GridIoManager$ConcurrentHashMap0 org.apache.ignite.internal.managers.communication.GridIoMessage org.apache.ignite.internal.managers.communication.GridIoUserMessage +org.apache.ignite.internal.managers.communication.IgniteIoTestMessage org.apache.ignite.internal.managers.deployment.GridDeploymentInfoBean org.apache.ignite.internal.managers.deployment.GridDeploymentPerVersionStore$2 org.apache.ignite.internal.managers.deployment.GridDeploymentRequest @@ -387,20 +392,20 @@ org.apache.ignite.internal.processors.cache.GridCacheAdapter$3 org.apache.ignite.internal.processors.cache.GridCacheAdapter$30 org.apache.ignite.internal.processors.cache.GridCacheAdapter$32 org.apache.ignite.internal.processors.cache.GridCacheAdapter$4 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$48 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$49 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$50 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$51 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$52 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$53 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$54 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$55 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$57 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$58 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$58$1 +org.apache.ignite.internal.processors.cache.GridCacheAdapter$59 org.apache.ignite.internal.processors.cache.GridCacheAdapter$6 org.apache.ignite.internal.processors.cache.GridCacheAdapter$60 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$61 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$62 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$63 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$64 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$65 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$66 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$67 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$69 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$70 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$70$1 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$71 -org.apache.ignite.internal.processors.cache.GridCacheAdapter$72 org.apache.ignite.internal.processors.cache.GridCacheAdapter$9 org.apache.ignite.internal.processors.cache.GridCacheAdapter$AsyncOp$1 org.apache.ignite.internal.processors.cache.GridCacheAdapter$AsyncOp$1$1 @@ -719,8 +724,11 @@ org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtFor org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponse 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$DemandWorker$1 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$DemandWorker$2 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId @@ -1110,6 +1118,12 @@ org.apache.ignite.internal.processors.hadoop.HadoopJobStatus org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo org.apache.ignite.internal.processors.hadoop.HadoopTaskType +org.apache.ignite.internal.processors.hadoop.message.HadoopMessage +org.apache.ignite.internal.processors.hadoop.shuffle.HadoopDirectShuffleMessage +org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleAck +org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleFinishRequest +org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleFinishResponse +org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleMessage org.apache.ignite.internal.processors.igfs.IgfsAckMessage org.apache.ignite.internal.processors.igfs.IgfsAttributes org.apache.ignite.internal.processors.igfs.IgfsBlockKey @@ -1207,6 +1221,7 @@ org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryProcessor org.apache.ignite.internal.processors.platform.cache.PlatformCachePartialUpdateException org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinity$1 org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinityFunction +org.apache.ignite.internal.processors.platform.cache.expiry.PlatformExpiryPolicyFactory org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQuery org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryFilter org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryImpl @@ -1243,6 +1258,9 @@ org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$9 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetConfigurationClosure org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetService org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetServiceImpl +org.apache.ignite.internal.processors.platform.entityframework.PlatformDotNetEntityFrameworkCacheExtension$CleanupCompletionListener +org.apache.ignite.internal.processors.platform.entityframework.PlatformDotNetEntityFrameworkCacheExtension$RemoveOldEntriesRunnable +org.apache.ignite.internal.processors.platform.entityframework.PlatformDotNetEntityFrameworkIncreaseVersionProcessor org.apache.ignite.internal.processors.platform.events.PlatformEventFilterListenerImpl org.apache.ignite.internal.processors.platform.message.PlatformMessageFilter org.apache.ignite.internal.processors.platform.messaging.PlatformMessageFilterImpl @@ -1265,6 +1283,7 @@ 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$IndexType +org.apache.ignite.internal.processors.query.IgniteSQLException org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest @@ -1330,6 +1349,9 @@ org.apache.ignite.internal.processors.rest.handlers.datastructures.DataStructure org.apache.ignite.internal.processors.rest.handlers.query.CacheQueryFieldsMetaResult org.apache.ignite.internal.processors.rest.handlers.query.CacheQueryResult org.apache.ignite.internal.processors.rest.handlers.query.QueryCommandHandler$QueryCursorIterator +org.apache.ignite.internal.processors.rest.handlers.redis.GridRedisRestCommandHandler$1 +org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisGenericException +org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisTypeException org.apache.ignite.internal.processors.rest.handlers.task.GridTaskCommandHandler$2 org.apache.ignite.internal.processors.rest.handlers.task.GridTaskCommandHandler$ExeCallable org.apache.ignite.internal.processors.rest.handlers.task.GridTaskResultRequest @@ -1341,6 +1363,9 @@ org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpMemcachedNioList org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpMemcachedNioListener$2 org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpRestNioListener$1 org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpRestNioListener$1$1 +org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand +org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisMessage +org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisNioListener$1 org.apache.ignite.internal.processors.rest.request.RestQueryRequest$QueryType org.apache.ignite.internal.processors.service.GridServiceAssignments org.apache.ignite.internal.processors.service.GridServiceAssignmentsKey @@ -1585,10 +1610,13 @@ org.apache.ignite.internal.util.lang.IgniteReducer2X org.apache.ignite.internal.util.lang.IgniteReducer3 org.apache.ignite.internal.util.lang.IgniteReducer3X org.apache.ignite.internal.util.lang.IgniteReducerX +org.apache.ignite.internal.util.lang.IgniteSingletonIterator org.apache.ignite.internal.util.nio.GridNioEmbeddedFuture$1 org.apache.ignite.internal.util.nio.GridNioException org.apache.ignite.internal.util.nio.GridNioMessageTracker org.apache.ignite.internal.util.nio.GridNioServer$NioOperation +org.apache.ignite.internal.util.nio.GridNioServer$RandomBalancer +org.apache.ignite.internal.util.nio.GridNioServer$SizeBasedBalancer org.apache.ignite.internal.util.nio.GridNioSessionMetaKey org.apache.ignite.internal.util.nio.ssl.GridNioSslHandler org.apache.ignite.internal.util.offheap.GridOffHeapEvent @@ -1801,6 +1829,11 @@ org.apache.ignite.internal.visor.query.VisorQueryResult org.apache.ignite.internal.visor.query.VisorQueryResultEx org.apache.ignite.internal.visor.query.VisorQueryScanSubstringFilter org.apache.ignite.internal.visor.query.VisorQueryTask +org.apache.ignite.internal.visor.service.VisorCancelServiceTask +org.apache.ignite.internal.visor.service.VisorCancelServiceTask$VisorCancelServiceJob +org.apache.ignite.internal.visor.service.VisorServiceDescriptor +org.apache.ignite.internal.visor.service.VisorServiceTask +org.apache.ignite.internal.visor.service.VisorServiceTask$VisorServiceJob org.apache.ignite.internal.visor.util.VisorClusterGroupEmptyException org.apache.ignite.internal.visor.util.VisorEventMapper org.apache.ignite.internal.visor.util.VisorExceptionWrapper @@ -1858,12 +1891,15 @@ org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointData org.apache.ignite.spi.collision.jobstealing.JobStealingRequest org.apache.ignite.spi.collision.priorityqueue.PriorityQueueCollisionSpi$PriorityGridCollisionJobContextComparator org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$1 +org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$10 +org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$11 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$8 -org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$9 +org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosureNew +org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosureNew$1 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeClosure org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeMessage +org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeMessage2 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeTimeoutException org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$NodeIdMessage org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$RecoveryLastReceivedMessage @@ -1923,3 +1959,4 @@ org.apache.ignite.transactions.TransactionOptimisticException org.apache.ignite.transactions.TransactionRollbackException org.apache.ignite.transactions.TransactionState org.apache.ignite.transactions.TransactionTimeoutException +org.apache.ignite.util.AttributeNodeFilter