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/09/04 18:28:04 UTC

[50/55] [abbrv] ignite git commit: IGNITE-1348: Moved GridGain's .Net module to Ignite.

IGNITE-1348: Moved GridGain's .Net module to Ignite.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/5cec202c
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/5cec202c
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/5cec202c

Branch: refs/heads/ignite-1093-2
Commit: 5cec202cbe029ca563c112c4916b25605930ceea
Parents: 4d3c1f0
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Fri Sep 4 16:31:55 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Sep 4 16:31:59 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/PortableContext.java      |    7 +
 .../dotnet/PlatformDotNetConfiguration.java     |  119 +
 .../PlatformDotNetPortableConfiguration.java    |  228 ++
 ...PlatformDotNetPortableTypeConfiguration.java |  248 ++
 modules/platform/pom.xml                        |   14 +
 .../Apache.Ignite.Core.csproj                   |  283 +-
 .../Cache/CacheAtomicUpdateTimeoutException.cs  |   67 +
 .../Cache/CacheEntryProcessorException.cs       |   79 +
 .../Apache.Ignite.Core/Cache/CacheException.cs  |   68 +
 .../Cache/CachePartialUpdateException.cs        |  119 +
 .../Apache.Ignite.Core/Cache/CachePeekMode.cs   |   68 +
 .../Cache/Event/CacheEntryEventType.cs          |   41 +
 .../Cache/Event/ICacheEntryEvent.cs             |   40 +
 .../Cache/Event/ICacheEntryEventFilter.cs       |   31 +
 .../Cache/Event/ICacheEntryEventListener.cs     |   33 +
 .../Cache/Expiry/ExpiryPolicy.cs                |   89 +
 .../Cache/Expiry/IExpiryPolicy.cs               |   59 +
 .../dotnet/Apache.Ignite.Core/Cache/ICache.cs   |  542 +++
 .../Apache.Ignite.Core/Cache/ICacheAffinity.cs  |  161 +
 .../Apache.Ignite.Core/Cache/ICacheEntry.cs     |   37 +
 .../Cache/ICacheEntryFilter.cs                  |   34 +
 .../Cache/ICacheEntryProcessor.cs               |   45 +
 .../Cache/ICacheEntryProcessorResult.cs         |   40 +
 .../Apache.Ignite.Core/Cache/ICacheLock.cs      |   58 +
 .../Apache.Ignite.Core/Cache/ICacheMetrics.cs   |  486 +++
 .../Cache/IMutableCacheEntry.cs                 |   47 +
 .../Cache/Query/Continuous/ContinuousQuery.cs   |  170 +
 .../Query/Continuous/IContinuousQueryHandle.cs  |   51 +
 .../Cache/Query/IQueryCursor.cs                 |   40 +
 .../Apache.Ignite.Core/Cache/Query/QueryBase.cs |   82 +
 .../Apache.Ignite.Core/Cache/Query/ScanQuery.cs |   77 +
 .../Cache/Query/SqlFieldsQuery.cs               |   81 +
 .../Apache.Ignite.Core/Cache/Query/SqlQuery.cs  |  119 +
 .../Apache.Ignite.Core/Cache/Query/TextQuery.cs |  104 +
 .../Store/CacheParallelLoadStoreAdapter.cs      |  205 ++
 .../Cache/Store/CacheStoreAdapter.cs            |  146 +
 .../Cache/Store/CacheStoreException.cs          |   66 +
 .../Cache/Store/ICacheStore.cs                  |  184 +
 .../Cache/Store/ICacheStoreSession.cs           |   42 +
 .../Cluster/ClusterGroupEmptyException.cs       |   70 +
 .../Cluster/ClusterTopologyException.cs         |   69 +
 .../Apache.Ignite.Core/Cluster/ICluster.cs      |   80 +
 .../Apache.Ignite.Core/Cluster/IClusterGroup.cs |  229 ++
 .../Cluster/IClusterMetrics.cs                  |  515 +++
 .../Apache.Ignite.Core/Cluster/IClusterNode.cs  |  138 +
 .../Cluster/IClusterNodeFilter.cs               |   32 +
 .../Apache.Ignite.Core/Common/IAsyncSupport.cs  |    2 +-
 .../Common/IgniteException.cs                   |    2 +-
 .../Apache.Ignite.Core/Common/IgniteGuid.cs     |  138 +
 .../ComputeExecutionRejectedException.cs        |   69 +
 .../Compute/ComputeJobAdapter.cs                |  122 +
 .../Compute/ComputeJobFailoverException.cs      |   72 +
 .../Compute/ComputeJobResultPolicy.cs           |   45 +
 .../Compute/ComputeTaskAdapter.cs               |   93 +
 .../Compute/ComputeTaskCancelledException.cs    |   69 +
 .../ComputeTaskNoResultCacheAttribute.cs        |   35 +
 .../Compute/ComputeTaskSplitAdapter.cs          |   95 +
 .../Compute/ComputeTaskTimeoutException.cs      |   67 +
 .../Compute/ComputeUserUndeclaredException.cs   |   70 +
 .../Apache.Ignite.Core/Compute/ICompute.cs      |  274 ++
 .../Apache.Ignite.Core/Compute/IComputeFunc.cs  |   55 +
 .../Apache.Ignite.Core/Compute/IComputeJob.cs   |   58 +
 .../Compute/IComputeJobResult.cs                |   73 +
 .../Compute/IComputeReducer.cs                  |   39 +
 .../Apache.Ignite.Core/Compute/IComputeTask.cs  |  132 +
 .../Datastream/IDataStreamer.cs                 |  206 ++
 .../Datastream/IStreamReceiver.cs               |   38 +
 .../Datastream/StreamTransformer.cs             |   73 +
 .../Datastream/StreamVisitor.cs                 |   55 +
 .../Apache.Ignite.Core/Events/CacheEvent.cs     |  176 +
 .../Events/CacheQueryExecutedEvent.cs           |   97 +
 .../Events/CacheQueryReadEvent.cs               |  134 +
 .../Events/CacheRebalancingEvent.cs             |   98 +
 .../Events/CheckpointEvent.cs                   |   50 +
 .../Apache.Ignite.Core/Events/DiscoveryEvent.cs |   80 +
 .../Apache.Ignite.Core/Events/EventBase.cs      |  160 +
 .../Apache.Ignite.Core/Events/EventReader.cs    |   72 +
 .../Apache.Ignite.Core/Events/EventType.cs      |  514 +++
 .../dotnet/Apache.Ignite.Core/Events/IEvent.cs  |   74 +
 .../Apache.Ignite.Core/Events/IEventFilter.cs   |   36 +
 .../dotnet/Apache.Ignite.Core/Events/IEvents.cs |  182 +
 .../Apache.Ignite.Core/Events/JobEvent.cs       |  100 +
 .../Apache.Ignite.Core/Events/SwapSpaceEvent.cs |   50 +
 .../Apache.Ignite.Core/Events/TaskEvent.cs      |   91 +
 .../main/dotnet/Apache.Ignite.Core/IIgnite.cs   |  168 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  140 +
 .../main/dotnet/Apache.Ignite.Core/Ignition.cs  |  626 +++-
 .../Impl/Cache/CacheAffinityImpl.cs             |  275 ++
 .../Apache.Ignite.Core/Impl/Cache/CacheEntry.cs |  126 +
 .../Impl/Cache/CacheEntryFilterHolder.cs        |  147 +
 .../Impl/Cache/CacheEntryProcessorHolder.cs     |  145 +
 .../Impl/Cache/CacheEntryProcessorResult.cs     |   65 +
 .../Cache/CacheEntryProcessorResultHolder.cs    |  127 +
 .../Impl/Cache/CacheEnumerable.cs               |   82 +
 .../Impl/Cache/CacheEnumerator.cs               |  117 +
 .../Impl/Cache/CacheEnumeratorProxy.cs          |  156 +
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |  932 +++++
 .../Apache.Ignite.Core/Impl/Cache/CacheLock.cs  |  171 +
 .../Impl/Cache/CacheMetricsImpl.cs              |  248 ++
 .../Apache.Ignite.Core/Impl/Cache/CacheOp.cs    |   63 +
 .../Impl/Cache/CacheProxyImpl.cs                |  499 +++
 .../Impl/Cache/Event/CacheEntryCreateEvent.cs   |   74 +
 .../Impl/Cache/Event/CacheEntryRemoveEvent.cs   |   74 +
 .../Impl/Cache/Event/CacheEntryUpdateEvent.cs   |   79 +
 .../Impl/Cache/MutableCacheEntry.cs             |  163 +
 .../Impl/Cache/Query/AbstractQueryCursor.cs     |  264 ++
 .../Query/Continuous/ContinuousQueryFilter.cs   |  125 +
 .../Continuous/ContinuousQueryFilterHolder.cs   |  118 +
 .../Continuous/ContinuousQueryHandleImpl.cs     |  216 ++
 .../Query/Continuous/ContinuousQueryUtils.cs    |  115 +
 .../Impl/Cache/Query/FieldsQueryCursor.cs       |   54 +
 .../Impl/Cache/Query/QueryCursor.cs             |   50 +
 .../Impl/Cache/Store/CacheStore.cs              |  263 ++
 .../Impl/Cache/Store/CacheStoreSession.cs       |   53 +
 .../Impl/Cache/Store/CacheStoreSessionProxy.cs  |   63 +
 .../Impl/Cluster/ClusterGroupImpl.cs            |  577 +++
 .../Impl/Cluster/ClusterMetricsImpl.cs          |  292 ++
 .../Impl/Cluster/ClusterNodeImpl.cs             |  221 ++
 .../Impl/Cluster/IClusterGroupEx.cs             |   35 +
 .../Impl/Common/DelegateTypeDescriptor.cs       |  327 ++
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |    3 +-
 .../Impl/Common/FutureConverter.cs              |   62 +
 .../Impl/Common/GridArgumentCheck.cs            |   76 -
 .../Impl/Common/IgniteArgumentCheck.cs          |   76 +
 .../Impl/Common/PortableResultWrapper.cs        |   68 +
 .../Closure/ComputeAbstractClosureTask.cs       |  101 +
 .../Impl/Compute/Closure/ComputeActionJob.cs    |   83 +
 .../Impl/Compute/Closure/ComputeFuncJob.cs      |   89 +
 .../Compute/Closure/ComputeMultiClosureTask.cs  |   56 +
 .../Impl/Compute/Closure/ComputeOutFuncJob.cs   |   76 +
 .../Closure/ComputeReducingClosureTask.cs       |   61 +
 .../Compute/Closure/ComputeSingleClosureTask.cs |   48 +
 .../Compute/Closure/IComputeResourceInjector.cs |   31 +
 .../Apache.Ignite.Core/Impl/Compute/Compute.cs  |  213 ++
 .../Impl/Compute/ComputeAsync.cs                |  261 ++
 .../Impl/Compute/ComputeFunc.cs                 |  119 +
 .../Impl/Compute/ComputeImpl.cs                 |  645 ++++
 .../Impl/Compute/ComputeJob.cs                  |  163 +
 .../Impl/Compute/ComputeJobHolder.cs            |  240 ++
 .../Compute/ComputeJobResultGenericWrapper.cs   |   70 +
 .../Impl/Compute/ComputeJobResultImpl.cs        |   96 +
 .../Impl/Compute/ComputeOutFunc.cs              |  123 +
 .../Impl/Compute/ComputeTaskHolder.cs           |  484 +++
 .../Impl/Datastream/DataStreamerBatch.cs        |  269 ++
 .../Impl/Datastream/DataStreamerEntry.cs        |   64 +
 .../Impl/Datastream/DataStreamerImpl.cs         |  832 +++++
 .../Impl/Datastream/DataStreamerRemoveEntry.cs  |   48 +
 .../Impl/Datastream/StreamReceiverHolder.cs     |  144 +
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |  498 +++
 .../Impl/Events/EventsAsync.cs                  |  158 +
 .../Impl/Events/RemoteListenEventFilter.cs      |   85 +
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |  204 ++
 .../Impl/Handle/HandleRegistry.cs               |    4 +-
 .../Apache.Ignite.Core/Impl/IInteropCallback.cs |   34 +
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  547 +++
 .../Impl/IgniteConfigurationEx.cs               |   57 +
 .../Apache.Ignite.Core/Impl/IgniteManager.cs    |  492 +++
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |  351 ++
 .../Apache.Ignite.Core/Impl/IgniteUtils.cs      |  438 +++
 .../Impl/Interop/InteropDotNetConfiguration.cs  |   62 +
 .../InteropDotNetPortableConfiguration.cs       |  127 +
 .../InteropDotNetPortableTypeConfiguration.cs   |  151 +
 .../Impl/InteropExceptionHolder.cs              |   85 +
 .../Impl/LifecycleBeanHolder.cs                 |   66 +
 .../Impl/Memory/InteropExternalMemory.cs        |   46 +
 .../Impl/Memory/InteropMemoryUtils.cs           |   38 +
 .../Impl/Memory/PlatformMemoryManager.cs        |    5 +-
 .../Impl/Memory/PlatformMemoryStream.cs         |    3 +-
 .../Impl/Memory/PlatformMemoryUtils.cs          |    2 +-
 .../Impl/Memory/PlatformPooledMemory.cs         |    2 +-
 .../Impl/Messaging/MessageFilterHolder.cs       |  179 +
 .../Impl/Messaging/Messaging.cs                 |  262 ++
 .../Impl/Messaging/MessagingAsync.cs            |   68 +
 .../Apache.Ignite.Core/Impl/NativeMethods.cs    |   47 +
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |  715 ++++
 .../Portable/IPortableSystemTypeSerializer.cs   |   34 +
 .../Impl/Portable/IPortableTypeDescriptor.cs    |  124 +
 .../Impl/Portable/IPortableWriteAware.cs        |   34 +
 .../Impl/Portable/Io/PortableAbstractStream.cs  | 1299 +++++++
 .../Impl/Portable/Io/PortableHeapStream.cs      |  447 +++
 .../Impl/Portable/Io/PortableStreamAdapter.cs   |  114 +
 .../Metadata/IPortableMetadataHandler.cs        |   41 +
 .../Metadata/PortableHashsetMetadataHandler.cs  |   69 +
 .../Portable/Metadata/PortableMetadataHolder.cs |  149 +
 .../Portable/Metadata/PortableMetadataImpl.cs   |  200 ++
 .../Impl/Portable/PortableBuilderField.cs       |   73 +
 .../Impl/Portable/PortableBuilderImpl.cs        |  923 +++++
 .../Impl/Portable/PortableCollectionInfo.cs     |  251 ++
 .../Impl/Portable/PortableFullTypeDescriptor.cs |  203 ++
 .../Impl/Portable/PortableHandleDictionary.cs   |  187 +
 .../Portable/PortableMarshalAwareSerializer.cs  |   45 +
 .../Impl/Portable/PortableMarshaller.cs         |  603 ++++
 .../Impl/Portable/PortableMode.cs               |   40 +
 .../Impl/Portable/PortableObjectHandle.cs       |   59 +
 .../PortableOrSerializableObjectHolder.cs       |   66 +
 .../Portable/PortableReaderHandleDictionary.cs  |   42 +
 .../Impl/Portable/PortableReaderImpl.cs         | 1013 ++++++
 .../Impl/Portable/PortableReflectiveRoutines.cs |  483 +++
 .../Portable/PortableReflectiveSerializer.cs    |  218 ++
 .../Portable/PortableSurrogateTypeDescriptor.cs |  133 +
 .../Impl/Portable/PortableSystemHandlers.cs     | 1336 +++++++
 .../Portable/PortableSystemTypeSerializer.cs    |   62 +
 .../Impl/Portable/PortableUserObject.cs         |  385 ++
 .../Impl/Portable/PortableUtils.cs              | 2039 +++++++++++
 .../Impl/Portable/PortableWriterImpl.cs         | 1305 +++++++
 .../Impl/Portable/PortablesImpl.cs              |  205 ++
 .../Impl/Portable/SerializableObjectHolder.cs   |   66 +
 .../Impl/Portable/TypeResolver.cs               |  227 ++
 .../Impl/Resource/IResourceInjector.cs          |   27 +
 .../Impl/Resource/ResourceFieldInjector.cs      |   47 +
 .../Impl/Resource/ResourceMethodInjector.cs     |   48 +
 .../Impl/Resource/ResourceProcessor.cs          |  105 +
 .../Impl/Resource/ResourcePropertyInjector.cs   |   47 +
 .../Impl/Resource/ResourceTypeDescriptor.cs     |  291 ++
 .../Impl/Services/ServiceContext.cs             |   60 +
 .../Impl/Services/ServiceDescriptor.cs          |  106 +
 .../Impl/Services/ServiceProxy.cs               |   71 +
 .../Impl/Services/ServiceProxyInvoker.cs        |  136 +
 .../Impl/Services/ServiceProxySerializer.cs     |  140 +
 .../Impl/Services/Services.cs                   |  316 ++
 .../Impl/Services/ServicesAsync.cs              |   89 +
 .../Impl/Transactions/AsyncTransaction.cs       |   78 +
 .../Impl/Transactions/Transaction.cs            |  155 +
 .../Impl/Transactions/TransactionImpl.cs        |  489 +++
 .../Impl/Transactions/TransactionMetricsImpl.cs |   62 +
 .../Impl/Transactions/TransactionsImpl.cs       |  201 ++
 .../Impl/Unmanaged/IUnmanagedTarget.cs          |   42 +
 .../Impl/Unmanaged/UnmanagedCallbackHandlers.cs |   99 +
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        | 1152 ++++++
 .../Impl/Unmanaged/UnmanagedContext.cs          |   53 +
 .../Unmanaged/UnmanagedNonReleaseableTarget.cs  |   68 +
 .../Impl/Unmanaged/UnmanagedTarget.cs           |   77 +
 .../Impl/Unmanaged/UnmanagedUtils.cs            | 1252 +++++++
 .../Lifecycle/ILifecycleBean.cs                 |   64 +
 .../Lifecycle/LifecycleEventType.cs             |   49 +
 .../Messaging/IMessageFilter.cs                 |   35 +
 .../Apache.Ignite.Core/Messaging/IMessaging.cs  |  105 +
 .../Portable/IPortableBuilder.cs                |   78 +
 .../Portable/IPortableIdMapper.cs               |   40 +
 .../Portable/IPortableMarshalAware.cs           |   39 +
 .../Portable/IPortableMetadata.cs               |   61 +
 .../Portable/IPortableNameMapper.cs             |   39 +
 .../Portable/IPortableObject.cs                 |   44 +
 .../Portable/IPortableRawReader.cs              |  264 ++
 .../Portable/IPortableRawWriter.cs              |  221 ++
 .../Portable/IPortableReader.cs                 |  340 ++
 .../Portable/IPortableSerializer.cs             |   39 +
 .../Portable/IPortableWriter.cs                 |  259 ++
 .../Apache.Ignite.Core/Portable/IPortables.cs   |  120 +
 .../Portable/PortableConfiguration.cs           |  122 +
 .../Portable/PortableException.cs               |   64 +
 .../Portable/PortableTypeConfiguration.cs       |  162 +
 .../Portable/PortableTypeNames.cs               |  115 +
 .../Resource/InstanceResourceAttribute.cs       |   35 +
 .../Resource/StoreSessionResourceAttribute.cs   |   32 +
 .../Apache.Ignite.Core/Services/IService.cs     |   51 +
 .../Services/IServiceContext.cs                 |   69 +
 .../Services/IServiceDescriptor.cs              |   96 +
 .../Apache.Ignite.Core/Services/IServices.cs    |  181 +
 .../Services/ServiceConfiguration.cs            |   62 +
 .../Services/ServiceInvocationException.cs      |  101 +
 .../Transactions/ITransaction.cs                |  230 ++
 .../Transactions/ITransactionMetrics.cs         |   47 +
 .../Transactions/ITransactions.cs               |   73 +
 .../Transactions/TransactionConcurrency.cs      |   36 +
 .../TransactionHeuristicException.cs            |   72 +
 .../Transactions/TransactionIsolation.cs        |   41 +
 .../TransactionOptimisticException.cs           |   69 +
 .../TransactionRollbackException.cs             |   68 +
 .../Transactions/TransactionState.cs            |   70 +
 .../Transactions/TransactionTimeoutException.cs |   69 +
 .../platform/src/main/dotnet/Apache.Ignite.sln  |   27 +-
 .../dotnet/PlatformDotNetConfiguration.java     |  119 -
 .../PlatformDotNetPortableConfiguration.java    |  228 --
 ...PlatformDotNetPortableTypeConfiguration.java |  248 --
 .../Apache.Ignite.Core.Tests.TestDll.csproj     |   52 +
 .../Properties/AssemblyInfo.cs                  |   49 +
 .../TestClass.cs                                |   35 +
 .../Apache.Ignite.Core.Tests.csproj             |  165 +-
 .../Cache/CacheAbstractTest.cs                  | 3321 ++++++++++++++++++
 .../Cache/CacheAffinityTest.cs                  |  139 +
 .../Cache/CacheDynamicStartTest.cs              |  281 ++
 .../Cache/CacheEntryTest.cs                     |   69 +
 .../Cache/CacheForkedTest.cs                    |   82 +
 .../Cache/CacheLocalAtomicTest.cs               |   57 +
 .../Cache/CacheLocalTest.cs                     |   56 +
 .../CachePartitionedAtomicNearEnabledTest.cs    |   50 +
 .../Cache/CachePartitionedAtomicTest.cs         |   50 +
 .../Cache/CachePartitionedNearEnabledTest.cs    |   50 +
 .../Cache/CachePartitionedTest.cs               |   50 +
 .../Cache/CacheReplicatedAtomicTest.cs          |   60 +
 .../Cache/CacheReplicatedTest.cs                |   60 +
 .../Cache/CacheTestAsyncWrapper.cs              |  436 +++
 .../Cache/Query/CacheQueriesTest.cs             |  928 +++++
 .../Continuous/ContinuousQueryAbstractTest.cs   | 1181 +++++++
 .../ContinuousQueryAtomicBackupTest.cs          |   33 +
 .../ContinuousQueryAtomicNoBackupTest.cs        |   34 +
 .../ContinuousQueryNoBackupAbstractTest.cs      |   72 +
 .../ContinuousQueryTransactionalBackupTest.cs   |   34 +
 .../ContinuousQueryTransactionalNoBackupTest.cs |   33 +
 .../Cache/Store/CacheParallelLoadStoreTest.cs   |  110 +
 .../Cache/Store/CacheStoreSessionTest.cs        |  285 ++
 .../Cache/Store/CacheStoreTest.cs               |  510 +++
 .../Cache/Store/CacheTestParallelLoadStore.cs   |   91 +
 .../Cache/Store/CacheTestStore.cs               |  155 +
 .../Compute/AbstractTaskTest.cs                 |  217 ++
 .../Compute/ClosureTaskTest.cs                  |  390 ++
 .../Compute/ComputeApiTest.cs                   | 1281 +++++++
 .../Compute/ComputeMultithreadedTest.cs         |  269 ++
 .../Compute/FailoverTaskSelfTest.cs             |  246 ++
 .../Forked/ForkedPortableClosureTaskTest.cs     |   33 +
 .../Compute/Forked/ForkedResourceTaskTest.cs    |   33 +
 .../Forked/ForkedSerializableClosureTaskTest.cs |   33 +
 .../Compute/Forked/ForkedTaskAdapterTest.cs     |   33 +
 .../Compute/IgniteExceptionTaskSelfTest.cs      |  753 ++++
 .../Compute/PortableClosureTaskTest.cs          |  217 ++
 .../Compute/PortableTaskTest.cs                 |  253 ++
 .../Compute/ResourceTaskTest.cs                 |  568 +++
 .../Compute/SerializableClosureTaskTest.cs      |  217 ++
 .../Compute/TaskAdapterTest.cs                  |  274 ++
 .../Compute/TaskResultTest.cs                   |  437 +++
 .../Config/Cache/Store/cache-store-session.xml  |   80 +
 .../Config/Compute/compute-grid1.xml            |   90 +
 .../Config/Compute/compute-grid2.xml            |   63 +
 .../Config/Compute/compute-grid3.xml            |   52 +
 .../Config/Compute/compute-standalone.xml       |   87 +
 .../Config/Dynamic/dynamic-client.xml           |   51 +
 .../Config/Dynamic/dynamic-data-no-cfg.xml      |   47 +
 .../Config/Dynamic/dynamic-data.xml             |   65 +
 .../Config/Ignite.exe.config.test               |   41 +
 .../Config/Lifecycle/lifecycle-beans.xml        |   66 +
 .../Config/Lifecycle/lifecycle-no-beans.xml     |   44 +
 .../Config/cache-portables.xml                  |   78 +
 .../Config/cache-query-continuous.xml           |  171 +
 .../Config/cache-query.xml                      |  100 +
 .../Config/marshaller-default.xml               |   43 +
 .../Config/marshaller-invalid.xml               |   46 +
 .../Config/marshaller-portable.xml              |   43 +
 .../native-client-test-cache-affinity.xml       |   70 +
 .../native-client-test-cache-parallel-store.xml |   69 +
 .../native-client-test-cache-portables.xml      |  226 ++
 .../Config/native-client-test-cache-store.xml   |  125 +
 .../Config/native-client-test-cache.xml         |  224 ++
 .../Config/portable.xml                         |   56 +
 .../Config/start-test-grid1.xml                 |   54 +
 .../Config/start-test-grid2.xml                 |   45 +
 .../Config/start-test-grid3.xml                 |   43 +
 .../Dataload/DataStreamerTest.cs                |  592 ++++
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |  961 +++++
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |  352 ++
 .../Apache.Ignite.Core.Tests/ExecutableTest.cs  |  444 +++
 .../Apache.Ignite.Core.Tests/FutureTest.cs      |  278 ++
 .../IgniteManagerTest.cs                        |   51 +
 .../IgniteStartStopTest.cs                      |  380 ++
 .../Apache.Ignite.Core.Tests/IgnitionTest.cs    |   30 -
 .../Apache.Ignite.Core.Tests/LifecycleTest.cs   |  288 ++
 .../Apache.Ignite.Core.Tests/LoadDllTest.cs     |  243 ++
 .../Apache.Ignite.Core.Tests/MarshallerTest.cs  |   71 +
 .../Apache.Ignite.Core.Tests/MessagingTest.cs   |  646 ++++
 .../Portable/PortableApiSelfTest.cs             | 1787 ++++++++++
 .../Portable/PortableSelfTest.cs                | 2078 +++++++++++
 .../PortableConfigurationTest.cs                |  173 +
 .../Process/IIgniteProcessOutputReader.cs       |   35 +
 .../Process/IgniteProcess.cs                    |  292 ++
 .../Process/IgniteProcessConsoleOutputReader.cs |   40 +
 .../Query/ImplicitPortablePerson.cs             |   46 +
 .../Query/NoDefPortablePerson.cs                |   35 +
 .../Query/PortablePerson.cs                     |   69 +
 .../SerializationTest.cs                        |  240 ++
 .../Services/ServiceProxyTest.cs                |  741 ++++
 .../Services/ServicesAsyncWrapper.cs            |  174 +
 .../Services/ServicesTest.cs                    |  823 +++++
 .../Services/ServicesTestAsync.cs               |   33 +
 .../Apache.Ignite.Core.Tests/TestRunner.cs      |   15 +-
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |  292 ++
 .../TypeResolverTest.cs                         |  107 +
 .../platform/PlatformComputeBroadcastTask.java  |   73 +
 .../platform/PlatformComputeDecimalTask.java    |  106 +
 .../platform/PlatformComputeEchoTask.java       |  188 +
 .../ignite/platform/PlatformComputeEnum.java    |   28 +
 .../platform/PlatformComputeJavaPortable.java   |   39 +
 .../platform/PlatformComputePortable.java       |   42 +
 .../PlatformComputePortableArgTask.java         |  119 +
 .../platform/PlatformEventsWriteEventTask.java  |  146 +
 .../ignite/platform/PlatformMaxMemoryTask.java  |   57 +
 .../ignite/platform/PlatformMinMemoryTask.java  |   57 +
 .../lifecycle/PlatformJavaLifecycleBean.java    |   47 +
 .../lifecycle/PlatformJavaLifecycleTask.java    |   65 +
 parent/pom.xml                                  |    6 +
 389 files changed, 75307 insertions(+), 729 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
