Merge branch 'sprint-2' into ignite-394

Conflicts:
        
examples/src/main/java/org/apache/ignite/examples/datagrid/CachePopularNumbersExample.java
        
examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCachePopularNumbersExample.scala


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

Branch: refs/heads/ignite-406
Commit: 1511004805af38185af83b98d636830e6c6a105b
Parents: 98b6643 d423fc7
Author: Artem Shutak <ashu...@gridgain.com>
Authored: Thu Mar 12 13:15:49 2015 +0300
Committer: Artem Shutak <ashu...@gridgain.com>
Committed: Thu Mar 12 13:15:49 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |  13 +-
 LICENSE.txt                                     |  54 ++++-
 NOTICE.txt                                      |  11 +
 docs/hadoop_readme.md                           |  17 ++
 examples/config/example-cache.xml               |   4 +-
 .../datagrid/CachePopularNumbersExample.java    |   4 +-
 .../MemcacheRestExampleNodeStartup.java         |   5 +-
 .../ScalarCachePopularNumbersExample.scala      |  39 +++-
 ipc/shmem/compile                               |   4 +
 ipc/shmem/config.guess                          |   4 +
 ipc/shmem/config.sub                            |   4 +
 ipc/shmem/depcomp                               |   4 +
 ipc/shmem/ltmain.sh                             |   3 +
 ipc/shmem/missing                               |   4 +
 modules/clients/readme.md                       |  17 ++
 .../clients/src/test/resources/spring-cache.xml |   8 +-
 .../java/org/apache/ignite/IgniteCache.java     |  24 ++
 .../apache/ignite/cache/CachePreloadMode.java   |  67 ------
 .../apache/ignite/cache/CacheRebalanceMode.java |  67 ++++++
 .../java/org/apache/ignite/cache/GridCache.java |  12 +-
 .../configuration/CacheConfiguration.java       | 231 +++++++++----------
 .../ignite/events/CachePreloadingEvent.java     | 172 --------------
 .../ignite/events/CacheRebalancingEvent.java    | 172 ++++++++++++++
 .../java/org/apache/ignite/events/Event.java    |   2 +-
 .../org/apache/ignite/events/EventType.java     |  44 ++--
 .../ignite/internal/AsyncSupportAdapter.java    |  15 +-
 .../ignite/internal/GridTaskSessionImpl.java    |   3 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   8 +-
 .../processors/cache/CachePeekModes.java        |  28 +++
 .../processors/cache/GridCacheAdapter.java      |  25 ++
 .../processors/cache/GridCacheAttributes.java   |  32 +--
 .../processors/cache/GridCacheContext.java      |   8 +-
 .../processors/cache/GridCacheEventManager.java |  12 +-
 .../cache/GridCacheEvictionManager.java         |   6 +-
 .../GridCachePartitionExchangeManager.java      |   6 +-
 .../processors/cache/GridCacheProcessor.java    |  74 +++---
 .../processors/cache/GridCacheUtils.java        |   4 +-
 .../processors/cache/IgniteCacheProxy.java      |   7 +
 .../distributed/dht/GridDhtLocalPartition.java  |   4 +-
 .../distributed/dht/GridDhtLockFuture.java      |   4 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |  14 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   4 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |  30 +--
 .../preloader/GridDhtPartitionDemandPool.java   |  78 +++----
 .../preloader/GridDhtPartitionSupplyPool.java   |  22 +-
 .../dht/preloader/GridDhtPreloader.java         |  14 +-
 .../cache/query/GridCacheQueryManager.java      |   4 +-
 .../closure/GridClosureProcessor.java           |  79 ++++---
 .../resource/GridNoImplicitInjection.java       |  25 ++
 .../processors/resource/GridResourceField.java  |  15 +-
 .../processors/resource/GridResourceIoc.java    |  90 ++++----
 .../processors/resource/GridResourceMethod.java |   3 +
 .../resource/GridResourceProcessor.java         |  55 ++---
 .../internal/util/GridConcurrentFactory.java    |  59 -----
 .../internal/util/GridIdentityHashSet.java      |  63 -----
 .../internal/util/GridLeanIdentitySet.java      | 117 ++++++++++
 .../ignite/internal/util/GridLeanMap.java       |   2 +-
 .../ignite/internal/util/IgniteUtils.java       |   2 +-
 .../ignite/internal/util/lang/GridFunc.java     |   2 +-
 .../visor/cache/VisorCacheCompactTask.java      |  66 ------
 .../cache/VisorCachePreloadConfiguration.java   |  16 +-
 .../org/jdk8/backport/ConcurrentHashMap8.java   |  14 +-
 .../resources/META-INF/classnames.properties    |  34 +--
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../core/src/test/config/discovery-stress.xml   |   2 +-
 modules/core/src/test/config/example-cache.xml  |   4 +-
 .../config/load/dsi-49-server-production.xml    |   4 +-
 .../src/test/config/load/dsi-load-client.xml    |   4 +-
 .../src/test/config/load/dsi-load-server.xml    |   4 +-
 .../core/src/test/config/spring-cache-load.xml  |   2 +-
 .../core/src/test/config/spring-multicache.xml  |  16 +-
 modules/core/src/test/config/start-nodes.ini    |  15 ++
 .../test/config/websession/spring-cache-1.xml   |   6 +-
 .../test/config/websession/spring-cache-2.xml   |   6 +-
 .../test/config/websession/spring-cache-3.xml   |   6 +-
 .../internal/GridUpdateNotifierSelfTest.java    |  16 ++
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   4 +-
 .../GridCacheAbstractFailoverSelfTest.java      |   4 +-
 .../GridCacheAbstractLocalStoreSelfTest.java    |   8 +-
 ...acheAbstractUsersAffinityMapperSelfTest.java |   4 +-
 .../cache/GridCacheBasicStoreAbstractTest.java  |   4 +-
 .../GridCacheConcurrentTxMultiNodeTest.java     |   4 +-
 ...idCacheConfigurationConsistencySelfTest.java |   6 +-
 ...ridCacheConfigurationValidationSelfTest.java |   8 +-
 .../cache/GridCacheDeploymentSelfTest.java      |   4 +-
 ...idCacheGetAndTransformStoreAbstractTest.java |   4 +-
 .../cache/GridCacheIncrementTransformTest.java  |   4 +-
 .../cache/GridCacheMultiUpdateLockSelfTest.java |   4 +-
 .../GridCacheOrderedPreloadingSelfTest.java     |   6 +-
 .../cache/GridCacheP2PUndeploySelfTest.java     |   8 +-
 .../cache/GridCachePartitionedGetSelfTest.java  |   4 +-
 ...hePartitionedProjectionAffinitySelfTest.java |   4 +-
 .../GridCachePreloadingEvictionsSelfTest.java   |   6 +-
 .../GridCacheQueryInternalKeysSelfTest.java     |   4 +-
 .../cache/GridCacheSwapPreloadSelfTest.java     |   4 +-
 .../GridCacheValueBytesPreloadingSelfTest.java  |   2 +-
 ...idCacheValueConsistencyAbstractSelfTest.java |   4 +-
 .../IgniteCacheAbstractStopBusySelfTest.java    |   2 +-
 ...tAllUpdateNonPreloadedPartitionSelfTest.java |   4 +-
 ...dCacheQueueMultiNodeConsistencySelfTest.java |   3 +-
 .../IgniteCollectionAbstractTest.java           |   4 +-
 ...GridCachePartitionedNodeRestartSelfTest.java |   4 +-
 ...idCachePartitionedNodeRestartTxSelfTest.java |   4 +-
 ...PartitionedQueueCreateMultiNodeSelfTest.java |   4 +-
 .../GridCacheAbstractNodeRestartSelfTest.java   |   6 +-
 .../GridCacheAbstractPrimarySyncSelfTest.java   |   4 +-
 ...acheEntrySetIterationPreloadingSelfTest.java |   2 +-
 ...heExpiredEntriesPreloadAbstractSelfTest.java |   6 +-
 .../distributed/GridCacheLockAbstractTest.java  |   4 +-
 ...dCacheMultithreadedFailoverAbstractTest.java |   4 +-
 ...dCachePartitionedAffinityFilterSelfTest.java |   4 +-
 .../GridCachePreloadEventsAbstractSelfTest.java |   8 +-
 .../GridCachePreloadLifecycleAbstractTest.java  |   6 +-
 ...GridCachePreloadRestartAbstractSelfTest.java |  10 +-
 ...iteTxConsistencyRestartAbstractSelfTest.java |   4 +-
 .../IgniteTxPreloadAbstractTest.java            |   6 +-
 .../dht/GridCacheAtomicNearCacheSelfTest.java   |   4 +-
 ...dCacheColocatedTxSingleThreadedSelfTest.java |   4 +-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |   4 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |   4 +-
 .../dht/GridCacheDhtInternalEntrySelfTest.java  |   4 +-
 .../dht/GridCacheDhtMappingSelfTest.java        |   4 +-
 .../dht/GridCacheDhtPreloadBigDataSelfTest.java |  10 +-
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |  26 +--
 .../GridCacheDhtPreloadDisabledSelfTest.java    |   8 +-
 .../GridCacheDhtPreloadMessageCountTest.java    |   4 +-
 .../dht/GridCacheDhtPreloadPutGetSelfTest.java  |   6 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |  28 +--
 .../GridCacheDhtPreloadStartStopSelfTest.java   |  12 +-
 .../dht/GridCacheDhtPreloadUnloadSelfTest.java  |  10 +-
 ...ePartitionedNearDisabledMetricsSelfTest.java |   4 +-
 ...idCachePartitionedPreloadEventsSelfTest.java |   8 +-
 ...dCachePartitionedTopologyChangeSelfTest.java |   4 +-
 ...ridCachePartitionedUnloadEventsSelfTest.java |  12 +-
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   4 +-
 ...unctionExcludeNeighborsAbstractSelfTest.java |   4 +-
 ...GridCacheAtomicMultiNodeFullApiSelfTest.java |   4 +-
 ...idCacheAtomicPartitionedMetricsSelfTest.java |   4 +-
 .../near/GridCacheNearEvictionSelfTest.java     |   7 +-
 .../near/GridCacheNearMultiGetSelfTest.java     |   4 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |   4 +-
 .../GridCacheNearPartitionedClearSelfTest.java  |   4 +-
 .../near/GridCacheNearReadersSelfTest.java      |   4 +-
 .../near/GridCacheNearTxMultiNodeSelfTest.java  |   4 +-
 ...AffinityExcludeNeighborsPerformanceTest.java |   4 +-
 .../GridCachePartitionedAffinitySelfTest.java   |   4 +-
 .../GridCachePartitionedBasicOpSelfTest.java    |   2 +-
 .../near/GridCachePartitionedEventSelfTest.java |   4 +-
 ...idCachePartitionedHitsAndMissesSelfTest.java |   2 +-
 .../GridCachePartitionedMetricsSelfTest.java    |   4 +-
 ...achePartitionedMultiNodeCounterSelfTest.java |   4 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |   4 +-
 .../GridCachePartitionedNodeRestartTest.java    |   4 +-
 ...ePartitionedOptimisticTxNodeRestartTest.java |   4 +-
 ...achePartitionedPreloadLifecycleSelfTest.java |   4 +-
 ...hePartitionedQueryMultiThreadedSelfTest.java |   2 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |   2 +-
 ...achePartitionedTxSingleThreadedSelfTest.java |   4 +-
 .../GridCacheReplicatedInvalidateSelfTest.java  |   4 +-
 .../GridCacheReplicatedNodeRestartSelfTest.java |   6 +-
 .../GridCacheSyncReplicatedPreloadSelfTest.java |   6 +-
 ...CacheReplicatedPreloadLifecycleSelfTest.java |   4 +-
 .../GridCacheReplicatedPreloadSelfTest.java     |  16 +-
 ...eplicatedPreloadStartStopEventsSelfTest.java |   6 +-
 .../GridCacheEvictionFilterSelfTest.java        |   4 +-
 .../GridCacheLruNearEvictionPolicySelfTest.java |   4 +-
 ...heNearOnlyLruNearEvictionPolicySelfTest.java |   4 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |   4 +-
 ...dCacheAbstractReduceFieldsQuerySelfTest.java |   4 +-
 .../processors/igfs/IgfsSizeSelfTest.java       |   7 +-
 .../ignite/jvmtest/ConcurrentMapTest.java       |   5 +-
 .../lang/utils/GridLeanIdentitySetSelfTest.java |  62 +++++
 .../loadtests/GridCacheMultiNodeLoadTest.java   |   4 +-
 .../capacity/spring-capacity-cache.xml          |   4 +-
 .../loadtests/colocation/spring-colocation.xml  |   4 +-
 .../GridCachePartitionedAtomicLongLoadTest.java |   2 +-
 .../GridP2PContinuousDeploymentSelfTest.java    |   4 +-
 .../tcp/GridCacheDhtLockBackupSelfTest.java     |   4 +-
 .../ignite/testframework/GridTestUtils.java     |  50 ++++
 .../testframework/junits/GridAbstractTest.java  |   1 +
 .../junits/common/GridCommonAbstractTest.java   |   4 +-
 .../testsuites/IgniteLangSelfTestSuite.java     |   1 +
 .../webapp/META-INF/ignite-webapp-config.xml    |  12 +-
 .../GridCacheAbstractFieldsQuerySelfTest.java   |   4 +-
 .../cache/GridCacheAbstractQuerySelfTest.java   |   4 +-
 .../cache/GridCacheCrossCacheQuerySelfTest.java |   4 +-
 .../GridCacheCrossCacheQuerySelfTestNewApi.java |   4 +-
 .../cache/GridIndexingWithNoopSwapSelfTest.java |   4 +-
 .../near/GridCacheQueryNodeRestartSelfTest.java |   6 +-
 .../query/h2/sql/GridQueryParsingTest.java      |   5 +-
 .../tcp/GridOrderedMessageCancelSelfTest.java   |   4 +-
 .../ignite/visor/commands/VisorConsole.scala    |   3 +-
 .../commands/cache/VisorCacheCommand.scala      |  30 +--
 .../cache/VisorCacheCompactCommand.scala        | 151 ------------
 .../commands/events/VisorEventsCommand.scala    |   4 +-
 .../scala/org/apache/ignite/visor/visor.scala   |   4 +-
 .../cache/VisorCacheCompactCommandSpec.scala    | 103 ---------
 .../testsuites/VisorConsoleSelfTestSuite.scala  |   3 +-
 modules/winservice/README.md                    |  17 ++
 modules/yardstick/README.md                     |  17 ++
 pom.xml                                         |  21 +-
 201 files changed, 1633 insertions(+), 1534 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/15110048/examples/src/main/java/org/apache/ignite/examples/datagrid/CachePopularNumbersExample.java
