You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2015/10/29 12:53:42 UTC

[48/50] ignite git commit: Merge branch 'master' into ignite-1282

Merge branch 'master' into ignite-1282

Conflicts:
	assembly/release-fabric.xml
	modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
	modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java


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

Branch: refs/heads/ignite-1770
Commit: e4c6a9081f032b48b2cf8fcafd460b401f8649c3
Parents: 54c44b8 303def3
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Oct 29 11:57:03 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Oct 29 11:57:03 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |    5 +-
 assembly/dependencies-fabric-lgpl.xml           |  169 +
 assembly/dependencies-fabric.xml                |    3 +
 assembly/release-fabric-base.xml                |  107 +
 assembly/release-fabric-lgpl.xml                |   41 +-
 assembly/release-fabric.xml                     |  177 +-
 assembly/release-hadoop-lgpl.xml                |   39 -
 config/fabric-lgpl/default-config.xml           |   29 +
 examples-lgpl/README.txt                        |   27 -
 examples-lgpl/config/example-cache.xml          |   73 -
 examples-lgpl/config/example-ignite.xml         |   83 -
 examples-lgpl/config/hibernate/README.txt       |    8 -
 .../hibernate/example-hibernate-L2-cache.xml    |   64 -
 examples-lgpl/pom-standalone.xml                |  186 -
 examples-lgpl/pom.xml                           |  128 -
 .../hibernate/HibernateL2CacheExample.java      |  245 -
 .../examples/datagrid/hibernate/Post.java       |  130 -
 .../examples/datagrid/hibernate/User.java       |  154 -
 .../datagrid/hibernate/package-info.java        |   22 -
 .../hibernate/CacheHibernatePersonStore.java    |  122 -
 .../hibernate/CacheHibernateStoreExample.java   |  151 -
 .../datagrid/store/hibernate/Person.hbm.xml     |   34 -
 .../datagrid/store/hibernate/hibernate.cfg.xml  |   41 -
 .../datagrid/store/hibernate/package-info.java  |   22 -
 .../misc/schedule/ComputeScheduleExample.java   |   82 -
 .../examples/misc/schedule/package-info.java    |   22 -
 .../misc/schedule/ComputeScheduleExample.java   |   68 -
 .../java8/misc/schedule/package-info.java       |   22 -
 .../ignite/examples/java8/package-info.java     |   23 -
 .../scalar/examples/ScalarScheduleExample.scala |   66 -
 ...ibernateL2CacheExampleMultiNodeSelfTest.java |   31 -
 .../HibernateL2CacheExampleSelfTest.java        |   33 -
 .../IgniteLgplExamplesSelfTestSuite.java        |   48 -
 ...ibernateL2CacheExampleMultiNodeSelfTest.java |   29 -
 .../HibernateL2CacheExampleSelfTest.java        |   37 -
 .../IgniteLgplExamplesJ8SelfTestSuite.java      |   46 -
 .../ScalarLgplExamplesMultiNodeSelfTest.scala   |   33 -
 .../examples/ScalarLgplExamplesSelfTest.scala   |   36 -
 .../ScalarLgplExamplesSelfTestSuite.scala       |   37 -
 examples/README-LGPL.txt                        |   33 +
 examples/README.txt                             |   21 +-
 examples/config/hibernate/README.txt            |    8 +
 .../hibernate/example-hibernate-L2-cache.xml    |   64 +
 examples/pom-standalone-lgpl.xml                |  217 +
 examples/pom-standalone.xml                     |  156 +-
 examples/pom.xml                                |  157 +-
 .../hibernate/HibernateL2CacheExample.java      |  245 +
 .../examples/datagrid/hibernate/Post.java       |  130 +
 .../examples/datagrid/hibernate/User.java       |  154 +
 .../datagrid/hibernate/package-info.java        |   22 +
 .../hibernate/CacheHibernatePersonStore.java    |  122 +
 .../hibernate/CacheHibernateStoreExample.java   |  151 +
 .../datagrid/store/hibernate/Person.hbm.xml     |   34 +
 .../datagrid/store/hibernate/hibernate.cfg.xml  |   41 +
 .../datagrid/store/hibernate/package-info.java  |   22 +
 .../misc/schedule/ComputeScheduleExample.java   |   82 +
 .../examples/misc/schedule/package-info.java    |   22 +
 ...ibernateL2CacheExampleMultiNodeSelfTest.java |   31 +
 .../HibernateL2CacheExampleSelfTest.java        |   33 +
 .../IgniteLgplExamplesSelfTestSuite.java        |   48 +
 .../ClientAbstractMultiThreadedSelfTest.java    |    3 +-
 .../apache/ignite/IgniteSystemProperties.java   |    9 +
 .../affinity/fair/FairAffinityFunction.java     |  497 +-
 .../rendezvous/RendezvousAffinityFunction.java  |  140 +-
 .../configuration/CacheConfiguration.java       |   30 +-
 .../configuration/TransactionConfiguration.java |    6 +-
 .../internal/GridEventConsumeHandler.java       |   19 +-
 .../apache/ignite/internal/IgniteKernal.java    |   69 +-
 .../ignite/internal/IgniteNodeAttributes.java   |    5 +-
 .../deployment/GridDeploymentLocalStore.java    |    8 +-
 .../GridDeploymentPerLoaderStore.java           |    8 +-
 .../GridDeploymentPerVersionStore.java          |    8 +-
 .../internal/portable/PortableContext.java      |   70 +-
 .../processors/cache/CacheMetricsImpl.java      |   12 +-
 .../processors/cache/CacheObjectContext.java    |   17 +-
 .../processors/cache/GridCacheAdapter.java      |  459 +-
 .../processors/cache/GridCacheContext.java      |   10 +-
 .../processors/cache/GridCacheEntryEx.java      |   62 +-
 .../cache/GridCacheEvictionManager.java         |    5 +-
 .../cache/GridCacheEvictionRequest.java         |   18 +-
 .../cache/GridCacheEvictionResponse.java        |    7 +-
 .../processors/cache/GridCacheIoManager.java    |   34 +-
 .../processors/cache/GridCacheMapEntry.java     |  137 +-
 .../processors/cache/GridCacheMessage.java      |   72 +-
 .../processors/cache/GridCacheMvcc.java         |  143 +-
 .../cache/GridCacheMvccCandidate.java           |   26 +-
 .../processors/cache/GridCacheProcessor.java    |   26 -
 .../cache/GridCacheSharedContext.java           |    3 +
 .../processors/cache/GridCacheUtils.java        |   70 +-
 .../distributed/GridCacheTtlUpdateRequest.java  |    7 +-
 .../distributed/GridCacheTxRecoveryFuture.java  |    9 +-
 .../distributed/GridCacheTxRecoveryRequest.java |    8 +-
 .../GridCacheTxRecoveryResponse.java            |   10 +-
 .../distributed/GridDistributedBaseMessage.java |   16 +-
 .../distributed/GridDistributedCacheEntry.java  |   12 +-
 .../distributed/GridDistributedLockRequest.java |    8 +-
 .../GridDistributedLockResponse.java            |   20 +-
 .../GridDistributedTxFinishRequest.java         |    6 +-
 .../GridDistributedTxFinishResponse.java        |    7 +-
 .../GridDistributedTxPrepareRequest.java        |   13 +-
 .../GridDistributedTxPrepareResponse.java       |   12 +-
 .../GridDistributedTxRemoteAdapter.java         |   61 +-
 .../GridDistributedUnlockRequest.java           |    7 +-
 .../dht/CacheDistributedGetFutureAdapter.java   |  158 +
 .../dht/GridDhtAffinityAssignmentRequest.java   |    7 +-
 .../dht/GridDhtAffinityAssignmentResponse.java  |    7 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   20 +-
 .../distributed/dht/GridDhtCacheEntry.java      |   23 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |  199 +-
 .../distributed/dht/GridDhtLockFuture.java      |   10 +-
 .../distributed/dht/GridDhtLockRequest.java     |    7 +-
 .../distributed/dht/GridDhtLockResponse.java    |   14 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |  127 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |    9 +-
 .../distributed/dht/GridDhtTxFinishRequest.java |    7 +-
 .../dht/GridDhtTxFinishResponse.java            |    2 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |    4 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |  118 +-
 .../dht/GridDhtTxPrepareRequest.java            |   28 +-
 .../dht/GridDhtTxPrepareResponse.java           |   13 +-
 .../distributed/dht/GridDhtUnlockRequest.java   |    7 +-
 .../dht/GridPartitionedGetFuture.java           |  182 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   28 +-
 .../GridDhtAtomicDeferredUpdateResponse.java    |   12 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |    6 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |   25 +-
 .../dht/atomic/GridDhtAtomicUpdateResponse.java |   11 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   12 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |   33 +-
 .../atomic/GridNearAtomicUpdateResponse.java    |   11 +-
 .../dht/colocated/GridDhtColocatedCache.java    |  138 +-
 .../colocated/GridDhtColocatedLockFuture.java   |    9 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |    5 +-
 .../dht/preloader/GridDhtForceKeysRequest.java  |   35 +-
 .../dht/preloader/GridDhtForceKeysResponse.java |   11 +-
 .../GridDhtPartitionDemandMessage.java          |    5 +
 .../GridDhtPartitionSupplyMessage.java          |    9 +-
 .../preloader/GridDhtPartitionSupplyPool.java   |   10 +-
 .../GridDhtPartitionsAbstractMessage.java       |    7 +-
 .../dht/preloader/GridDhtPreloader.java         |    3 +-
 .../distributed/near/GridNearAtomicCache.java   |    2 -
 .../distributed/near/GridNearCacheAdapter.java  |   32 +-
 .../distributed/near/GridNearCacheEntry.java    |   81 +-
 .../distributed/near/GridNearGetFuture.java     |  345 +-
 .../distributed/near/GridNearGetRequest.java    |   15 +-
 .../distributed/near/GridNearGetResponse.java   |   12 +-
 .../distributed/near/GridNearLockFuture.java    |    3 +-
 .../distributed/near/GridNearLockRequest.java   |    8 +-
 .../distributed/near/GridNearLockResponse.java  |    8 +-
 ...arOptimisticSerializableTxPrepareFuture.java |  930 ++++
 .../near/GridNearOptimisticTxPrepareFuture.java |  255 +-
 ...ridNearOptimisticTxPrepareFutureAdapter.java |  222 +
 .../GridNearPessimisticTxPrepareFuture.java     |   11 +-
 .../near/GridNearTransactionalCache.java        |   22 +-
 .../near/GridNearTxFinishFuture.java            |   20 +-
 .../near/GridNearTxFinishRequest.java           |    7 +-
 .../near/GridNearTxFinishResponse.java          |    2 +-
 .../cache/distributed/near/GridNearTxLocal.java |  157 +-
 .../near/GridNearTxPrepareFutureAdapter.java    |   13 +-
 .../near/GridNearTxPrepareRequest.java          |    8 +-
 .../near/GridNearTxPrepareResponse.java         |    8 +-
 .../distributed/near/GridNearUnlockRequest.java |    7 +-
 .../cache/local/GridLocalCacheEntry.java        |   23 +-
 .../cache/local/GridLocalLockFuture.java        |    2 +
 .../local/atomic/GridLocalAtomicCache.java      |    9 -
 .../portable/CacheObjectPortableContext.java    |    6 +-
 .../CacheObjectPortableProcessorImpl.java       |    5 +-
 .../query/GridCacheDistributedQueryFuture.java  |    5 +-
 .../query/GridCacheDistributedQueryManager.java |   25 +-
 .../cache/query/GridCacheQueryRequest.java      |   46 +-
 .../cache/query/GridCacheQueryResponse.java     |   29 +-
 .../continuous/CacheContinuousQueryHandler.java |    8 +-
 .../continuous/CacheContinuousQueryManager.java |    6 +-
 .../cache/transactions/IgniteInternalTx.java    |    8 +-
 .../transactions/IgniteTransactionsImpl.java    |    6 -
 .../cache/transactions/IgniteTxAdapter.java     |  128 +-
 .../cache/transactions/IgniteTxEntry.java       |   59 +-
 .../cache/transactions/IgniteTxHandler.java     |   20 +-
 .../transactions/IgniteTxLocalAdapter.java      |  843 ++--
 .../cache/transactions/IgniteTxLocalEx.java     |   27 +-
 .../cache/transactions/IgniteTxManager.java     |  327 +-
 .../cache/version/GridCacheVersionManager.java  |   73 +-
 .../IgniteCacheObjectProcessorImpl.java         |    7 +-
 .../datastreamer/DataStreamerImpl.java          |    2 +-
 .../ignite/internal/util/lang/GridFunc.java     |    8 +-
 .../ignite/internal/util/nio/GridNioServer.java |    2 +-
 .../ignite/marshaller/AbstractMarshaller.java   |   12 +-
 .../ignite/marshaller/jdk/JdkMarshaller.java    |    7 +-
 .../optimized/OptimizedMarshaller.java          |   14 +-
 .../optimized/OptimizedMarshallerUtils.java     |   35 +-
 .../marshaller/portable/PortableMarshaller.java |    5 +
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   52 +
 .../apache/ignite/transactions/Transaction.java |    2 +-
 .../transactions/TransactionIsolation.java      |    3 +-
 modules/core/src/test/config/tests.properties   |    4 +-
 .../AbstractAffinityFunctionSelfTest.java       |  293 ++
 .../affinity/AffinityClientNodeSelfTest.java    |  194 +
 ...ityFunctionBackupFilterAbstractSelfTest.java |  138 +
 ...unctionExcludeNeighborsAbstractSelfTest.java |  182 +
 .../affinity/IgniteClientNodeAffinityTest.java  |  194 -
 .../fair/FairAffinityDynamicCacheSelfTest.java  |   99 +
 ...airAffinityFunctionBackupFilterSelfTest.java |   35 +
 ...ffinityFunctionExcludeNeighborsSelfTest.java |   31 +
 .../fair/FairAffinityFunctionNodesSelfTest.java |  245 +
 .../fair/FairAffinityFunctionSelfTest.java      |   31 +
 .../GridFairAffinityFunctionNodesSelfTest.java  |  245 -
 .../fair/GridFairAffinityFunctionSelfTest.java  |  270 --
 .../IgniteFairAffinityDynamicCacheSelfTest.java |   97 -
 ...ousAffinityFunctionBackupFilterSelfTest.java |   35 +
 ...ffinityFunctionExcludeNeighborsSelfTest.java |   32 +
 .../RendezvousAffinityFunctionSelfTest.java     |   50 +
 .../GridDiscoveryManagerAttributesSelfTest.java |   45 +
 .../cache/CacheNearReaderUpdateTest.java        |  388 ++
 .../CacheSerializableTransactionsTest.java      | 4297 ++++++++++++++++++
 .../cache/CrossCacheTxRandomOperationsTest.java |   10 +-
 .../GridCacheAbstractFailoverSelfTest.java      |   14 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |    4 +-
 .../GridCacheAbstractLocalStoreSelfTest.java    |    5 +
 .../GridCacheAbstractRemoveFailureTest.java     |   94 +-
 ...eAtomicEntryProcessorDeploymentSelfTest.java |  211 +
 .../GridCacheConcurrentTxMultiNodeTest.java     |    3 -
 .../GridCacheConditionalDeploymentSelfTest.java |  190 +
 ...idCacheConfigurationConsistencySelfTest.java |   17 -
 .../GridCacheDeploymentOffHeapSelfTest.java     |   17 +-
 .../cache/GridCacheDeploymentSelfTest.java      |   26 +-
 .../cache/GridCacheMvccFlagsTest.java           |    6 +-
 .../cache/GridCacheMvccPartitionedSelfTest.java |  164 +
 .../processors/cache/GridCacheMvccSelfTest.java |    3 +-
 .../processors/cache/GridCacheTestEntryEx.java  |   53 +-
 ...ctionalEntryProcessorDeploymentSelfTest.java |   31 +
 .../cache/IgniteCacheCreateRestartSelfTest.java |    2 +
 .../IgniteCacheEntryListenerAbstractTest.java   |   50 +-
 .../processors/cache/IgniteTxAbstractTest.java  |   42 +-
 .../IgniteTxMultiThreadedAbstractTest.java      |  106 +-
 ...IgnitePartitionedCountDownLatchSelfTest.java |    7 +-
 ...dCachePartitionedAffinityFilterSelfTest.java |  143 -
 ...onedNearDisabledTxMultiThreadedSelfTest.java |   31 +
 ...niteCacheClientNodeChangingTopologyTest.java |  170 +-
 .../dht/GridCacheDhtPreloadPutGetSelfTest.java  |    3 +
 .../IgniteCacheCrossCacheTxFailoverTest.java    |   19 +
 .../dht/IgniteCacheLockFailoverSelfTest.java    |   11 +
 ...eAtomicInvalidPartitionHandlingSelfTest.java |    6 +-
 ...unctionExcludeNeighborsAbstractSelfTest.java |  184 -
 .../near/GridCacheNearTxExceptionSelfTest.java  |    1 +
 ...CachePartitionedTxMultiThreadedSelfTest.java |   15 +-
 ...ffinityFunctionExcludeNeighborsSelfTest.java |   32 -
 ...xcludeNeighborsMultiNodeFullApiSelfTest.java |   36 +
 ...tedFairAffinityMultiNodeFullApiSelfTest.java |   35 +
 ...xcludeNeighborsMultiNodeFullApiSelfTest.java |   36 +
 ...dezvousAffinityMultiNodeFullApiSelfTest.java |   36 +
 .../DataStreamerUpdateAfterLoadTest.java        |  184 +
 .../nio/IgniteExceptionInNioWorkerSelfTest.java |  105 +
 .../loadtests/hashmap/GridHashMapLoadTest.java  |    6 +-
 .../communication/GridCacheMessageSelfTest.java |   17 +-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |    4 +-
 .../inmemory/GridTestSwapSpaceSpi.java          |    8 +
 .../ignite/testframework/GridTestUtils.java     |   19 +-
 .../junits/common/GridCommonAbstractTest.java   |   19 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    7 +-
 .../IgniteCacheFullApiSelfTestSuite.java        |    8 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   38 +-
 .../testsuites/IgniteCacheTestSuite2.java       |   14 +-
 .../testsuites/IgniteCacheTestSuite3.java       |    8 +-
 .../testsuites/IgniteCacheTestSuite5.java       |   40 +
 .../testsuites/IgniteP2PSelfTestSuite.java      |   16 +-
 .../p2p/CacheDeploymentEntryProcessor.java      |   35 +
 .../CacheDeploymentPortableEntryProcessor.java  |   35 +
 modules/extdata/uri/pom.xml                     |   21 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |    4 +
 modules/jms11/pom.xml                           |    9 +-
 .../stream/jms11/IgniteJmsStreamerTest.java     |    9 +-
 .../apache/ignite/stream/mqtt/MqttStreamer.java |  386 +-
 .../stream/mqtt/IgniteMqttStreamerTest.java     |  142 +-
 .../mqtt/IgniteMqttStreamerTestSuite.java       |    4 +-
 modules/spark-2.10/pom.xml                      |    4 +-
 modules/spark/pom.xml                           |    4 +-
 .../config/benchmark-multicast.properties       |    5 +-
 .../yardstick/IgniteBenchmarkArguments.java     |   11 +
 .../IgniteAccountSerializableTxBenchmark.java   |   81 +
 .../cache/IgniteAccountTxAbstractBenchmark.java |   61 +
 .../cache/IgniteAccountTxBenchmark.java         |   74 +
 .../cache/IgniteCacheAbstractBenchmark.java     |   68 +
 .../IgnitePutAllSerializableTxBenchmark.java    |   77 +
 .../cache/IgnitePutAllTxBenchmark.java          |    6 +-
 .../ignite/yardstick/cache/model/Account.java   |   42 +
 .../ipfinder/zk/ZookeeperIpFinderTestSuite.java |   32 +
 parent/pom.xml                                  |    1 +
 pom.xml                                         |   64 -
 288 files changed, 15447 insertions(+), 6858 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e4c6a908/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 2eb7b8d,374743f..ae3f21f
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@@ -19,7 -19,9 +19,8 @@@ package org.apache.ignite.configuration
  
  import java.io.Serializable;
  import java.util.Collection;
 -import java.util.HashSet;
  import javax.cache.Cache;