index 33a105b..52df2a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
@@ -61,6 +61,9 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.marshaller.MarshallerContext;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration;
+import org.apache.ignite.platform.dotnet.PlatformDotNetPortableConfiguration;
+import org.apache.ignite.platform.dotnet.PlatformDotNetPortableTypeConfiguration;
 import org.apache.ignite.portable.PortableException;
 import org.apache.ignite.portable.PortableIdMapper;
 import org.apache.ignite.portable.PortableInvalidClassException;
@@ -239,6 +242,10 @@ public class PortableContext implements Externalizable {
 
         registerPredefinedType(PortableObjectImpl.class, 200);
         registerPredefinedType(PortableMetaDataImpl.class, 201);
+
+        registerPredefinedType(PlatformDotNetConfiguration.class, 202);
+        registerPredefinedType(PlatformDotNetPortableConfiguration.class, 203);
+        registerPredefinedType(PlatformDotNetPortableTypeConfiguration.class, 204);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetConfiguration.java b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetConfiguration.java
new file mode 100644
index 0000000..80f4b26
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetConfiguration.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.platform.dotnet;
+
+import org.apache.ignite.configuration.PlatformConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.portable.PortableMarshalAware;
+import org.apache.ignite.portable.PortableRawReader;
+import org.apache.ignite.portable.PortableRawWriter;
+import org.apache.ignite.portable.PortableReader;
+import org.apache.ignite.portable.PortableWriter;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Mirror of .Net class Configuration.cs
+ */
+public class PlatformDotNetConfiguration implements PlatformConfiguration, PortableMarshalAware {
+    /** */
+    private PlatformDotNetPortableConfiguration portableCfg;
+
+    /** */
+    private List<String> assemblies;
+
+    /**
+     * Default constructor.
+     */
+    public PlatformDotNetConfiguration() {
+        // No-op.
+    }
+
+    /**
+     * Copy constructor.
+     *
+     * @param cfg Configuration to copy.
+     */
+    public PlatformDotNetConfiguration(PlatformDotNetConfiguration cfg) {
+        if (cfg.getPortableConfiguration() != null)
+            portableCfg = new PlatformDotNetPortableConfiguration(cfg.getPortableConfiguration());
+
+        if (cfg.getAssemblies() != null)
+            assemblies = new ArrayList<>(cfg.getAssemblies());
+    }
+
+    /**
+     * @return Configuration.
+     */
+    public PlatformDotNetPortableConfiguration getPortableConfiguration() {
+        return portableCfg;
+    }
+
+    /**
+     * @param portableCfg Configuration.
+     */
+    public void setPortableConfiguration(PlatformDotNetPortableConfiguration portableCfg) {
+        this.portableCfg = portableCfg;
+    }
+
+    /**
+     * @return Assemblies.
+     */
+    public List<String> getAssemblies() {
+        return assemblies;
+    }
+
+    /**
+     *
+     * @param assemblies Assemblies.
+     */
+    public void setAssemblies(List<String> assemblies) {
+        this.assemblies = assemblies;
+    }
+
+    /**
+     * @return Configuration copy.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private PlatformDotNetConfiguration copy() {
+        return new PlatformDotNetConfiguration(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writePortable(PortableWriter writer) throws PortableException {
+        PortableRawWriter rawWriter = writer.rawWriter();
+
+        rawWriter.writeObject(portableCfg);
+        rawWriter.writeCollection(assemblies);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readPortable(PortableReader reader) throws PortableException {
+        PortableRawReader rawReader = reader.rawReader();
+
+        portableCfg = rawReader.readObject();
+        assemblies = (List<String>)rawReader.<String>readCollection();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PlatformDotNetConfiguration.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableConfiguration.java b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableConfiguration.java
new file mode 100644
index 0000000..644a8e6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableConfiguration.java
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.platform.dotnet;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.portable.PortableMarshalAware;
+import org.apache.ignite.portable.PortableRawReader;
+import org.apache.ignite.portable.PortableRawWriter;
+import org.apache.ignite.portable.PortableReader;
+import org.apache.ignite.portable.PortableWriter;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+/**
+ * Mirror of .Net class PortableConfiguration.cs
+ */
+public class PlatformDotNetPortableConfiguration implements PortableMarshalAware {
+    /** Type cfgs. */
+    private Collection<PlatformDotNetPortableTypeConfiguration> typesCfg;
+
+    /** Types. */
+    private Collection<String> types;
+
+    /** Default name mapper. */
+    private String dfltNameMapper;
+
+    /** Default id mapper. */
+    private String dfltIdMapper;
+
+    /** Default serializer. */
+    private String dfltSerializer;
+
+    /** Default metadata enabled. */
+    private boolean dfltMetadataEnabled = true;
+
+    /** Whether to cache deserialized value in IGridPortableObject */
+    private boolean dfltKeepDeserialized = true;
+
+    /**
+     * Default constructor.
+     */
+    public PlatformDotNetPortableConfiguration() {
+        // No-op.
+    }
+
+    /**
+     * Copy constructor.
+     * @param cfg configuration to copy.
+     */
+    public PlatformDotNetPortableConfiguration(PlatformDotNetPortableConfiguration cfg) {
+        if (cfg.getTypesConfiguration() != null) {
+            typesCfg = new ArrayList<>();
+
+            for (PlatformDotNetPortableTypeConfiguration typeCfg : cfg.getTypesConfiguration())
+                typesCfg.add(new PlatformDotNetPortableTypeConfiguration(typeCfg));
+        }
+
+        if (cfg.getTypes() != null)
+            types = new ArrayList<>(cfg.getTypes());
+
+        dfltNameMapper = cfg.getDefaultNameMapper();
+        dfltIdMapper = cfg.getDefaultIdMapper();
+        dfltSerializer = cfg.getDefaultSerializer();
+        dfltMetadataEnabled = cfg.isDefaultMetadataEnabled();
+        dfltKeepDeserialized = cfg.isDefaultKeepDeserialized();
+    }
+
+    /**
+     * @return Type cfgs.
+     */
+    public Collection<PlatformDotNetPortableTypeConfiguration> getTypesConfiguration() {
+        return typesCfg;
+    }
+
+    /**
+     * @param typesCfg New type cfgs.
+     */
+    public void setTypesConfiguration(Collection<PlatformDotNetPortableTypeConfiguration> typesCfg) {
+        this.typesCfg = typesCfg;
+    }
+
+    /**
+     * @return Types.
+     */
+    public Collection<String> getTypes() {
+        return types;
+    }
+
+    /**
+     * @param types New types.
+     */
+    public void setTypes(Collection<String> types) {
+        this.types = types;
+    }
+
+    /**
+     * @return Default name mapper.
+     */
+    public String getDefaultNameMapper() {
+        return dfltNameMapper;
+    }
+
+    /**
+     * @param dfltNameMapper New default name mapper.
+     */
+    public void setDefaultNameMapper(String dfltNameMapper) {
+        this.dfltNameMapper = dfltNameMapper;
+    }
+
+    /**
+     * @return Default id mapper.
+     */
+    public String getDefaultIdMapper() {
+        return dfltIdMapper;
+    }
+
+    /**
+     * @param dfltIdMapper New default id mapper.
+     */
+    public void setDefaultIdMapper(String dfltIdMapper) {
+        this.dfltIdMapper = dfltIdMapper;
+    }
+
+    /**
+     * @return Default serializer.
+     */
+    public String getDefaultSerializer() {
+        return dfltSerializer;
+    }
+
+    /**
+     * @param dfltSerializer New default serializer.
+     */
+    public void setDefaultSerializer(String dfltSerializer) {
+        this.dfltSerializer = dfltSerializer;
+    }
+
+    /**
+     * Gets default metadata enabled flag. See {@link #setDefaultMetadataEnabled(boolean)} for more information.
+     *
+     * @return Default metadata enabled flag.
+     */
+    public boolean isDefaultMetadataEnabled() {
+        return dfltMetadataEnabled;
+    }
+
+    /**
+     * Sets default metadata enabled flag. When set to {@code true} all portable types will save it's metadata to
+     * cluster.
+     * <p />
+     * Can be overridden for particular type using
+     * {@link PlatformDotNetPortableTypeConfiguration#setMetadataEnabled(Boolean)}.
+     *
+     * @param dfltMetadataEnabled Default metadata enabled flag.
+     */
+    public void setDefaultMetadataEnabled(boolean dfltMetadataEnabled) {
+        this.dfltMetadataEnabled = dfltMetadataEnabled;
+    }
+
+    /**
+     * Gets default keep deserialized flag. See {@link #setDefaultKeepDeserialized(boolean)} for more information.
+     *
+     * @return  Flag indicates whether to cache deserialized value in IGridPortableObject.
+     */
+    public boolean isDefaultKeepDeserialized() {
+        return dfltKeepDeserialized;
+    }
+
+    /**
+     * Sets default keep deserialized flag.
+     * <p />
+     * Can be overridden for particular type using
+     * {@link PlatformDotNetPortableTypeConfiguration#setKeepDeserialized(Boolean)}.
+     *
+     * @param keepDeserialized Keep deserialized flag.
+     */
+    public void setDefaultKeepDeserialized(boolean keepDeserialized) {
+        this.dfltKeepDeserialized = keepDeserialized;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writePortable(PortableWriter writer) throws PortableException {
+        PortableRawWriter rawWriter = writer.rawWriter();
+
+        rawWriter.writeCollection(typesCfg);
+        rawWriter.writeCollection(types);
+        rawWriter.writeString(dfltNameMapper);
+        rawWriter.writeString(dfltIdMapper);
+        rawWriter.writeString(dfltSerializer);
+        rawWriter.writeBoolean(dfltMetadataEnabled);
+        rawWriter.writeBoolean(dfltKeepDeserialized);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readPortable(PortableReader reader) throws PortableException {
+        PortableRawReader rawReader = reader.rawReader();
+
+        typesCfg = rawReader.readCollection();
+        types = rawReader.readCollection();
+        dfltNameMapper = rawReader.readString();
+        dfltIdMapper = rawReader.readString();
+        dfltSerializer = rawReader.readString();
+        dfltMetadataEnabled = rawReader.readBoolean();
+        dfltKeepDeserialized = rawReader.readBoolean();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PlatformDotNetPortableConfiguration.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableTypeConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableTypeConfiguration.java b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableTypeConfiguration.java
new file mode 100644
index 0000000..b6fdbde
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetPortableTypeConfiguration.java
@@ -0,0 +1,248 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.platform.dotnet;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.portable.PortableMarshalAware;
+import org.apache.ignite.portable.PortableRawReader;
+import org.apache.ignite.portable.PortableRawWriter;
+import org.apache.ignite.portable.PortableReader;
+import org.apache.ignite.portable.PortableWriter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Mirror of .Net class GridDotNetPortableTypeConfiguration.cs
+ */
+public class PlatformDotNetPortableTypeConfiguration implements PortableMarshalAware {
+    /** Assembly name. */
+    private String assemblyName;
+
+    /** Type name. */
+    private String typeName;
+
+    /** Name mapper. */
+    private String nameMapper;
+
+    /** Id mapper. */
+    private String idMapper;
+
+    /** Serializer. */
+    private String serializer;
+
+    /** Affinity key field name. */
+    private String affinityKeyFieldName;
+
+    /** Metadata enabled. */
+    private Boolean metadataEnabled;
+
+    /** Whether to cache deserialized value in IGridPortableObject. */
+    private Boolean keepDeserialized;
+
+    /**
+     * Default constructor.
+     */
+    public PlatformDotNetPortableTypeConfiguration() {
+        // No-op.
+    }
+
+    /**
+     * Copy constructor.
+     * @param cfg configuration to copy.
+     */
+    public PlatformDotNetPortableTypeConfiguration(PlatformDotNetPortableTypeConfiguration cfg) {
+        assemblyName = cfg.getAssemblyName();
+        typeName = cfg.getTypeName();
+        nameMapper = cfg.getNameMapper();
+        idMapper = cfg.getIdMapper();
+        serializer = cfg.getSerializer();
+        affinityKeyFieldName = cfg.getAffinityKeyFieldName();
+        metadataEnabled = cfg.getMetadataEnabled();
+        keepDeserialized = cfg.isKeepDeserialized();
+    }
+
+    /**
+     * @return Assembly name.
+     */
+    public String getAssemblyName() {
+        return assemblyName;
+    }
+
+    /**
+     * @param assemblyName New assembly name.
+     */
+    public void setAssemblyName(String assemblyName) {
+        this.assemblyName = assemblyName;
+    }
+
+    /**
+     * @return Type name.
+     */
+    public String getTypeName() {
+        return typeName;
+    }
+
+    /**
+     * @param typeName New type name.
+     */
+    public void setTypeName(String typeName) {
+        this.typeName = typeName;
+    }
+
+    /**
+     * @return Name mapper.
+     */
+    public String getNameMapper() {
+        return nameMapper;
+    }
+
+    /**
+     * @param nameMapper New name mapper.
+     */
+    public void setNameMapper(String nameMapper) {
+        this.nameMapper = nameMapper;
+    }
+
+    /**
+     * @return Id mapper.
+     */
+    public String getIdMapper() {
+        return idMapper;
+    }
+
+    /**
+     * @param idMapper New id mapper.
+     */
+    public void setIdMapper(String idMapper) {
+        this.idMapper = idMapper;
+    }
+
+    /**
+     * @return Serializer.
+     */
+    public String getSerializer() {
+        return serializer;
+    }
+
+    /**
+     * @param serializer New serializer.
+     */
+    public void setSerializer(String serializer) {
+        this.serializer = serializer;
+    }
+
+    /**
+     * Gets metadata enabled flag. See {@link #setMetadataEnabled(Boolean)} for more information.
+     *
+     * @return Metadata enabled flag.
+     */
+    public Boolean getMetadataEnabled() {
+        return metadataEnabled;
+    }
+
+    /**
+     * Sets metadata enabled flag.
+     * <p />
+     * When set to {@code null} default value taken from
+     * {@link PlatformDotNetPortableConfiguration#isDefaultMetadataEnabled()} will be used.
+     *
+     * @param metadataEnabled New metadata enabled.
+     */
+    public void setMetadataEnabled(Boolean metadataEnabled) {
+        this.metadataEnabled = metadataEnabled;
+    }
+
+    /**
+     * @return Affinity key field name.
+     */
+    public String getAffinityKeyFieldName() {
+        return affinityKeyFieldName;
+    }
+
+    /**
+     * @param affinityKeyFieldName Affinity key field name.
+     */
+    public void setAffinityKeyFieldName(String affinityKeyFieldName) {
+        this.affinityKeyFieldName = affinityKeyFieldName;
+    }
+
+    /**
+     * Gets keep deserialized flag.
+     *
+     * @return Flag indicates whether to cache deserialized value in IGridPortableObject.
+     * @deprecated Use {@link #getKeepDeserialized()} instead.
+     */
+    @Deprecated
+    @Nullable public Boolean isKeepDeserialized() {
+        return keepDeserialized;
+    }
+
+    /**
+     * Gets keep deserialized flag. See {@link #setKeepDeserialized(Boolean)} for more information.
+     *
+     * @return Flag indicates whether to cache deserialized value in IGridPortableObject.
+     */
+    @Nullable public Boolean getKeepDeserialized() {
+        return keepDeserialized;
+    }
+
+    /**
+     * Sets keep deserialized flag.
+     * <p />
+     * When set to {@code null} default value taken from
+     * {@link PlatformDotNetPortableConfiguration#isDefaultKeepDeserialized()} will be used.
+     *
+     * @param keepDeserialized Keep deserialized flag.
+     */
+    public void setKeepDeserialized(@Nullable Boolean keepDeserialized) {
+        this.keepDeserialized = keepDeserialized;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writePortable(PortableWriter writer) throws PortableException {
+        PortableRawWriter rawWriter = writer.rawWriter();
+
+        rawWriter.writeString(assemblyName);
+        rawWriter.writeString(typeName);
+        rawWriter.writeString(nameMapper);
+        rawWriter.writeString(idMapper);
+        rawWriter.writeString(serializer);
+        rawWriter.writeString(affinityKeyFieldName);
+        rawWriter.writeObject(metadataEnabled);
+        rawWriter.writeObject(keepDeserialized);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readPortable(PortableReader reader) throws PortableException {
+        PortableRawReader rawReader = reader.rawReader();
+
+        assemblyName = rawReader.readString();
+        typeName = rawReader.readString();
+        nameMapper = rawReader.readString();
+        idMapper = rawReader.readString();
+        serializer = rawReader.readString();
+        affinityKeyFieldName = rawReader.readString();
+        metadataEnabled = rawReader.readObject();
+        keepDeserialized = rawReader.readObject();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PlatformDotNetPortableTypeConfiguration.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/pom.xml
----------------------------------------------------------------------
diff --git a/modules/platform/pom.xml b/modules/platform/pom.xml
index 56e1821..ac34436 100644
--- a/modules/platform/pom.xml
+++ b/modules/platform/pom.xml
@@ -63,4 +63,18 @@
         </dependency>
     </dependencies>
 
+    <build>
+        <resources>
+            <resource>
+                <directory>src/main/java</directory>
+                <excludes>
+                    <exclude>**/*.java</exclude>
+                </excludes>
+            </resource>
+
+            <resource>
+                <directory>src/main/resources</directory>
+            </resource>
+        </resources>
+    </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 12e335a..52f9b48 100644
--- a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -49,11 +49,121 @@
     <Reference Include="System.Core" />
   </ItemGroup>
   <ItemGroup>
+    <Compile Include="Cache\CacheAtomicUpdateTimeoutException.cs" />
+    <Compile Include="Cache\CacheEntryProcessorException.cs" />
+    <Compile Include="Cache\CacheException.cs" />
+    <Compile Include="Cache\CachePartialUpdateException.cs" />
+    <Compile Include="Cache\CachePeekMode.cs" />
+    <Compile Include="Cache\Event\CacheEntryEventType.cs" />
+    <Compile Include="Cache\Event\ICacheEntryEvent.cs" />
+    <Compile Include="Cache\Event\ICacheEntryEventFilter.cs" />
+    <Compile Include="Cache\Event\ICacheEntryEventListener.cs" />
+    <Compile Include="Cache\Expiry\ExpiryPolicy.cs" />
+    <Compile Include="Cache\Expiry\IExpiryPolicy.cs" />
+    <Compile Include="Cache\ICache.cs" />
+    <Compile Include="Cache\ICacheAffinity.cs" />
+    <Compile Include="Cache\ICacheEntry.cs" />
+    <Compile Include="Cache\ICacheEntryFilter.cs" />
+    <Compile Include="Cache\ICacheEntryProcessor.cs" />
+    <Compile Include="Cache\ICacheEntryProcessorResult.cs" />
+    <Compile Include="Cache\ICacheLock.cs" />
+    <Compile Include="Cache\ICacheMetrics.cs" />
+    <Compile Include="Cache\IMutableCacheEntry.cs" />
+    <Compile Include="Cache\Query\Continuous\ContinuousQuery.cs" />
+    <Compile Include="Cache\Query\Continuous\IContinuousQueryHandle.cs" />
+    <Compile Include="Cache\Query\IQueryCursor.cs" />
+    <Compile Include="Cache\Query\QueryBase.cs" />
+    <Compile Include="Cache\Query\ScanQuery.cs" />
+    <Compile Include="Cache\Query\SqlFieldsQuery.cs" />
+    <Compile Include="Cache\Query\SqlQuery.cs" />
+    <Compile Include="Cache\Query\TextQuery.cs" />
+    <Compile Include="Cache\Store\CacheParallelLoadStoreAdapter.cs" />
+    <Compile Include="Cache\Store\CacheStoreAdapter.cs" />
+    <Compile Include="Cache\Store\CacheStoreException.cs" />
+    <Compile Include="Cache\Store\ICacheStore.cs" />
+    <Compile Include="Cache\Store\ICacheStoreSession.cs" />
+    <Compile Include="Cluster\ClusterGroupEmptyException.cs" />
+    <Compile Include="Cluster\ClusterTopologyException.cs" />
+    <Compile Include="Cluster\ICluster.cs" />
+    <Compile Include="Cluster\IClusterGroup.cs" />
+    <Compile Include="Cluster\IClusterMetrics.cs" />
+    <Compile Include="Cluster\IClusterNode.cs" />
+    <Compile Include="Cluster\IClusterNodeFilter.cs" />
     <Compile Include="Common\IgniteException.cs" />
     <Compile Include="Common\IAsyncSupport.cs" />
     <Compile Include="Common\IFuture.cs" />
+    <Compile Include="Common\IgniteGuid.cs" />
+    <Compile Include="Compute\ComputeExecutionRejectedException.cs" />
+    <Compile Include="Compute\ComputeJobAdapter.cs" />
+    <Compile Include="Compute\ComputeJobFailoverException.cs" />
+    <Compile Include="Compute\ComputeJobResultPolicy.cs" />
+    <Compile Include="Compute\ComputeTaskAdapter.cs" />
+    <Compile Include="Compute\ComputeTaskCancelledException.cs" />
+    <Compile Include="Compute\ComputeTaskNoResultCacheAttribute.cs" />
+    <Compile Include="Compute\ComputeTaskSplitAdapter.cs" />
+    <Compile Include="Compute\ComputeTaskTimeoutException.cs" />
+    <Compile Include="Compute\ComputeUserUndeclaredException.cs" />
+    <Compile Include="Compute\ICompute.cs" />
+    <Compile Include="Compute\IComputeFunc.cs" />
+    <Compile Include="Compute\IComputeJob.cs" />
+    <Compile Include="Compute\IComputeJobResult.cs" />
+    <Compile Include="Compute\IComputeReducer.cs" />
+    <Compile Include="Compute\IComputeTask.cs" />
+    <Compile Include="Datastream\IDataStreamer.cs" />
+    <Compile Include="Datastream\IStreamReceiver.cs" />
+    <Compile Include="Datastream\StreamTransformer.cs" />
+    <Compile Include="Datastream\StreamVisitor.cs" />
+    <Compile Include="Events\CacheEvent.cs" />
+    <Compile Include="Events\CacheQueryExecutedEvent.cs" />
+    <Compile Include="Events\CacheQueryReadEvent.cs" />
+    <Compile Include="Events\CacheRebalancingEvent.cs" />
+    <Compile Include="Events\CheckpointEvent.cs" />
+    <Compile Include="Events\DiscoveryEvent.cs" />
+    <Compile Include="Events\EventBase.cs" />
+    <Compile Include="Events\EventReader.cs" />
+    <Compile Include="Events\EventType.cs" />
+    <Compile Include="Events\IEvent.cs" />
+    <Compile Include="Events\IEventFilter.cs" />
+    <Compile Include="Events\IEvents.cs" />
+    <Compile Include="Events\JobEvent.cs" />
+    <Compile Include="Events\SwapSpaceEvent.cs" />
+    <Compile Include="Events\TaskEvent.cs" />
+    <Compile Include="IgniteConfiguration.cs" />
     <Compile Include="Ignition.cs" />
     <Compile Include="Common\AsyncSupportedAttribute.cs" />
+    <Compile Include="IIgnite.cs" />
+    <Compile Include="Impl\Cache\CacheAffinityImpl.cs" />
+    <Compile Include="Impl\Cache\CacheEntry.cs" />
+    <Compile Include="Impl\Cache\CacheEntryFilterHolder.cs" />
+    <Compile Include="Impl\Cache\CacheEntryProcessorHolder.cs" />
+    <Compile Include="Impl\Cache\CacheEntryProcessorResult.cs" />
+    <Compile Include="Impl\Cache\CacheEntryProcessorResultHolder.cs" />
+    <Compile Include="Impl\Cache\CacheEnumerable.cs" />
+    <Compile Include="Impl\Cache\CacheEnumerator.cs" />
+    <Compile Include="Impl\Cache\CacheEnumeratorProxy.cs" />
+    <Compile Include="Impl\Cache\CacheImpl.cs" />
+    <Compile Include="Impl\Cache\CacheLock.cs" />
+    <Compile Include="Impl\Cache\CacheMetricsImpl.cs" />
+    <Compile Include="Impl\Cache\CacheOp.cs" />
+    <Compile Include="Impl\Cache\CacheProxyImpl.cs" />
+    <Compile Include="Impl\Cache\Event\CacheEntryCreateEvent.cs" />
+    <Compile Include="Impl\Cache\Event\CacheEntryRemoveEvent.cs" />
+    <Compile Include="Impl\Cache\Event\CacheEntryUpdateEvent.cs" />
+    <Compile Include="Impl\Cache\MutableCacheEntry.cs" />
+    <Compile Include="Impl\Cache\Query\AbstractQueryCursor.cs" />
+    <Compile Include="Impl\Cache\Query\Continuous\ContinuousQueryFilter.cs" />
+    <Compile Include="Impl\Cache\Query\Continuous\ContinuousQueryFilterHolder.cs" />
+    <Compile Include="Impl\Cache\Query\Continuous\ContinuousQueryHandleImpl.cs" />
+    <Compile Include="Impl\Cache\Query\Continuous\ContinuousQueryUtils.cs" />
+    <Compile Include="Impl\Cache\Query\FieldsQueryCursor.cs" />
+    <Compile Include="Impl\Cache\Query\QueryCursor.cs" />
+    <Compile Include="Impl\Cache\Store\CacheStore.cs" />
+    <Compile Include="Impl\Cache\Store\CacheStoreSession.cs" />
+    <Compile Include="Impl\Cache\Store\CacheStoreSessionProxy.cs" />
+    <Compile Include="Impl\Cluster\ClusterGroupImpl.cs" />
+    <Compile Include="Impl\Cluster\ClusterMetricsImpl.cs" />
+    <Compile Include="Impl\Cluster\ClusterNodeImpl.cs" />
+    <Compile Include="Impl\Cluster\IClusterGroupEx.cs" />
     <Compile Include="Impl\Collections\CollectionExtensions.cs" />
     <Compile Include="Impl\Collections\MultiValueDictionary.cs" />
     <Compile Include="Impl\Collections\ReadOnlyCollection.cs" />
@@ -62,16 +172,60 @@
     <Compile Include="Impl\Common\CompletedAsyncResult.cs" />
     <Compile Include="Impl\Common\CopyOnWriteConcurrentDictionary.cs" />
     <Compile Include="Impl\Common\DelegateConverter.cs" />
+    <Compile Include="Impl\Common\DelegateTypeDescriptor.cs" />
     <Compile Include="Impl\Common\Future.cs" />
+    <Compile Include="Impl\Common\FutureConverter.cs" />
     <Compile Include="Impl\Common\FutureType.cs" />
-    <Compile Include="Impl\Common\GridArgumentCheck.cs" />
+    <Compile Include="Impl\Common\IgniteArgumentCheck.cs" />
     <Compile Include="Impl\Common\IFutureConverter.cs" />
     <Compile Include="Impl\Common\IFutureInternal.cs" />
     <Compile Include="Impl\Common\LoadedAssembliesResolver.cs" />
+    <Compile Include="Impl\Common\PortableResultWrapper.cs" />
     <Compile Include="Impl\Common\TypeCaster.cs" />
+    <Compile Include="Impl\Compute\Closure\ComputeAbstractClosureTask.cs" />
+    <Compile Include="Impl\Compute\Closure\ComputeActionJob.cs" />
+    <Compile Include="Impl\Compute\Closure\ComputeFuncJob.cs" />
+    <Compile Include="Impl\Compute\Closure\ComputeMultiClosureTask.cs" />
+    <Compile Include="Impl\Compute\Closure\ComputeOutFuncJob.cs" />
+    <Compile Include="Impl\Compute\Closure\ComputeReducingClosureTask.cs" />
+    <Compile Include="Impl\Compute\Closure\ComputeSingleClosureTask.cs" />
+    <Compile Include="Impl\Compute\Closure\IComputeResourceInjector.cs" />
+    <Compile Include="Impl\Compute\Compute.cs" />
+    <Compile Include="Impl\Compute\ComputeAsync.cs" />
+    <Compile Include="Impl\Compute\ComputeFunc.cs" />
+    <Compile Include="Impl\Compute\ComputeImpl.cs" />
+    <Compile Include="Impl\Compute\ComputeJob.cs" />
+    <Compile Include="Impl\Compute\ComputeJobHolder.cs" />
+    <Compile Include="Impl\Compute\ComputeJobResultGenericWrapper.cs" />
+    <Compile Include="Impl\Compute\ComputeJobResultImpl.cs" />
+    <Compile Include="Impl\Compute\ComputeOutFunc.cs" />
+    <Compile Include="Impl\Compute\ComputeTaskHolder.cs" />
+    <Compile Include="Impl\Datastream\DataStreamerBatch.cs" />
+    <Compile Include="Impl\Datastream\DataStreamerEntry.cs" />
+    <Compile Include="Impl\Datastream\DataStreamerImpl.cs" />
+    <Compile Include="Impl\Datastream\DataStreamerRemoveEntry.cs" />
+    <Compile Include="Impl\Datastream\StreamReceiverHolder.cs" />
+    <Compile Include="Impl\Events\Events.cs" />
+    <Compile Include="Impl\Events\EventsAsync.cs" />
+    <Compile Include="Impl\Events\RemoteListenEventFilter.cs" />
+    <Compile Include="Impl\ExceptionUtils.cs" />
+    <Compile Include="Impl\IgniteConfigurationEx.cs" />
+    <Compile Include="Impl\Ignite.cs" />
+    <Compile Include="Impl\IgniteManager.cs" />
+    <Compile Include="Impl\IgniteProxy.cs" />
+    <Compile Include="Impl\PlatformTarget.cs" />
+    <Compile Include="Impl\IgniteUtils.cs" />
     <Compile Include="Impl\Handle\Handle.cs" />
     <Compile Include="Impl\Handle\HandleRegistry.cs" />
     <Compile Include="Impl\Handle\IHandle.cs" />
+    <Compile Include="Impl\IInteropCallback.cs" />
+    <Compile Include="Impl\InteropExceptionHolder.cs" />
+    <Compile Include="Impl\Interop\InteropDotNetConfiguration.cs" />
+    <Compile Include="Impl\Interop\InteropDotNetPortableConfiguration.cs" />
+    <Compile Include="Impl\Interop\InteropDotNetPortableTypeConfiguration.cs" />
+    <Compile Include="Impl\LifecycleBeanHolder.cs" />
+    <Compile Include="Impl\Memory\InteropExternalMemory.cs" />
+    <Compile Include="Impl\Memory\InteropMemoryUtils.cs" />
     <Compile Include="Impl\Memory\IPlatformMemory.cs" />
     <Compile Include="Impl\Memory\PlatformBigEndianMemoryStream.cs" />
     <Compile Include="Impl\Memory\PlatformMemory.cs" />
@@ -82,10 +236,135 @@
     <Compile Include="Impl\Memory\PlatformPooledMemory.cs" />
     <Compile Include="Impl\Memory\PlatformRawMemory.cs" />
     <Compile Include="Impl\Memory\PlatformUnpooledMemory.cs" />
+    <Compile Include="Impl\Messaging\MessageFilterHolder.cs" />
+    <Compile Include="Impl\Messaging\Messaging.cs" />
+    <Compile Include="Impl\Messaging\MessagingAsync.cs" />
+    <Compile Include="Impl\NativeMethods.cs" />
     <Compile Include="Impl\Portable\IO\IPortableStream.cs" />
+    <Compile Include="Impl\Portable\IO\PortableAbstractStream.cs" />
+    <Compile Include="Impl\Portable\IO\PortableHeapStream.cs" />
+    <Compile Include="Impl\Portable\IO\PortableStreamAdapter.cs" />
+    <Compile Include="Impl\Portable\IPortableSystemTypeSerializer.cs" />
+    <Compile Include="Impl\Portable\IPortableTypeDescriptor.cs" />
+    <Compile Include="Impl\Portable\IPortableWriteAware.cs" />
+    <Compile Include="Impl\Portable\Metadata\IPortableMetadataHandler.cs" />
+    <Compile Include="Impl\Portable\Metadata\PortableHashsetMetadataHandler.cs" />
+    <Compile Include="Impl\Portable\Metadata\PortableMetadataHolder.cs" />
+    <Compile Include="Impl\Portable\Metadata\PortableMetadataImpl.cs" />
+    <Compile Include="Impl\Portable\PortableBuilderField.cs" />
+    <Compile Include="Impl\Portable\PortableBuilderImpl.cs" />
+    <Compile Include="Impl\Portable\PortableCollectionInfo.cs" />
+    <Compile Include="Impl\Portable\PortableFullTypeDescriptor.cs" />
+    <Compile Include="Impl\Portable\PortableHandleDictionary.cs" />
+    <Compile Include="Impl\Portable\PortableMarshalAwareSerializer.cs" />
+    <Compile Include="Impl\Portable\PortableMarshaller.cs" />
+    <Compile Include="Impl\Portable\PortableMode.cs" />
+    <Compile Include="Impl\Portable\PortableObjectHandle.cs" />
+    <Compile Include="Impl\Portable\PortableOrSerializableObjectHolder.cs" />
+    <Compile Include="Impl\Portable\PortableReaderHandleDictionary.cs" />
+    <Compile Include="Impl\Portable\PortableReaderImpl.cs" />
+    <Compile Include="Impl\Portable\PortableReflectiveRoutines.cs" />
+    <Compile Include="Impl\Portable\PortableReflectiveSerializer.cs" />
+    <Compile Include="Impl\Portable\PortablesImpl.cs" />
+    <Compile Include="Impl\Portable\PortableSurrogateTypeDescriptor.cs" />
+    <Compile Include="Impl\Portable\PortableSystemHandlers.cs" />
+    <Compile Include="Impl\Portable\PortableSystemTypeSerializer.cs" />
+    <Compile Include="Impl\Portable\PortableUserObject.cs" />
+    <Compile Include="Impl\Portable\PortableUtils.cs" />
+    <Compile Include="Impl\Portable\PortableWriterImpl.cs" />
+    <Compile Include="Impl\Portable\SerializableObjectHolder.cs" />
+    <Compile Include="Impl\Portable\TypeResolver.cs" />
+    <Compile Include="Impl\Resource\IResourceInjector.cs" />
+    <Compile Include="Impl\Resource\ResourceFieldInjector.cs" />
+    <Compile Include="Impl\Resource\ResourceMethodInjector.cs" />
+    <Compile Include="Impl\Resource\ResourceProcessor.cs" />
+    <Compile Include="Impl\Resource\ResourcePropertyInjector.cs" />
+    <Compile Include="Impl\Resource\ResourceTypeDescriptor.cs" />
+    <Compile Include="Impl\Services\ServiceContext.cs" />
+    <Compile Include="Impl\Services\ServiceDescriptor.cs" />
+    <Compile Include="Impl\Services\ServiceProxy.cs" />
+    <Compile Include="Impl\Services\ServiceProxyInvoker.cs" />
+    <Compile Include="Impl\Services\ServiceProxySerializer.cs" />
+    <Compile Include="Impl\Services\Services.cs" />
+    <Compile Include="Impl\Services\ServicesAsync.cs" />
+    <Compile Include="Impl\Transactions\AsyncTransaction.cs" />
+    <Compile Include="Impl\Transactions\Transaction.cs" />
+    <Compile Include="Impl\Transactions\TransactionImpl.cs" />
+    <Compile Include="Impl\Transactions\TransactionMetricsImpl.cs" />
+    <Compile Include="Impl\Transactions\TransactionsImpl.cs" />
+    <Compile Include="Impl\Unmanaged\IUnmanagedTarget.cs" />
+    <Compile Include="Impl\Unmanaged\UnmanagedCallbackHandlers.cs" />
+    <Compile Include="Impl\Unmanaged\UnmanagedCallbacks.cs" />
+    <Compile Include="Impl\Unmanaged\UnmanagedContext.cs" />
+    <Compile Include="Impl\Unmanaged\UnmanagedNonReleaseableTarget.cs" />
+    <Compile Include="Impl\Unmanaged\UnmanagedTarget.cs" />
+    <Compile Include="Impl\Unmanaged\UnmanagedUtils.cs" />
+    <Compile Include="Lifecycle\ILifecycleBean.cs" />
+    <Compile Include="Lifecycle\LifecycleEventType.cs" />
+    <Compile Include="Messaging\IMessageFilter.cs" />
+    <Compile Include="Messaging\IMessaging.cs" />
+    <Compile Include="Portable\IPortableBuilder.cs" />
+    <Compile Include="Portable\IPortableIdMapper.cs" />
+    <Compile Include="Portable\IPortableMarshalAware.cs" />
+    <Compile Include="Portable\IPortableMetadata.cs" />
+    <Compile Include="Portable\IPortableNameMapper.cs" />
+    <Compile Include="Portable\IPortableObject.cs" />
+    <Compile Include="Portable\IPortableRawReader.cs" />
+    <Compile Include="Portable\IPortableRawWriter.cs" />
+    <Compile Include="Portable\IPortableReader.cs" />
+    <Compile Include="Portable\IPortables.cs" />
+    <Compile Include="Portable\IPortableSerializer.cs" />
+    <Compile Include="Portable\IPortableWriter.cs" />
+    <Compile Include="Portable\PortableConfiguration.cs" />
+    <Compile Include="Portable\PortableException.cs" />
+    <Compile Include="Portable\PortableTypeConfiguration.cs" />
+    <Compile Include="Portable\PortableTypeNames.cs" />
     <Compile Include="Properties\AssemblyInfo.cs" />
+    <Compile Include="Resource\InstanceResourceAttribute.cs" />
+    <Compile Include="Resource\StoreSessionResourceAttribute.cs" />
+    <Compile Include="Services\IService.cs" />
+    <Compile Include="Services\IServiceContext.cs" />
+    <Compile Include="Services\IServiceDescriptor.cs" />
+    <Compile Include="Services\IServices.cs" />
+    <Compile Include="Services\ServiceConfiguration.cs" />
+    <Compile Include="Services\ServiceInvocationException.cs" />
+    <Compile Include="Transactions\ITransaction.cs" />
+    <Compile Include="Transactions\ITransactionMetrics.cs" />
+    <Compile Include="Transactions\ITransactions.cs" />
+    <Compile Include="Transactions\TransactionConcurrency.cs" />
+    <Compile Include="Transactions\TransactionHeuristicException.cs" />
+    <Compile Include="Transactions\TransactionIsolation.cs" />
+    <Compile Include="Transactions\TransactionOptimisticException.cs" />
+    <Compile Include="Transactions\TransactionRollbackException.cs" />
+    <Compile Include="Transactions\TransactionState.cs" />
+    <Compile Include="Transactions\TransactionTimeoutException.cs" />
+  </ItemGroup>
+  <ItemGroup>
+    <ProjectReference Include="..\..\cpp\common\project\vs\common.vcxproj">
+      <Project>{4f7e4917-4612-4b96-9838-025711ade391}</Project>
+      <Name>common</Name>
+    </ProjectReference>
+  </ItemGroup>
+  <ItemGroup Condition="'$(Configuration)|$(Platform)' == 'Debug|x64'">
+    <EmbeddedResource Include="..\x64\Debug\ignite.common.dll">
+      <Link>resources\debug\x64\ignite.common.dll</Link>
+    </EmbeddedResource>
+  </ItemGroup>
+  <ItemGroup Condition="'$(Configuration)|$(Platform)' == 'Release|x64'">
+    <EmbeddedResource Include="..\x64\Release\ignite.common.dll">
+      <Link>resources\release\x64\ignite.common.dll</Link>
+    </EmbeddedResource>
+  </ItemGroup>
+  <ItemGroup Condition="'$(Configuration)|$(Platform)' == 'Debug|x86'">
+    <EmbeddedResource Include="..\Win32\Debug\ignite.common.dll">
+      <Link>resources\debug\x86\ignite.common.dll</Link>
+    </EmbeddedResource>
+  </ItemGroup>
+  <ItemGroup Condition="'$(Configuration)|$(Platform)' == 'Release|x86'">
+    <EmbeddedResource Include="..\Win32\Release\ignite.common.dll">
+      <Link>resources\release\x86\ignite.common.dll</Link>
+    </EmbeddedResource>
   </ItemGroup>
-  <ItemGroup />
   <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
   <!-- To modify your build process, add your task inside one of the targets below and uncomment it. 
        Other similar extension points exist, see Microsoft.Common.targets.

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheAtomicUpdateTimeoutException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheAtomicUpdateTimeoutException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheAtomicUpdateTimeoutException.cs
new file mode 100644
index 0000000..f0b5987
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheAtomicUpdateTimeoutException.cs
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Cache
+{
+    using System;
+    using System.Runtime.Serialization;
+
+    /// <summary>
+    /// Indicates atomic operation timeout.
+    /// </summary>
+    [Serializable]
+    public class CacheAtomicUpdateTimeoutException : CacheException
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheAtomicUpdateTimeoutException"/> class.
+        /// </summary>
+        public CacheAtomicUpdateTimeoutException()
+        {
+            // No-op.
+        }
+
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheAtomicUpdateTimeoutException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public CacheAtomicUpdateTimeoutException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheAtomicUpdateTimeoutException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public CacheAtomicUpdateTimeoutException(string message, Exception cause) : base(message, cause)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheAtomicUpdateTimeoutException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected CacheAtomicUpdateTimeoutException(SerializationInfo info, StreamingContext ctx) : base(info, ctx)
+        {
+            // No-op.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheEntryProcessorException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheEntryProcessorException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheEntryProcessorException.cs
new file mode 100644
index 0000000..341c713
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheEntryProcessorException.cs
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Cache
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// An exception to indicate a problem occurred attempting to execute an 
+    /// <see cref="ICacheEntryProcessor{K, V, A, R}"/> against an entry.
+    /// </summary>
+    [Serializable]
+    public class CacheEntryProcessorException : IgniteException
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheEntryProcessorException"/> class.
+        /// </summary>
+        public CacheEntryProcessorException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheEntryProcessorException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public CacheEntryProcessorException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheEntryProcessorException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public CacheEntryProcessorException(string message, Exception cause)
+            : base(message, cause)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheEntryProcessorException"/> class.
+        /// </summary>
+        /// <param name="innerException">The inner exception.</param>
+        public CacheEntryProcessorException(Exception innerException)
+            : base("Error occurred in CacheEntryProcessor, see InnerException for details.", innerException)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheEntryProcessorException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected CacheEntryProcessorException(SerializationInfo info, StreamingContext ctx) : base(info, ctx)
+        {
+            // No-op.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheException.cs
new file mode 100644
index 0000000..c00f115
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CacheException.cs
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Cache
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Indicates an error during Cache operation.
+    /// </summary>
+    [Serializable]
+    public class CacheException : IgniteException
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheException"/> class.
+        /// </summary>
+        public CacheException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public CacheException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public CacheException(string message, Exception cause) : base(message, cause)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CacheException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected CacheException(SerializationInfo info, StreamingContext ctx)
+            : base(info, ctx)
+        {
+            // No-op.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs
new file mode 100644
index 0000000..b3ed537
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Cache
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Linq;
+    using System.Runtime.Serialization;
+
+    /// <summary>
+    /// Exception thrown from non-transactional cache in case when update succeeded only partially.
+    /// </summary>
+    [Serializable]
+    public class CachePartialUpdateException : CacheException
+    {
+        /** Serializer key. */
+        private const string KeyFailedKeys = "FailedKeys";
+
+        /** Failed keys. */
+        private readonly IList<object> _failedKeys;
+
+        /** Failed keys exception. */
+        private readonly Exception _failedKeysException;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CachePartialUpdateException"/> class.
+        /// </summary>
+        public CachePartialUpdateException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CachePartialUpdateException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public CachePartialUpdateException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CachePartialUpdateException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected CachePartialUpdateException(SerializationInfo info, StreamingContext ctx)
+            : base(info, ctx)
+        {
+            _failedKeys = (IList<object>) info.GetValue(KeyFailedKeys, typeof (IList<object>));
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="msg">Exception message.</param>
+        /// <param name="failedKeysException">Exception occurred during failed keys read/write.</param>
+        public CachePartialUpdateException(string msg, Exception failedKeysException) : this(msg, null, failedKeysException)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="msg">Exception message.</param>
+        /// <param name="failedKeys">Failed keys.</param>
+        public CachePartialUpdateException(string msg, IList<object> failedKeys) : this(msg, failedKeys, null)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Constructor.
+        /// </summary>
+        /// <param name="msg">Exception message.</param>
+        /// <param name="failedKeys">Failed keys.</param>
+        /// <param name="failedKeysException">Exception occurred during failed keys read/write.</param>
+        private CachePartialUpdateException(string msg, IList<object> failedKeys, Exception failedKeysException) : base(msg)
+        {
+            _failedKeys = failedKeys;
+            _failedKeysException = failedKeysException;
+        }
+
+        /// <summary>
+        /// Gets the failed keys.
+        /// </summary>
+        public IEnumerable<T> GetFailedKeys<T>()
+        {
+            if (_failedKeysException != null)
+                throw _failedKeysException;
+            
+            return _failedKeys == null ? null : _failedKeys.Cast<T>();
+        }
+
+        /** <inheritdoc /> */
+        public override void GetObjectData(SerializationInfo info, StreamingContext context)
+        {
+            info.AddValue(KeyFailedKeys, _failedKeys);
+
+            base.GetObjectData(info, context);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CachePeekMode.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CachePeekMode.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CachePeekMode.cs
new file mode 100644
index 0000000..0a089ad
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/CachePeekMode.cs
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Cache
+{
+    using System;
+    using System.Diagnostics.CodeAnalysis;
+
+    /// <summary>
+    /// Enumeration of all supported cache peek modes.
+    /// </summary>
+    [Flags]
+    [SuppressMessage("Microsoft.Naming", "CA1714:FlagsEnumsShouldHavePluralNames")]
+    public enum CachePeekMode
+    {
+        /// <summary>
+        /// Peeks into all available cache storages.
+        /// </summary>
+        All = 0x01,
+
+        /// <summary>
+        /// Peek into near cache only (don't peek into partitioned cache).
+        /// In case of LOCAL cache, behaves as <see cref="All"/> mode.
+        /// </summary>
+        Near = 0x02,
+
+        /// <summary>
+        /// Peek value from primary copy of partitioned cache only (skip near cache).
+        /// In case of LOCAL cache, behaves as <see cref="All"/> mode.
+        /// </summary>
+        Primary = 0x04,
+
+        /// <summary>
+        /// Peek value from backup copies of partitioned cache only (skip near cache).
+        /// In case of LOCAL cache, behaves as <see cref="All"/> mode.
+        /// </summary>
+        Backup = 0x08,
+
+        /// <summary>
+        /// Peeks value from the on-heap storage only.
+        /// </summary>
+        Onheap = 0x10,
+
+        /// <summary>
+        /// Peeks value from the off-heap storage only, without loading off-heap value into cache.
+        /// </summary>
+        Offheap = 0x20,
+
+        /// <summary>
+        /// Peeks value from the swap storage only, without loading swapped value into cache.
+        /// </summary>
+        Swap = 0x40
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/CacheEntryEventType.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/CacheEntryEventType.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/CacheEntryEventType.cs
new file mode 100644
index 0000000..8339257
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/CacheEntryEventType.cs
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Cache.Event
+{
+    /// <summary>
+    /// Cache event type.
+    /// </summary>
+    public enum CacheEntryEventType
+    {
+        /// <summary>
+        /// An event type indicating that the cache entry was created.
+        /// </summary>
+        Created,
+
+        /// <summary>
+        /// An event type indicating that the cache entry was updated. i.e. a previous
+        /// mapping existed.
+        /// </summary>
+        Updated,
+
+        /// <summary>
+        /// An event type indicating that the cache entry was removed.
+        /// </summary>
+        Removed
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEvent.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEvent.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEvent.cs
new file mode 100644
index 0000000..9c2665e
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEvent.cs
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Cache.Event
+{
+    /// <summary>
+    /// Cache entry event.
+    /// </summary>
+    public interface ICacheEntryEvent<TK, TV> : ICacheEntry<TK, TV>
+    {
+        /// <summary>
+        /// Event type.
+        /// </summary>
+        CacheEntryEventType EventType { get; }
+
+        /// <summary>
+        /// Gets old the value.
+        /// </summary>
+        TV OldValue { get; }
+
+        /// <summary>
+        /// Whether old value exists.
+        /// </summary>
+        bool HasOldValue { get; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cec202c/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEventFilter.cs
----------------------------------------------------------------------
diff --git a/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEventFilter.cs b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEventFilter.cs
new file mode 100644
index 0000000..98f5c5a
--- /dev/null
+++ b/modules/platform/src/main/dotnet/Apache.Ignite.Core/Cache/Event/ICacheEntryEventFilter.cs
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Cache.Event
+{
+    /// <summary>
+    /// Cache entry event filter.
+    /// </summary>
+    public interface ICacheEntryEventFilter<TK, TV>
+    {
+        /// <summary>
+        /// Evaluates cache entry event.
+        /// </summary>
+        /// <param name="evt">Event.</param>
+        bool Evaluate(ICacheEntryEvent<TK, TV> evt);
+    }
+}