You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/08/25 09:05:47 UTC

[06/14] ignite git commit: ignite-1258: portable objects API support in Ignite

http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/portable/PortableSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableSerializer.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableSerializer.java
new file mode 100644
index 0000000..54b0937
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/portable/PortableSerializer.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.portable;
+
+import org.apache.ignite.marshaller.portable.*;
+
+/**
+ * Interface that allows to implement custom serialization logic for portable objects.
+ * Can be used instead of {@link PortableMarshalAware} in case if the class
+ * cannot be changed directly.
+ * <p>
+ * Portable serializer can be configured for all portable objects via
+ * {@link PortableMarshaller#getSerializer()} method, or for a specific
+ * portable type via {@link PortableTypeConfiguration#getSerializer()} method.
+ */
+public interface PortableSerializer {
+    /**
+     * Writes fields to provided writer.
+     *
+     * @param obj Empty object.
+     * @param writer Portable object writer.
+     * @throws PortableException In case of error.
+     */
+    public void writePortable(Object obj, PortableWriter writer) throws PortableException;
+
+    /**
+     * Reads fields from provided reader.
+     *
+     * @param obj Empty object
+     * @param reader Portable object reader.
+     * @throws PortableException In case of error.
+     */
+    public void readPortable(Object obj, PortableReader reader) throws PortableException;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/portable/PortableTypeConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableTypeConfiguration.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableTypeConfiguration.java
new file mode 100644
index 0000000..b221298
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/portable/PortableTypeConfiguration.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.portable;
+
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.marshaller.portable.*;
+
+import java.sql.*;
+import java.util.*;
+
+/**
+ * Defines configuration properties for a specific portable type. Providing per-type
+ * configuration is optional, as it is generally enough, and also optional, to provide global portable
+ * configuration using {@link PortableMarshaller#setClassNames(Collection)}.
+ * However, this class allows you to change configuration properties for a specific
+ * portable type without affecting configuration for other portable types.
+ * <p>
+ * Per-type portable configuration can be specified in {@link PortableMarshaller#getTypeConfigurations()} method.
+ */
+public class PortableTypeConfiguration {
+    /** Class name. */
+    private String clsName;
+
+    /** ID mapper. */
+    private PortableIdMapper idMapper;
+
+    /** Serializer. */
+    private PortableSerializer serializer;
+
+    /** Use timestamp flag. */
+    private Boolean useTs;
+
+    /** Meta data enabled flag. */
+    private Boolean metaDataEnabled;
+
+    /** Keep deserialized flag. */
+    private Boolean keepDeserialized;
+
+    /** Affinity key field name. */
+    private String affKeyFieldName;
+
+    /**
+     */
+    public PortableTypeConfiguration() {
+        // No-op.
+    }
+
+    /**
+     * @param clsName Class name.
+     */
+    public PortableTypeConfiguration(String clsName) {
+        this.clsName = clsName;
+    }
+
+    /**
+     * Gets type name.
+     *
+     * @return Type name.
+     */
+    public String getClassName() {
+        return clsName;
+    }
+
+    /**
+     * Sets type name.
+     *
+     * @param clsName Type name.
+     */
+    public void setClassName(String clsName) {
+        this.clsName = clsName;
+    }
+
+    /**
+     * Gets ID mapper.
+     *
+     * @return ID mapper.
+     */
+    public PortableIdMapper getIdMapper() {
+        return idMapper;
+    }
+
+    /**
+     * Sets ID mapper.
+     *
+     * @param idMapper ID mapper.
+     */
+    public void setIdMapper(PortableIdMapper idMapper) {
+        this.idMapper = idMapper;
+    }
+
+    /**
+     * Gets serializer.
+     *
+     * @return Serializer.
+     */
+    public PortableSerializer getSerializer() {
+        return serializer;
+    }
+
+    /**
+     * Sets serializer.
+     *
+     * @param serializer Serializer.
+     */
+    public void setSerializer(PortableSerializer serializer) {
+        this.serializer = serializer;
+    }
+
+    /**
+     * If {@code true} then date values converted to {@link Timestamp} during unmarshalling.
+     *
+     * @return Flag indicating whether date values converted to {@link Timestamp} during unmarshalling.
+     */
+    public Boolean isUseTimestamp() {
+        return useTs;
+    }
+
+    /**
+     * @param useTs Flag indicating whether date values converted to {@link Timestamp} during unmarshalling.
+     */
+    public void setUseTimestamp(Boolean useTs) {
+        this.useTs = useTs;
+    }
+
+    /**
+     * Defines whether meta data is collected for this type. If provided, this value will override
+     * {@link PortableMarshaller#isMetaDataEnabled()} property.
+     *
+     * @return Whether meta data is collected.
+     */
+    public Boolean isMetaDataEnabled() {
+        return metaDataEnabled;
+    }
+
+    /**
+     * @param metaDataEnabled Whether meta data is collected.
+     */
+    public void setMetaDataEnabled(Boolean metaDataEnabled) {
+        this.metaDataEnabled = metaDataEnabled;
+    }
+
+    /**
+     * Defines whether {@link PortableObject} should cache deserialized instance. If provided,
+     * this value will override {@link PortableMarshaller#isKeepDeserialized()}
+     * property.
+     *
+     * @return Whether deserialized value is kept.
+     */
+    public Boolean isKeepDeserialized() {
+        return keepDeserialized;
+    }
+
+    /**
+     * @param keepDeserialized Whether deserialized value is kept.
+     */
+    public void setKeepDeserialized(Boolean keepDeserialized) {
+        this.keepDeserialized = keepDeserialized;
+    }
+
+    /**
+     * Gets affinity key field name.
+     *
+     * @return Affinity key field name.
+     */
+    public String getAffinityKeyFieldName() {
+        return affKeyFieldName;
+    }
+
+    /**
+     * Sets affinity key field name.
+     *
+     * @param affKeyFieldName Affinity key field name.
+     */
+    public void setAffinityKeyFieldName(String affKeyFieldName) {
+        this.affKeyFieldName = affKeyFieldName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PortableTypeConfiguration.class, this, super.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/portable/PortableWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableWriter.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableWriter.java
new file mode 100644
index 0000000..36fa608
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/portable/PortableWriter.java
@@ -0,0 +1,265 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.portable;
+
+import org.jetbrains.annotations.*;
+
+import java.math.*;
+import java.sql.*;
+import java.util.*;
+import java.util.Date;
+
+/**
+ * Writer for portable object used in {@link PortableMarshalAware} implementations.
+ * Useful for the cases when user wants a fine-grained control over serialization.
+ * <p>
+ * Note that Ignite never writes full strings for field or type names. Instead,
+ * for performance reasons, Ignite writes integer hash codes for type and field names.
+ * It has been tested that hash code conflicts for the type names or the field names
+ * within the same type are virtually non-existent and, to gain performance, it is safe
+ * to work with hash codes. For the cases when hash codes for different types or fields
+ * actually do collide, Ignite provides {@link PortableIdMapper} which
+ * allows to override the automatically generated hash code IDs for the type and field names.
+ */
+public interface PortableWriter {
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeByte(String fieldName, byte val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeShort(String fieldName, short val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeInt(String fieldName, int val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeLong(String fieldName, long val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeFloat(String fieldName, float val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeDouble(String fieldName, double val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeChar(String fieldName, char val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeBoolean(String fieldName, boolean val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeDecimal(String fieldName, @Nullable BigDecimal val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeString(String fieldName, @Nullable String val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val UUID to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeUuid(String fieldName, @Nullable UUID val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Date to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeDate(String fieldName, @Nullable Date val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Timestamp to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeTimestamp(String fieldName, @Nullable Timestamp val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param obj Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeObject(String fieldName, @Nullable Object obj) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeByteArray(String fieldName, @Nullable byte[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeShortArray(String fieldName, @Nullable short[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeIntArray(String fieldName, @Nullable int[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeLongArray(String fieldName, @Nullable long[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeFloatArray(String fieldName, @Nullable float[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeDoubleArray(String fieldName, @Nullable double[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeCharArray(String fieldName, @Nullable char[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeDecimalArray(String fieldName, @Nullable BigDecimal[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeStringArray(String fieldName, @Nullable String[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeUuidArray(String fieldName, @Nullable UUID[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeDateArray(String fieldName, @Nullable Date[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public void writeObjectArray(String fieldName, @Nullable Object[] val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param col Collection to write.
+     * @throws PortableException In case of error.
+     */
+    public <T> void writeCollection(String fieldName, @Nullable Collection<T> col) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param map Map to write.
+     * @throws PortableException In case of error.
+     */
+    public <K, V> void writeMap(String fieldName, @Nullable Map<K, V> map) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public <T extends Enum<?>> void writeEnum(String fieldName, T val) throws PortableException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws PortableException In case of error.
+     */
+    public <T extends Enum<?>> void writeEnumArray(String fieldName, T[] val) throws PortableException;
+
+    /**
+     * Gets raw writer. Raw writer does not write field name hash codes, therefore,
+     * making the format even more compact. However, if the raw writer is used,
+     * dynamic structure changes to the portable objects are not supported.
+     *
+     * @return Raw writer.
+     */
+    public PortableRawWriter rawWriter();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/main/java/org/apache/ignite/portable/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/package-info.java b/modules/core/src/main/java/org/apache/ignite/portable/package-info.java
new file mode 100644
index 0000000..0105b15
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/portable/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * <!-- Package description. -->
+ * Contains portable objects API classes.
+ */
+package org.apache.ignite.portable;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/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 1fb9a37..c5f060b 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -17,6 +17,7 @@
 
 org.apache.ignite.IgniteAuthenticationException
 org.apache.ignite.IgniteCheckedException
+org.apache.ignite.IgniteClientDisconnectedException
 org.apache.ignite.IgniteDeploymentException
 org.apache.ignite.IgniteException
 org.apache.ignite.IgniteIllegalStateException
@@ -35,7 +36,6 @@ org.apache.ignite.cache.CacheMode
 org.apache.ignite.cache.CachePartialUpdateException
 org.apache.ignite.cache.CachePeekMode
 org.apache.ignite.cache.CacheRebalanceMode
-org.apache.ignite.cache.CacheEntry
 org.apache.ignite.cache.CacheServerNotFoundException
 org.apache.ignite.cache.CacheTypeFieldMetadata
 org.apache.ignite.cache.CacheTypeMetadata
@@ -70,6 +70,8 @@ org.apache.ignite.cache.query.SqlQuery
 org.apache.ignite.cache.query.TextQuery
 org.apache.ignite.cache.store.jdbc.CacheAbstractJdbcStore$EntryMapping$1
 org.apache.ignite.cache.store.jdbc.CacheAbstractJdbcStore$EntryMapping$2
+org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactory
+org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory
 org.apache.ignite.cache.store.jdbc.dialect.BasicJdbcDialect$1
 org.apache.ignite.cache.store.jdbc.dialect.BasicJdbcDialect$2
 org.apache.ignite.cache.store.jdbc.dialect.BasicJdbcDialect$3
@@ -111,7 +113,6 @@ org.apache.ignite.compute.gridify.aop.GridifyDefaultRangeTask
 org.apache.ignite.compute.gridify.aop.GridifyDefaultTask
 org.apache.ignite.configuration.CacheConfiguration
 org.apache.ignite.configuration.CacheConfiguration$IgniteAllNodesPredicate
-org.apache.ignite.configuration.CacheConfiguration$IgniteServerNodePredicate
 org.apache.ignite.configuration.CollectionConfiguration
 org.apache.ignite.configuration.DeploymentMode
 org.apache.ignite.configuration.IgniteReflectionFactory
@@ -186,6 +187,7 @@ org.apache.ignite.internal.GridTopic$T5
 org.apache.ignite.internal.GridTopic$T6
 org.apache.ignite.internal.GridTopic$T7
 org.apache.ignite.internal.GridTopic$T8
+org.apache.ignite.internal.IgniteClientDisconnectedCheckedException
 org.apache.ignite.internal.IgniteComponentType
 org.apache.ignite.internal.IgniteComputeImpl
 org.apache.ignite.internal.IgniteDeploymentCheckedException
@@ -196,6 +198,7 @@ org.apache.ignite.internal.IgniteFutureTimeoutCheckedException
 org.apache.ignite.internal.IgniteInterruptedCheckedException
 org.apache.ignite.internal.IgniteKernal
 org.apache.ignite.internal.IgniteKernal$1
+org.apache.ignite.internal.IgniteKernal$5
 org.apache.ignite.internal.IgniteMessagingImpl
 org.apache.ignite.internal.IgniteSchedulerImpl
 org.apache.ignite.internal.IgniteServicesImpl
@@ -226,6 +229,7 @@ org.apache.ignite.internal.cluster.ClusterGroupAdapter$AttributeFilter
 org.apache.ignite.internal.cluster.ClusterGroupAdapter$CachesFilter
 org.apache.ignite.internal.cluster.ClusterGroupAdapter$DaemonFilter
 org.apache.ignite.internal.cluster.ClusterGroupAdapter$GroupPredicate
+org.apache.ignite.internal.cluster.ClusterGroupAdapter$HostsFilter
 org.apache.ignite.internal.cluster.ClusterGroupAdapter$OthersFilter
 org.apache.ignite.internal.cluster.ClusterGroupEmptyCheckedException
 org.apache.ignite.internal.cluster.ClusterNodeLocalMapImpl
@@ -243,33 +247,48 @@ org.apache.ignite.internal.executor.GridExecutorService
 org.apache.ignite.internal.executor.GridExecutorService$1
 org.apache.ignite.internal.executor.GridExecutorService$TaskTerminateListener
 org.apache.ignite.internal.igfs.common.IgfsIpcCommand
+org.apache.ignite.internal.interop.InteropAwareEventFilter
+org.apache.ignite.internal.interop.InteropBootstrapFactory
+org.apache.ignite.internal.interop.InteropException
+org.apache.ignite.internal.interop.InteropNoCallbackException
 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.GridIoPolicy
 org.apache.ignite.internal.managers.communication.GridIoUserMessage
 org.apache.ignite.internal.managers.communication.GridLifecycleAwareMessageFilter
 org.apache.ignite.internal.managers.deployment.GridDeploymentInfoBean
 org.apache.ignite.internal.managers.deployment.GridDeploymentPerVersionStore$2
 org.apache.ignite.internal.managers.deployment.GridDeploymentRequest
 org.apache.ignite.internal.managers.deployment.GridDeploymentResponse
+org.apache.ignite.internal.managers.discovery.CustomMessageWrapper
+org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage
 org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$1
-org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$4$1
-org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$6
+org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$2
+org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$5$1
+org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$5$2
+org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$7
 org.apache.ignite.internal.managers.discovery.GridDiscoveryManager$DiscoCache$1
 org.apache.ignite.internal.managers.discovery.GridLocalMetrics
 org.apache.ignite.internal.managers.eventstorage.GridEventStorageMessage
 org.apache.ignite.internal.managers.indexing.GridIndexingManager$1
 org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerAdapter
 org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager$1
+org.apache.ignite.internal.portable.PortableClassDescriptor$Mode
+org.apache.ignite.internal.portable.PortableContext
+org.apache.ignite.internal.portable.PortableLazyMap$1$1$1
+org.apache.ignite.internal.portable.PortableMetaDataImpl
+org.apache.ignite.internal.portable.PortableObjectEx
+org.apache.ignite.internal.portable.PortableObjectImpl
+org.apache.ignite.internal.portable.PortableObjectOffheapImpl
 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.cache.CacheAtomicUpdateTimeoutCheckedException
 org.apache.ignite.internal.processors.cache.CacheEntryImpl
+org.apache.ignite.internal.processors.cache.CacheEntryImplEx
 org.apache.ignite.internal.processors.cache.CacheEntryInfoCollection
 org.apache.ignite.internal.processors.cache.CacheEntryPredicate
 org.apache.ignite.internal.processors.cache.CacheEntryPredicateAdapter
@@ -290,47 +309,48 @@ org.apache.ignite.internal.processors.cache.CacheOperationContext
 org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException
 org.apache.ignite.internal.processors.cache.CacheStorePartialUpdateException
 org.apache.ignite.internal.processors.cache.CacheType
-org.apache.ignite.internal.processors.cache.CacheEntryImplEx
 org.apache.ignite.internal.processors.cache.CacheWeakQueryIteratorsHolder$WeakQueryFutureIterator
 org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch
 org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest
 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$16$1
+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
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$18
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$2
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$26$1
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$28
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$29$1
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$25$1
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$27
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$28$1
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$29
 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$31
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$4
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$6
+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$68
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$69
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$69$1
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$7
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$70
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$71
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$71$1
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$72
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$73
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$74
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$9
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$AsyncOpRetryFuture$1
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$AsyncOpRetryFuture$1$1
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$ClearTask
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$GlobalClearAllJob
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$GlobalClearCallable
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$GlobalClearKeySetJob
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$GlobalSizeCallable
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$LoadCacheClosure
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$LoadKeysCallable
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$SizeJob
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$SizeTask
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$TopologyVersionAwareJob
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$TopologyVersionAwareJob$1
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$UpdateGetTimeStatClosure
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$UpdatePutAndGetTimeStatClosure
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$UpdatePutTimeStatClosure
@@ -370,6 +390,7 @@ org.apache.ignite.internal.processors.cache.GridCacheEvictionResponse
 org.apache.ignite.internal.processors.cache.GridCacheExplicitLockSpan
 org.apache.ignite.internal.processors.cache.GridCacheExplicitLockSpan$1
 org.apache.ignite.internal.processors.cache.GridCacheFilterFailedException
+org.apache.ignite.internal.processors.cache.GridCacheGateway$State
 org.apache.ignite.internal.processors.cache.GridCacheIndexUpdateException
 org.apache.ignite.internal.processors.cache.GridCacheIoManager$1$1
 org.apache.ignite.internal.processors.cache.GridCacheIoManager$2
@@ -384,6 +405,7 @@ org.apache.ignite.internal.processors.cache.GridCacheMessage
 org.apache.ignite.internal.processors.cache.GridCacheMultiTxFuture$1
 org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate
 org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate$Mask
+org.apache.ignite.internal.processors.cache.GridCacheMvccManager$4
 org.apache.ignite.internal.processors.cache.GridCacheMvccManager$5
 org.apache.ignite.internal.processors.cache.GridCacheMvccManager$6
 org.apache.ignite.internal.processors.cache.GridCacheMvccManager$7
@@ -394,27 +416,37 @@ org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$1$
 org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$2
 org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$3
 org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$4
+org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$5
+org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$6
 org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeFutureSet
 org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$MessageHandler
 org.apache.ignite.internal.processors.cache.GridCacheProcessor$2
 org.apache.ignite.internal.processors.cache.GridCacheProcessor$3
-org.apache.ignite.internal.processors.cache.GridCacheProcessor$5
+org.apache.ignite.internal.processors.cache.GridCacheProcessor$4
+org.apache.ignite.internal.processors.cache.GridCacheProcessor$6
 org.apache.ignite.internal.processors.cache.GridCacheProcessor$LocalAffinityFunction
 org.apache.ignite.internal.processors.cache.GridCacheProxyImpl
 org.apache.ignite.internal.processors.cache.GridCacheReturn
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$10
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$12
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$14
+org.apache.ignite.internal.processors.cache.GridCacheSwapManager$15
+org.apache.ignite.internal.processors.cache.GridCacheSwapManager$16
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$17
+org.apache.ignite.internal.processors.cache.GridCacheSwapManager$18
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$2
+org.apache.ignite.internal.processors.cache.GridCacheSwapManager$21
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$3
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$4
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$5
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$6
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$7
+org.apache.ignite.internal.processors.cache.GridCacheSwapManager$CloseablePartitionsIterator
 org.apache.ignite.internal.processors.cache.GridCacheSwapManager$IteratorWrapper
 org.apache.ignite.internal.processors.cache.GridCacheTryPutFailedException
+org.apache.ignite.internal.processors.cache.GridCacheTtlManager$GridConcurrentSkipListSetEx
 org.apache.ignite.internal.processors.cache.GridCacheUtilityKey
+org.apache.ignite.internal.processors.cache.GridCacheUtils$1
 org.apache.ignite.internal.processors.cache.GridCacheUtils$10
 org.apache.ignite.internal.processors.cache.GridCacheUtils$11
 org.apache.ignite.internal.processors.cache.GridCacheUtils$12
@@ -428,10 +460,8 @@ org.apache.ignite.internal.processors.cache.GridCacheUtils$19
 org.apache.ignite.internal.processors.cache.GridCacheUtils$2
 org.apache.ignite.internal.processors.cache.GridCacheUtils$20
 org.apache.ignite.internal.processors.cache.GridCacheUtils$21
-org.apache.ignite.internal.processors.cache.GridCacheUtils$22
 org.apache.ignite.internal.processors.cache.GridCacheUtils$23
-org.apache.ignite.internal.processors.cache.GridCacheUtils$25
-org.apache.ignite.internal.processors.cache.GridCacheUtils$26
+org.apache.ignite.internal.processors.cache.GridCacheUtils$24
 org.apache.ignite.internal.processors.cache.GridCacheUtils$3
 org.apache.ignite.internal.processors.cache.GridCacheUtils$4
 org.apache.ignite.internal.processors.cache.GridCacheUtils$5
@@ -445,7 +475,6 @@ org.apache.ignite.internal.processors.cache.GridLoadCacheCloseablePredicate
 org.apache.ignite.internal.processors.cache.IgniteCacheProxy
 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$1
 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$2
-org.apache.ignite.internal.processors.cache.IgniteCacheProxy$4
 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$5
 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$6
 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$7
@@ -456,14 +485,16 @@ org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresMa
 org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresManager$QueueHeaderPredicate
 org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresManager$RemoveSetDataCallable
 org.apache.ignite.internal.processors.cache.distributed.GridCacheCommittedTxInfo
+org.apache.ignite.internal.processors.cache.distributed.GridCacheTtlUpdateRequest
 org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryFuture$1
+org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryFuture$2
 org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryRequest
 org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryResponse
-org.apache.ignite.internal.processors.cache.distributed.GridCacheTtlUpdateRequest
 org.apache.ignite.internal.processors.cache.distributed.GridDistributedBaseMessage
 org.apache.ignite.internal.processors.cache.distributed.GridDistributedCacheAdapter
 org.apache.ignite.internal.processors.cache.distributed.GridDistributedCacheAdapter$1
-org.apache.ignite.internal.processors.cache.distributed.GridDistributedCacheAdapter$GlobalRemoveAllCallable
+org.apache.ignite.internal.processors.cache.distributed.GridDistributedCacheAdapter$GlobalRemoveAllJob
+org.apache.ignite.internal.processors.cache.distributed.GridDistributedCacheAdapter$RemoveAllTask
 org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockCancelledException
 org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockRequest
 org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockResponse
@@ -496,6 +527,7 @@ org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLockFuture$3
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLockRequest
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLockResponse
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState
+org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionsReservation$1
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter$1
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter$2
@@ -510,6 +542,7 @@ org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactional
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter$9$1$1
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter$9$2
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishFuture$1
+org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishFuture$2
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishResponse
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocal
@@ -517,6 +550,7 @@ org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocal$1
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocal$2
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter$1
+org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter$2
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$1
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$2
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$3
@@ -524,6 +558,7 @@ org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequ
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareResponse
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxRemote
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtUnlockRequest
+org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtUnreservedPartitionException
 org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedGetFuture$1
 org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedGetFuture$2
 org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedGetFuture$MiniFuture$1
@@ -556,6 +591,7 @@ org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomic
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$1
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$2
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$3
+org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$4
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateRequest
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse
 org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedCache
@@ -568,6 +604,7 @@ org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtCol
 org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedLockFuture$2
 org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedLockFuture$3
 org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedLockFuture$4
+org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedLockFuture$MiniFuture$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.GridDhtPartitionDemandMessage
@@ -584,8 +621,8 @@ org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPar
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsAbstractMessage
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture$1
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture$2
-org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture$3
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture$4
+org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture$5
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleRequest
@@ -622,12 +659,22 @@ org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$2
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$3
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$4
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$MiniFuture$1
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockRequest
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockResponse
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$1
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$2
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$3
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$4
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$MiniFuture$1
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearPessimisticTxPrepareFuture$1
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearPessimisticTxPrepareFuture$2
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearPessimisticTxPrepareFuture$3
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearTransactionalCache
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearTransactionalCache$1
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearTransactionalCache$2
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishFuture$1
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishFuture$2
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishRequest
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishResponse
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal
@@ -638,11 +685,7 @@ org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal$4
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal$5
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal$6
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal$7
-org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareFuture$1
-org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareFuture$2
-org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareFuture$3
-org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareFuture$4
-org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareFuture$5
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareFutureAdapter$1
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxRemote
@@ -656,6 +699,13 @@ org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache$4
 org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache$5
 org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache$6
 org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache$9
+org.apache.ignite.internal.processors.cache.portable.CacheDefaultPortableAffinityKeyMapper
+org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl$1
+org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl$4
+org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl$MetaDataEntryFilter
+org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl$MetaDataPredicate
+org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl$MetaDataProcessor
+org.apache.ignite.internal.processors.cache.portable.PortableMetaDataKey
 org.apache.ignite.internal.processors.cache.query.CacheQueryCloseableScanBiPredicate
 org.apache.ignite.internal.processors.cache.query.CacheQueryType
 org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryFuture$1
@@ -666,13 +716,15 @@ org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManag
 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.GridCacheQueryAdapter$1
+org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$2
+org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$CacheQueryFallbackFuture$1
+org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$CacheQueryFallbackFuture$1$1
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter$1
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter$2
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$1$1
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$1$2
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$10
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$11$1
-org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$12$1
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$2
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$3
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$4
@@ -699,7 +751,6 @@ org.apache.ignite.internal.processors.cache.query.GridCacheQueryType
 org.apache.ignite.internal.processors.cache.query.GridCacheSqlIndexMetadata
 org.apache.ignite.internal.processors.cache.query.GridCacheSqlMetadata
 org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery
-org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery
 org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryEntry
 org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryEvent
 org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFilterEx
@@ -769,6 +820,7 @@ org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$Po
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$PostMissClosure
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$2
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$3
+org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$4
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$AtomicInt
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$CommitListener
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$CommittedVersion
@@ -809,13 +861,19 @@ 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.continuous.AbstractContinuousMessage
 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$DiscoveryData
 org.apache.ignite.internal.processors.continuous.GridContinuousProcessor$DiscoveryDataItem
-org.apache.ignite.internal.processors.continuous.GridContinuousProcessor$StartRequestData
+org.apache.ignite.internal.processors.continuous.GridContinuousProcessor$LocalRoutineInfo
+org.apache.ignite.internal.processors.continuous.StartRequestData
+org.apache.ignite.internal.processors.continuous.StartRoutineAckDiscoveryMessage
+org.apache.ignite.internal.processors.continuous.StartRoutineDiscoveryMessage
+org.apache.ignite.internal.processors.continuous.StopRoutineAckDiscoveryMessage
+org.apache.ignite.internal.processors.continuous.StopRoutineDiscoveryMessage
 org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor$3
 org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor$4
 org.apache.ignite.internal.processors.datastreamer.DataStreamerCacheUpdaters$Batched
@@ -825,6 +883,7 @@ org.apache.ignite.internal.processors.datastreamer.DataStreamerEntry
 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$1
 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$4
 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$5
+org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$6
 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$Buffer$1
 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$Buffer$2
 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$DataStreamerPda
@@ -838,13 +897,13 @@ org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$1
 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$14
+org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$13
 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$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
@@ -857,6 +916,8 @@ org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor$Cac
 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.DataStructuresProcessor$RemoveDataStructureProcessor
 org.apache.ignite.internal.processors.datastructures.GridCacheAtomicLongImpl
@@ -930,6 +991,7 @@ org.apache.ignite.internal.processors.igfs.IgfsImpl$IgfsGlobalSpaceTask$1
 org.apache.ignite.internal.processors.igfs.IgfsInputStreamDescriptor
 org.apache.ignite.internal.processors.igfs.IgfsInputStreamImpl$1
 org.apache.ignite.internal.processors.igfs.IgfsInvalidRangeException
+org.apache.ignite.internal.processors.igfs.IgfsIpcHandler$2
 org.apache.ignite.internal.processors.igfs.IgfsJobImpl
 org.apache.ignite.internal.processors.igfs.IgfsListingEntry
 org.apache.ignite.internal.processors.igfs.IgfsMetaManager$1
@@ -956,6 +1018,14 @@ org.apache.ignite.internal.processors.job.GridJobWorker$3
 org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor$SnapshotReducer
 org.apache.ignite.internal.processors.query.GridQueryFieldMetadata
 org.apache.ignite.internal.processors.query.GridQueryIndexType
+org.apache.ignite.internal.processors.query.GridQueryProcessor$2
+org.apache.ignite.internal.processors.query.GridQueryProcessor$3
+org.apache.ignite.internal.processors.query.GridQueryProcessor$4
+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.h2.twostep.messages.GridQueryCancelRequest
 org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse
 org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest
@@ -981,26 +1051,40 @@ org.apache.ignite.internal.processors.rest.client.message.GridClientTaskResultBe
 org.apache.ignite.internal.processors.rest.client.message.GridClientTopologyRequest
 org.apache.ignite.internal.processors.rest.client.message.GridRouterRequest
 org.apache.ignite.internal.processors.rest.client.message.GridRouterResponse
+org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$1
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$2
+org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$4
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$AddCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$AppendCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$CacheCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$CacheOperationCallable
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$CacheProjectionCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$CasCommand
+org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$ContainsKeyCommand
+org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$ContainsKeysCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$FixedResult
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$FlaggedCacheOperationCallable
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$GetAllCommand
+org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$GetAndPutCommand
+org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$GetAndPutIfAbsentCommand
+org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$GetAndRemoveCommand
+org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$GetAndReplaceCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$GetCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$MetricsCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$PrependCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$PutAllCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$PutCommand
+org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$PutIfAbsentCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$RemoveAllCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$RemoveCommand
+org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$RemoveValueCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$ReplaceCommand
+org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$ReplaceValueCommand
+org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler$SizeCommand
 org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheRestResponse
 org.apache.ignite.internal.processors.rest.handlers.datastructures.DataStructuresCommandHandler$1
+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.task.GridTaskCommandHandler$2
 org.apache.ignite.internal.processors.rest.handlers.task.GridTaskCommandHandler$ExeCallable
 org.apache.ignite.internal.processors.rest.handlers.task.GridTaskResultRequest
@@ -1017,6 +1101,8 @@ org.apache.ignite.internal.processors.service.GridServiceDeployment
 org.apache.ignite.internal.processors.service.GridServiceDeploymentKey
 org.apache.ignite.internal.processors.service.GridServiceMethodNotFoundException
 org.apache.ignite.internal.processors.service.GridServiceNotFoundException
+org.apache.ignite.internal.processors.service.GridServiceProcessor$ServiceAssignmentsPredicate
+org.apache.ignite.internal.processors.service.GridServiceProcessor$ServiceDeploymentPredicate
 org.apache.ignite.internal.processors.service.GridServiceProxy
 org.apache.ignite.internal.processors.service.GridServiceProxy$ServiceProxyCallable
 org.apache.ignite.internal.processors.service.ServiceContextImpl
@@ -1097,13 +1183,14 @@ org.apache.ignite.internal.util.IgniteUtils$11
 org.apache.ignite.internal.util.IgniteUtils$12
 org.apache.ignite.internal.util.IgniteUtils$13
 org.apache.ignite.internal.util.IgniteUtils$14
+org.apache.ignite.internal.util.IgniteUtils$16
 org.apache.ignite.internal.util.IgniteUtils$2
-org.apache.ignite.internal.util.IgniteUtils$20
-org.apache.ignite.internal.util.IgniteUtils$21
 org.apache.ignite.internal.util.IgniteUtils$22
 org.apache.ignite.internal.util.IgniteUtils$23
 org.apache.ignite.internal.util.IgniteUtils$24
 org.apache.ignite.internal.util.IgniteUtils$25
+org.apache.ignite.internal.util.IgniteUtils$26
+org.apache.ignite.internal.util.IgniteUtils$27
 org.apache.ignite.internal.util.IgniteUtils$3
 org.apache.ignite.internal.util.IgniteUtils$4
 org.apache.ignite.internal.util.IgniteUtils$5
@@ -1143,48 +1230,10 @@ org.apache.ignite.internal.util.lang.GridCloseableIterator
 org.apache.ignite.internal.util.lang.GridClosureException
 org.apache.ignite.internal.util.lang.GridFunc$1
 org.apache.ignite.internal.util.lang.GridFunc$10
-org.apache.ignite.internal.util.lang.GridFunc$100
-org.apache.ignite.internal.util.lang.GridFunc$101
-org.apache.ignite.internal.util.lang.GridFunc$102
-org.apache.ignite.internal.util.lang.GridFunc$103
-org.apache.ignite.internal.util.lang.GridFunc$104
-org.apache.ignite.internal.util.lang.GridFunc$105
-org.apache.ignite.internal.util.lang.GridFunc$106
-org.apache.ignite.internal.util.lang.GridFunc$107
-org.apache.ignite.internal.util.lang.GridFunc$108
-org.apache.ignite.internal.util.lang.GridFunc$109
 org.apache.ignite.internal.util.lang.GridFunc$11
-org.apache.ignite.internal.util.lang.GridFunc$110
-org.apache.ignite.internal.util.lang.GridFunc$111
-org.apache.ignite.internal.util.lang.GridFunc$112
-org.apache.ignite.internal.util.lang.GridFunc$113
-org.apache.ignite.internal.util.lang.GridFunc$114
-org.apache.ignite.internal.util.lang.GridFunc$115
-org.apache.ignite.internal.util.lang.GridFunc$117
-org.apache.ignite.internal.util.lang.GridFunc$118
 org.apache.ignite.internal.util.lang.GridFunc$12
-org.apache.ignite.internal.util.lang.GridFunc$120
-org.apache.ignite.internal.util.lang.GridFunc$121
-org.apache.ignite.internal.util.lang.GridFunc$123
-org.apache.ignite.internal.util.lang.GridFunc$124
-org.apache.ignite.internal.util.lang.GridFunc$125
-org.apache.ignite.internal.util.lang.GridFunc$127
-org.apache.ignite.internal.util.lang.GridFunc$128
-org.apache.ignite.internal.util.lang.GridFunc$129
 org.apache.ignite.internal.util.lang.GridFunc$13
-org.apache.ignite.internal.util.lang.GridFunc$130
-org.apache.ignite.internal.util.lang.GridFunc$131
-org.apache.ignite.internal.util.lang.GridFunc$132
-org.apache.ignite.internal.util.lang.GridFunc$134
-org.apache.ignite.internal.util.lang.GridFunc$135
-org.apache.ignite.internal.util.lang.GridFunc$136
-org.apache.ignite.internal.util.lang.GridFunc$137
-org.apache.ignite.internal.util.lang.GridFunc$138
-org.apache.ignite.internal.util.lang.GridFunc$139
 org.apache.ignite.internal.util.lang.GridFunc$14
-org.apache.ignite.internal.util.lang.GridFunc$140
-org.apache.ignite.internal.util.lang.GridFunc$141
-org.apache.ignite.internal.util.lang.GridFunc$142
 org.apache.ignite.internal.util.lang.GridFunc$15
 org.apache.ignite.internal.util.lang.GridFunc$16
 org.apache.ignite.internal.util.lang.GridFunc$17
@@ -1205,8 +1254,9 @@ org.apache.ignite.internal.util.lang.GridFunc$3
 org.apache.ignite.internal.util.lang.GridFunc$30
 org.apache.ignite.internal.util.lang.GridFunc$31
 org.apache.ignite.internal.util.lang.GridFunc$32
+org.apache.ignite.internal.util.lang.GridFunc$32$1
 org.apache.ignite.internal.util.lang.GridFunc$33
-org.apache.ignite.internal.util.lang.GridFunc$34
+org.apache.ignite.internal.util.lang.GridFunc$33$1
 org.apache.ignite.internal.util.lang.GridFunc$35
 org.apache.ignite.internal.util.lang.GridFunc$36
 org.apache.ignite.internal.util.lang.GridFunc$37
@@ -1223,10 +1273,18 @@ org.apache.ignite.internal.util.lang.GridFunc$46
 org.apache.ignite.internal.util.lang.GridFunc$47
 org.apache.ignite.internal.util.lang.GridFunc$48
 org.apache.ignite.internal.util.lang.GridFunc$49
+org.apache.ignite.internal.util.lang.GridFunc$49$1
+org.apache.ignite.internal.util.lang.GridFunc$49$2
 org.apache.ignite.internal.util.lang.GridFunc$5
 org.apache.ignite.internal.util.lang.GridFunc$50
+org.apache.ignite.internal.util.lang.GridFunc$50$1
+org.apache.ignite.internal.util.lang.GridFunc$50$2
 org.apache.ignite.internal.util.lang.GridFunc$51
+org.apache.ignite.internal.util.lang.GridFunc$51$1
+org.apache.ignite.internal.util.lang.GridFunc$51$2
 org.apache.ignite.internal.util.lang.GridFunc$52
+org.apache.ignite.internal.util.lang.GridFunc$52$1
+org.apache.ignite.internal.util.lang.GridFunc$52$2
 org.apache.ignite.internal.util.lang.GridFunc$53
 org.apache.ignite.internal.util.lang.GridFunc$54
 org.apache.ignite.internal.util.lang.GridFunc$55
@@ -1239,57 +1297,16 @@ org.apache.ignite.internal.util.lang.GridFunc$60
 org.apache.ignite.internal.util.lang.GridFunc$61
 org.apache.ignite.internal.util.lang.GridFunc$62
 org.apache.ignite.internal.util.lang.GridFunc$63
-org.apache.ignite.internal.util.lang.GridFunc$64
-org.apache.ignite.internal.util.lang.GridFunc$65
-org.apache.ignite.internal.util.lang.GridFunc$66
-org.apache.ignite.internal.util.lang.GridFunc$66$1
-org.apache.ignite.internal.util.lang.GridFunc$67
-org.apache.ignite.internal.util.lang.GridFunc$67$1
-org.apache.ignite.internal.util.lang.GridFunc$69
 org.apache.ignite.internal.util.lang.GridFunc$7
-org.apache.ignite.internal.util.lang.GridFunc$70
-org.apache.ignite.internal.util.lang.GridFunc$71
-org.apache.ignite.internal.util.lang.GridFunc$72
-org.apache.ignite.internal.util.lang.GridFunc$73
-org.apache.ignite.internal.util.lang.GridFunc$74
-org.apache.ignite.internal.util.lang.GridFunc$75
-org.apache.ignite.internal.util.lang.GridFunc$76
-org.apache.ignite.internal.util.lang.GridFunc$77
-org.apache.ignite.internal.util.lang.GridFunc$78
-org.apache.ignite.internal.util.lang.GridFunc$79
 org.apache.ignite.internal.util.lang.GridFunc$8
-org.apache.ignite.internal.util.lang.GridFunc$80
-org.apache.ignite.internal.util.lang.GridFunc$82
-org.apache.ignite.internal.util.lang.GridFunc$83
-org.apache.ignite.internal.util.lang.GridFunc$84
-org.apache.ignite.internal.util.lang.GridFunc$85
-org.apache.ignite.internal.util.lang.GridFunc$86
-org.apache.ignite.internal.util.lang.GridFunc$87
 org.apache.ignite.internal.util.lang.GridFunc$9
-org.apache.ignite.internal.util.lang.GridFunc$90
-org.apache.ignite.internal.util.lang.GridFunc$91
-org.apache.ignite.internal.util.lang.GridFunc$92
-org.apache.ignite.internal.util.lang.GridFunc$93
-org.apache.ignite.internal.util.lang.GridFunc$94
-org.apache.ignite.internal.util.lang.GridFunc$95
-org.apache.ignite.internal.util.lang.GridFunc$96
-org.apache.ignite.internal.util.lang.GridFunc$96$1
-org.apache.ignite.internal.util.lang.GridFunc$96$2
-org.apache.ignite.internal.util.lang.GridFunc$97
-org.apache.ignite.internal.util.lang.GridFunc$97$1
-org.apache.ignite.internal.util.lang.GridFunc$97$2
-org.apache.ignite.internal.util.lang.GridFunc$98
-org.apache.ignite.internal.util.lang.GridFunc$98$1
-org.apache.ignite.internal.util.lang.GridFunc$98$2
-org.apache.ignite.internal.util.lang.GridFunc$99
-org.apache.ignite.internal.util.lang.GridFunc$99$1
-org.apache.ignite.internal.util.lang.GridFunc$99$2
 org.apache.ignite.internal.util.lang.GridIterable
 org.apache.ignite.internal.util.lang.GridIterableAdapter
 org.apache.ignite.internal.util.lang.GridIterableAdapter$IteratorWrapper
 org.apache.ignite.internal.util.lang.GridIterator
 org.apache.ignite.internal.util.lang.GridIteratorAdapter
 org.apache.ignite.internal.util.lang.GridMapEntry
+org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter$EntryKey
 org.apache.ignite.internal.util.lang.GridNodePredicate
 org.apache.ignite.internal.util.lang.GridPeerDeployAware
 org.apache.ignite.internal.util.lang.GridPeerDeployAwareAdapter
@@ -1327,6 +1344,7 @@ org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMap$Segment$1
 org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMap$Segment$2
 org.apache.ignite.internal.util.offheap.unsafe.GridUnsafePartitionedMap$2
 org.apache.ignite.internal.util.offheap.unsafe.GridUnsafePartitionedMap$3
+org.apache.ignite.internal.util.offheap.unsafe.GridUnsafePartitionedMap$PartitionedMapCloseableIterator
 org.apache.ignite.internal.util.snaptree.CopyOnWriteManager$COWEpoch
 org.apache.ignite.internal.util.snaptree.Epoch$Root
 org.apache.ignite.internal.util.snaptree.EpochNode
@@ -1504,7 +1522,9 @@ org.apache.ignite.internal.visor.query.VisorQueryNextPageTask$VisorQueryNextPage
 org.apache.ignite.internal.visor.query.VisorQueryResult
 org.apache.ignite.internal.visor.query.VisorQueryResultEx
 org.apache.ignite.internal.visor.query.VisorQueryTask
+org.apache.ignite.internal.visor.util.VisorClusterGroupEmptyException
 org.apache.ignite.internal.visor.util.VisorEventMapper
+org.apache.ignite.internal.visor.util.VisorExceptionWrapper
 org.apache.ignite.internal.visor.util.VisorTaskUtils$4
 org.apache.ignite.lang.IgniteBiClosure
 org.apache.ignite.lang.IgniteBiInClosure
@@ -1538,6 +1558,10 @@ org.apache.ignite.plugin.security.SecuritySubject
 org.apache.ignite.plugin.security.SecuritySubjectType
 org.apache.ignite.plugin.segmentation.SegmentationPolicy
 org.apache.ignite.plugin.segmentation.SegmentationResolver
+org.apache.ignite.portable.PortableException
+org.apache.ignite.portable.PortableInvalidClassException
+org.apache.ignite.portable.PortableObject
+org.apache.ignite.portable.PortableProtocolVersion
 org.apache.ignite.services.Service
 org.apache.ignite.services.ServiceConfiguration
 org.apache.ignite.services.ServiceContext
@@ -1546,6 +1570,7 @@ org.apache.ignite.spi.IgnitePortProtocol
 org.apache.ignite.spi.IgniteSpiCloseableIterator
 org.apache.ignite.spi.IgniteSpiException
 org.apache.ignite.spi.IgniteSpiMultiException
+org.apache.ignite.spi.IgniteSpiOperationTimeoutException
 org.apache.ignite.spi.IgniteSpiVersionCheckException
 org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointData
 org.apache.ignite.spi.collision.jobstealing.JobStealingRequest
@@ -1554,14 +1579,16 @@ org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$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$7
+org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$8
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeClosure
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeMessage
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeTimeoutException
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$NodeIdMessage
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$RecoveryLastReceivedMessage
-org.apache.ignite.spi.discovery.tcp.TcpClientDiscoverySpi$1
-org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi$IpFinderCleaner$1
-org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi$IpFinderCleaner$2
+org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage
+org.apache.ignite.spi.discovery.tcp.ClientImpl$State
+org.apache.ignite.spi.discovery.tcp.ServerImpl$IpFinderCleaner$1
+org.apache.ignite.spi.discovery.tcp.ServerImpl$IpFinderCleaner$2
 org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode
 org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode$1
 org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNodesRing$1
@@ -1571,7 +1598,12 @@ org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoverySpiState
 org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage
 org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAuthFailedMessage
 org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCheckFailedMessage
+org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientAckResponse
+org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientHeartbeatMessage
+org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientPingRequest
+org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientPingResponse
 org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientReconnectMessage
+org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryConnectionCheckMessage
 org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCustomEventMessage
 org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryDiscardMessage
 org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryDuplicateIdMessage
@@ -1591,6 +1623,7 @@ org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingResponse
 org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusCheckMessage
 org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi$1
 org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi$2
+org.apache.ignite.ssl.SslContextFactory
 org.apache.ignite.startup.BasicWarmupClosure
 org.apache.ignite.startup.cmdline.AboutDialog
 org.apache.ignite.startup.cmdline.AboutDialog$1

http://git-wip-us.apache.org/repos/asf/ignite/blob/878dcd92/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
new file mode 100644
index 0000000..00705dc
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.affinity.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.cacheobject.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.marshaller.portable.*;
+import org.apache.ignite.portable.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Test for portable object affinity key.
+ */
+public class GridPortableAffinityKeySelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final AtomicReference<UUID> nodeId = new AtomicReference<>();
+
+    /** VM ip finder for TCP discovery. */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static int GRID_CNT = 5;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration();
+
+        typeCfg.setClassName(TestObject.class.getName());
+        typeCfg.setAffinityKeyFieldName("affKey");
+
+        PortableMarshaller marsh = new PortableMarshaller();
+
+        marsh.setTypeConfigurations(Collections.singleton(typeCfg));
+
+        cfg.setMarshaller(marsh);
+
+        if (!gridName.equals(getTestGridName(GRID_CNT))) {
+            CacheConfiguration cacheCfg = new CacheConfiguration();
+
+            cacheCfg.setCacheMode(PARTITIONED);
+
+            cfg.setCacheConfiguration(cacheCfg);
+        }
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(GRID_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAffinity() throws Exception {
+        checkAffinity(grid(0));
+
+        try (Ignite igniteNoCache = startGrid(GRID_CNT)) {
+            try {
+                igniteNoCache.cache(null);
+            }
+            catch (IllegalArgumentException ignore) {
+                // Expected error.
+            }
+
+            checkAffinity(igniteNoCache);
+        }
+    }
+
+    /**
+     * @param ignite Ignite.
+     * @throws Exception If failed.
+     */
+    private void checkAffinity(Ignite ignite) throws Exception {
+        Affinity<Object> aff = ignite.affinity(null);
+
+        GridAffinityProcessor affProc = ((IgniteKernal)ignite).context().affinity();
+
+        IgniteCacheObjectProcessor cacheObjProc = ((IgniteKernal)ignite).context().cacheObjects();
+
+        CacheObjectContext cacheObjCtx = cacheObjProc.contextForCache(
+            ignite.cache(null).getConfiguration(CacheConfiguration.class));
+
+        for (int i = 0; i < 1000; i++) {
+            assertEquals(i, aff.affinityKey(i));
+
+            assertEquals(i, aff.affinityKey(new TestObject(i)));
+
+            CacheObject cacheObj = cacheObjProc.toCacheObject(cacheObjCtx, new TestObject(i), true);
+
+            assertEquals(i, aff.affinityKey(cacheObj));
+
+            assertEquals(aff.mapKeyToNode(i), aff.mapKeyToNode(new TestObject(i)));
+
+            assertEquals(aff.mapKeyToNode(i), aff.mapKeyToNode(cacheObj));
+
+            assertEquals(i, affProc.affinityKey(null, i));
+
+            assertEquals(i, affProc.affinityKey(null, new TestObject(i)));
+
+            assertEquals(i, affProc.affinityKey(null, cacheObj));
+
+            assertEquals(affProc.mapKeyToNode(null, i), affProc.mapKeyToNode(null, new TestObject(i)));
+
+            assertEquals(affProc.mapKeyToNode(null, i), affProc.mapKeyToNode(null, cacheObj));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAffinityRun() throws Exception {
+        Affinity<Object> aff = grid(0).affinity(null);
+
+        for (int i = 0; i < 1000; i++) {
+            nodeId.set(null);
+
+            grid(0).compute().affinityRun(null, new TestObject(i), new IgniteRunnable() {
+                @IgniteInstanceResource
+                private Ignite ignite;
+
+                @Override public void run() {
+                    nodeId.set(ignite.configuration().getNodeId());
+                }
+            });
+
+            assertEquals(aff.mapKeyToNode(i).id(), nodeId.get());
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAffinityCall() throws Exception {
+        Affinity<Object> aff = grid(0).affinity(null);
+
+        for (int i = 0; i < 1000; i++) {
+            nodeId.set(null);
+
+            grid(0).compute().affinityCall(null, new TestObject(i), new IgniteCallable<Object>() {
+                @IgniteInstanceResource
+                private Ignite ignite;
+
+                @Override public Object call() {
+                    nodeId.set(ignite.configuration().getNodeId());
+
+                    return null;
+                }
+            });
+
+            assertEquals(aff.mapKeyToNode(i).id(), nodeId.get());
+        }
+    }
+
+    /**
+     */
+    private static class TestObject {
+        /** */
+        @SuppressWarnings("UnusedDeclaration")
+        private int affKey;
+
+        /**
+         */
+        private TestObject() {
+            // No-op.
+        }
+
+        /**
+         * @param affKey Affinity key.
+         */
+        private TestObject(int affKey) {
+            this.affKey = affKey;
+        }
+    }
+}