+ import javax.cache.configuration.CacheEntryListenerConfiguration;
  import javax.cache.configuration.CompleteConfiguration;
  import javax.cache.configuration.Factory;
  import javax.cache.configuration.MutableConfiguration;

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4c6a908/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4c6a908/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4c6a908/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4c6a908/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4c6a908/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableContext.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4c6a908/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4c6a908/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
index ea2e1cc,0000000..649e69d
mode 100644,000000..100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
@@@ -1,304 -1,0 +1,309 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *      http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.ignite.marshaller.portable;
 +
 +import java.io.ByteArrayOutputStream;
 +import java.io.IOException;
 +import java.io.InputStream;
 +import java.io.OutputStream;
 +import java.util.ArrayList;
 +import java.util.Collection;
 +import org.apache.ignite.IgniteCheckedException;
 +import org.apache.ignite.internal.portable.GridPortableMarshaller;
 +import org.apache.ignite.internal.portable.PortableContext;
 +import org.apache.ignite.marshaller.AbstractMarshaller;
 +import org.apache.ignite.marshaller.MarshallerContext;
 +import org.apache.ignite.portable.PortableException;
 +import org.apache.ignite.portable.PortableIdMapper;
 +import org.apache.ignite.portable.PortableObject;
 +import org.apache.ignite.portable.PortableSerializer;
 +import org.apache.ignite.portable.PortableTypeConfiguration;
 +import org.jetbrains.annotations.Nullable;
 +
 +/**
 + * Implementation of {@link org.apache.ignite.marshaller.Marshaller} that lets to serialize and deserialize all objects
 + * in the portable format.
 + * <p>
 + * {@code PortableMarshaller} is tested only on Java HotSpot VM on other VMs it could yield unexpected results.
 + * <p>
 + * <h1 class="header">Configuration</h1>
 + * <h2 class="header">Mandatory</h2>
 + * This marshaller has no mandatory configuration parameters.
 + * <h2 class="header">Java Example</h2>
 + * <pre name="code" class="java">
 + * PortableMarshaller marshaller = new PortableMarshaller();
 + *
 + * IgniteConfiguration cfg = new IgniteConfiguration();
 + *
 + * // Override marshaller.
 + * cfg.setMarshaller(marshaller);
 + *
 + * // Starts grid.
 + * G.start(cfg);
 + * </pre>
 + * <h2 class="header">Spring Example</h2>
 + * PortableMarshaller can be configured from Spring XML configuration file:
 + * <pre name="code" class="xml">
 + * &lt;bean id="grid.custom.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" singleton="true"&gt;
 + *     ...
 + *     &lt;property name="marshaller"&gt;
 + *         &lt;bean class="org.apache.ignite.marshaller.portable.PortableMarshaller"&gt;
 + *            ...
 + *         &lt;/bean&gt;
 + *     &lt;/property&gt;
 + *     ...
 + * &lt;/bean&gt;
 + * </pre>
 + * <p>
 + * <img src="http://ignite.apache.org/images/spring-small.png">
 + * <br>
 + * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
 + */
 +public class PortableMarshaller extends AbstractMarshaller {
 +    /** Class names. */
 +    private Collection<String> clsNames;
 +
 +    /** ID mapper. */
 +    private PortableIdMapper idMapper;
 +
 +    /** Serializer. */
 +    private PortableSerializer serializer;
 +
 +    /** Types. */
 +    private Collection<PortableTypeConfiguration> typeCfgs;
 +
 +    /** Whether to convert string to bytes using UTF-8 encoding. */
 +    private boolean convertString = true;
 +
 +    /** Meta data enabled flag. */
 +    private boolean metaDataEnabled = true;
 +
 +    /** Keep deserialized flag. */
 +    private boolean keepDeserialized = true;
 +
 +    /** */
 +    private GridPortableMarshaller impl;
 +
 +    /**
 +     * Gets class names.
 +     *
 +     * @return Class names.
 +     */
 +    public Collection<String> getClassNames() {
 +        return clsNames;
 +    }
 +
 +    /**
 +     * Sets class names of portable objects explicitly.
 +     *
 +     * @param clsNames Class names.
 +     */
 +    public void setClassNames(Collection<String> clsNames) {
 +        this.clsNames = new ArrayList<>(clsNames.size());
 +
 +        for (String clsName : clsNames)
 +            this.clsNames.add(clsName.trim());
 +    }
 +
 +    /**
 +     * 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;
 +    }
 +
 +    /**
 +     * Gets types configuration.
 +     *
 +     * @return Types configuration.
 +     */
 +    public Collection<PortableTypeConfiguration> getTypeConfigurations() {
 +        return typeCfgs;
 +    }
 +
 +    /**
 +     * Sets type configurations.
 +     *
 +     * @param typeCfgs Type configurations.
 +     */
 +    public void setTypeConfigurations(Collection<PortableTypeConfiguration> typeCfgs) {
 +        this.typeCfgs = typeCfgs;
 +    }
 +
 +    /**
 +     * Gets strings must be converted to or from bytes using UTF-8 encoding.
 +     * <p>
 +     * Default value is {@code true}.
 +     *
 +     * @return Flag indicating whether string must be converted to byte array using UTF-8 encoding.
 +     */
 +    public boolean isConvertStringToBytes() {
 +        return convertString;
 +    }
 +
 +    /**
 +     * Sets strings must be converted to or from bytes using UTF-8 encoding.
 +     * <p>
 +     * Default value is {@code true}.
 +     *
 +     * @param convertString Flag indicating whether string must be converted to byte array using UTF-8 encoding.
 +     */
 +    public void setConvertStringToBytes(boolean convertString) {
 +        this.convertString = convertString;
 +    }
 +
 +    /**
 +     * If {@code true}, meta data will be collected or all types. If you need to override this behaviour for
 +     * some specific type, use {@link PortableTypeConfiguration#setMetaDataEnabled(Boolean)} method.
 +     * <p>
 +     * Default value if {@code true}.
 +     *
 +     * @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;
 +    }
 +
 +    /**
 +     * If {@code true}, {@link PortableObject} will cache deserialized instance after
 +     * {@link PortableObject#deserialize()} is called. All consequent calls of this
 +     * method on the same instance of {@link PortableObject} will return that cached
 +     * value without actually deserializing portable object. If you need to override this
 +     * behaviour for some specific type, use {@link PortableTypeConfiguration#setKeepDeserialized(Boolean)}
 +     * method.
 +     * <p>
 +     * Default value if {@code true}.
 +     *
 +     * @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;
 +    }
 +
 +    /**
 +     * Returns currently set {@link MarshallerContext}.
 +     *
 +     * @return Marshaller context.
 +     */
 +    public MarshallerContext getContext() {
 +        return ctx;
 +    }
 +
 +    /**
 +     * Sets {@link PortableContext}.
 +     * <p/>
 +     * @param ctx Portable context.
 +     */
 +    private void setPortableContext(PortableContext ctx) {
 +        ctx.configure(this);
 +
 +        impl = new GridPortableMarshaller(ctx);
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public byte[] marshal(@Nullable Object obj) throws IgniteCheckedException {
 +        return impl.marshal(obj, 0);
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public void marshal(@Nullable Object obj, OutputStream out) throws IgniteCheckedException {
 +        byte[] arr = marshal(obj);
 +
 +        try {
 +            out.write(arr);
 +        }
 +        catch (IOException e) {
 +            throw new PortableException("Failed to marshal the object: " + obj, e);
 +        }
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public <T> T unmarshal(byte[] bytes, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
 +        return impl.deserialize(bytes, clsLdr);
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public <T> T unmarshal(InputStream in, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
 +        ByteArrayOutputStream buf = new ByteArrayOutputStream();
 +
 +        byte[] arr = new byte[4096];
 +        int cnt;
 +
 +        // we have to fully read the InputStream because GridPortableMarshaller requires support of a method that
 +        // returns number of bytes remaining.
 +        try {
 +            while ((cnt = in.read(arr)) != -1)
 +                buf.write(arr, 0, cnt);
 +
 +            buf.flush();
 +
 +            return impl.deserialize(buf.toByteArray(), clsLdr);
 +        }
 +        catch (IOException e) {
 +            throw new PortableException("Failed to unmarshal the object from InputStream", e);
 +        }
 +    }
++
++    /** {@inheritDoc} */
++    @Override public void onUndeploy(ClassLoader ldr) {
++        impl.context().onUndeploy(ldr);
++    }
 +}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4c6a908/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4c6a908/parent/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/e4c6a908/pom.xml
----------------------------------------------------------------------