----------------------------------------------------------------------
diff --cc 
examples/src/main/java/org/apache/ignite/examples/datagrid/CachePopularNumbersExample.java
index e43eb55,f09661d..58a2e2f
--- 
a/examples/src/main/java/org/apache/ignite/examples/datagrid/CachePopularNumbersExample.java
+++ 
b/examples/src/main/java/org/apache/ignite/examples/datagrid/CachePopularNumbersExample.java
@@@ -140,8 -140,8 +140,8 @@@ public class CachePopularNumbersExampl
      /**
       * Increments value for key.
       */
 -    private static class IncrementingUpdater implements 
IgniteDataLoader.Updater<Integer, Long> {
 +    private static class IncrementingUpdater implements 
IgniteDataStreamer.Updater<Integer, Long> {
-         /** */
+         /** Process entries to increase value by entry key. */
          private static final EntryProcessor<Integer, Long, Void> INC = new 
EntryProcessor<Integer, Long, Void>() {
              @Override public Void process(MutableEntry<Integer, Long> e, 
Object... args) {
                  Long val = e.getValue();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/15110048/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCachePopularNumbersExample.scala
----------------------------------------------------------------------
diff --cc 
examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCachePopularNumbersExample.scala
index 594ade1,dbd7791..30565b5
--- 
a/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCachePopularNumbersExample.scala
+++ 
b/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCachePopularNumbersExample.scala
@@@ -17,11 -17,15 +17,15 @@@
  
  package org.apache.ignite.scalar.examples
  
- import org.apache.ignite.IgniteException
  import org.apache.ignite.examples.datagrid.CacheNodeStartup
+ import org.apache.ignite.internal.util.scala.impl
  import org.apache.ignite.scalar.scalar
  import org.apache.ignite.scalar.scalar._
 -import org.apache.ignite.{IgniteCache, IgniteDataLoader, IgniteException}
++import org.apache.ignite.{IgniteCache, IgniteDataStreamer, IgniteException}
  
+ import javax.cache.processor.{EntryProcessor, MutableEntry}
+ import java.util
+ import java.util.Map.Entry
  import java.util.Timer
  
  import scala.collection.JavaConversions._
@@@ -31,7 -35,7 +35,7 @@@ import scala.util.Rando
   * Real time popular number counter.
   * <p>
   * Remote nodes should always be started with special configuration file which
-- * enables P2P class loading: `ignite.sh examples/config/example-cache.xml`
++ * enables P2P class streaming: `ignite.sh examples/config/example-cache.xml`
   * <p>
   * Alternatively you can run [[CacheNodeStartup]] in another JVM which will
   * start node with `examples/config/example-cache.xml` configuration.
@@@ -92,13 -99,13 +99,13 @@@ object ScalarCachePopularNumbersExampl
      def streamData() {
          // Set larger per-node buffer size since our state is relatively 
small.
          // Reduce parallel operations since we running the whole ignite 
cluster locally under heavy load.
 -        val ldr = dataLoader$[Int, Long](CACHE_NAME, 2048)
 +        val smtr = dataStreamer$[Int, Long](CACHE_NAME, 2048)
  
-         // TODO IGNITE-44: restore invoke.
 -        ldr.updater(new IncrementingUpdater())
++        smtr.updater(new IncrementingUpdater())
  
-         (0 until CNT) foreach (_ => smtr.addData(Random.nextInt(RANGE), 1L))
 -        (0 until CNT) foreach (_ => ldr.addData(RAND.nextInt(RANGE), 1L))
++        (0 until CNT) foreach (_ => smtr.addData(RAND.nextInt(RANGE), 1L))
  
 -        ldr.close(false)
 +        smtr.close(false)
      }
  
      /**
@@@ -115,4 -122,22 +122,22 @@@
  
          println("------------------")
      }
+ 
+     /**
+      * Increments value for key.
+      */
 -    private class IncrementingUpdater extends IgniteDataLoader.Updater[Int, 
Long] {
++    private class IncrementingUpdater extends IgniteDataStreamer.Updater[Int, 
Long] {
+         private[this] final val INC = new EntryProcessor[Int, Long, Object]() 
{
+             /** Process entries to increase value by entry key. */
+             override def process(e: MutableEntry[Int, Long], args: AnyRef*): 
Object = {
+                 e.setValue(Option(e.getValue).map(_ + 1).getOrElse(1L))
+ 
+                 null
+             }
+         }
+ 
+         @impl def update(cache: IgniteCache[Int, Long], entries: 
util.Collection[Entry[Int, Long]]) {
+             entries.foreach(entry => cache.invoke(entry.getKey, INC))
+         }
+     }
  }

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/15110048/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxConsistencyRestartAbstractSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/15110048/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedHitsAndMissesSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/15110048/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheLruNearEvictionPolicySelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/15110048/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
----------------------------------------------------------------------

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

Reply via email to