Merge branch 'ignite-sprint-5' into ignite-745 Conflicts: modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMetricsSelfTest.java
Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/8796bc59 Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/8796bc59 Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/8796bc59 Branch: refs/heads/ignite-218 Commit: 8796bc592f7a9bb3f664cdfc29c687ba0bc12a9b Parents: f6eaaad f027ac5 Author: agura <ag...@gridgain.com> Authored: Tue May 12 19:53:52 2015 +0300 Committer: agura <ag...@gridgain.com> Committed: Tue May 12 19:53:52 2015 +0300 ---------------------------------------------------------------------- DEVNOTES.txt | 47 +- RELEASE_NOTES.txt | 13 +- assembly/dependencies-fabric.xml | 50 + assembly/dependencies-hadoop.xml | 1 - assembly/dependencies-optional-fabric.xml | 82 - assembly/dependencies-optional-hadoop.xml | 83 - assembly/release-base.xml | 11 +- assembly/release-fabric.xml | 5 + assembly/release-schema-import.xml | 50 + bin/ignite-schema-import.bat | 2 +- bin/ignite-schema-import.sh | 2 +- bin/ignite.bat | 2 +- bin/ignite.sh | 2 +- bin/ignitevisorcmd.bat | 2 +- bin/ignitevisorcmd.sh | 2 +- bin/include/build-classpath.bat | 46 + bin/include/build-classpath.sh | 71 + bin/include/target-classpath.bat | 46 - bin/include/target-classpath.sh | 71 - dev-tools/.gitignore | 2 + dev-tools/build.gradle | 45 + dev-tools/src/main/groovy/jiraslurp.groovy | 146 + examples/README.txt | 2 +- examples/pom.xml | 2 +- .../hibernate/CacheHibernateStoreExample.java | 3 - .../store/jdbc/CacheJdbcStoreExample.java | 3 - .../streaming/wordcount/CacheConfig.java | 5 - .../examples/ScalarContinuationExample.scala | 10 +- modules/aop/pom.xml | 2 +- .../aop/aspectj/GridifyAspectJAspect.java | 2 +- .../aspectj/GridifySetToSetAspectJAspect.java | 2 +- .../aspectj/GridifySetToValueAspectJAspect.java | 2 +- .../aop/spring/GridifySetToSetSpringAspect.java | 2 +- .../spring/GridifySetToValueSpringAspect.java | 2 +- .../gridify/aop/spring/GridifySpringAspect.java | 2 +- modules/aws/pom.xml | 2 +- .../spi/checkpoint/s3/S3CheckpointSpi.java | 2 +- .../s3/S3CheckpointManagerSelfTest.java | 2 +- .../checkpoint/s3/S3CheckpointSpiSelfTest.java | 2 +- .../s3/S3SessionCheckpointSelfTest.java | 2 +- .../config/grid-client-config.properties | 50 +- modules/clients/pom.xml | 2 +- .../ClientPropertiesConfigurationSelfTest.java | 12 +- .../clients/src/test/resources/spring-cache.xml | 4 +- .../src/test/resources/spring-server-node.xml | 4 +- .../test/resources/spring-server-ssl-node.xml | 4 +- modules/cloud/README.txt | 32 + modules/cloud/licenses/apache-2.0.txt | 202 + modules/cloud/pom.xml | 106 + .../cloud/TcpDiscoveryCloudIpFinder.java | 433 ++ .../tcp/ipfinder/cloud/package-info.java | 21 + .../TcpDiscoveryCloudIpFinderSelfTest.java | 122 + .../tcp/ipfinder/cloud/package-info.java | 22 + .../ignite/testsuites/IgniteCloudTestSuite.java | 112 + modules/codegen/pom.xml | 14 +- .../ignite/codegen/MessageCodeGenerator.java | 30 +- modules/core/pom.xml | 2 +- .../java/org/apache/ignite/IgniteCache.java | 5 + .../org/apache/ignite/IgniteJdbcDriver.java | 81 +- .../java/org/apache/ignite/IgniteLogger.java | 8 +- .../java/org/apache/ignite/IgniteServices.java | 2 +- .../apache/ignite/IgniteSystemProperties.java | 6 - .../main/java/org/apache/ignite/Ignition.java | 46 +- .../apache/ignite/cache/CacheInterceptor.java | 9 +- .../cache/CacheServerNotFoundException.java | 12 +- .../apache/ignite/cache/CachingProvider.java | 3 + .../cache/eviction/fifo/FifoEvictionPolicy.java | 7 +- .../igfs/IgfsPerBlockLruEvictionPolicy.java | 3 +- .../cache/eviction/lru/LruEvictionPolicy.java | 5 +- .../eviction/sorted/SortedEvictionPolicy.java | 431 ++ .../sorted/SortedEvictionPolicyMBean.java | 66 + .../cache/eviction/sorted/package-info.java | 21 + .../apache/ignite/cache/query/QueryMetrics.java | 8 +- .../ignite/compute/ComputeJobContinuation.java | 2 + .../configuration/CacheConfiguration.java | 288 +- .../configuration/ConnectorConfiguration.java | 2 +- .../configuration/IgniteConfiguration.java | 445 +- .../ignite/configuration/TopologyValidator.java | 35 + .../ignite/events/CacheQueryExecutedEvent.java | 3 +- .../ignite/events/CacheQueryReadEvent.java | 3 +- .../ignite/internal/GridDirectCollection.java | 3 + .../ignite/internal/GridJobContextImpl.java | 99 +- .../ignite/internal/GridUpdateNotifier.java | 66 +- .../ignite/internal/IgniteComponentType.java | 36 +- .../org/apache/ignite/internal/IgniteEx.java | 10 +- .../apache/ignite/internal/IgniteKernal.java | 108 +- .../org/apache/ignite/internal/IgnitionEx.java | 196 +- .../ignite/internal/MarshallerContextImpl.java | 2 +- .../client/GridClientConfiguration.java | 2 +- .../connection/GridClientNioTcpConnection.java | 9 +- .../internal/cluster/ClusterGroupAdapter.java | 16 + .../ClusterTopologyServerNotFoundException.java | 12 +- .../internal/direct/DirectByteBufferStream.java | 4 +- .../internal/managers/GridManagerAdapter.java | 8 +- .../managers/communication/GridIoManager.java | 69 +- .../communication/GridIoMessageFactory.java | 12 +- .../GridLifecycleAwareMessageFilter.java | 35 + .../deployment/GridDeploymentClassLoader.java | 2 +- .../deployment/GridDeploymentManager.java | 2 +- .../GridDeploymentPerVersionStore.java | 3 +- .../discovery/GridDiscoveryManager.java | 73 +- .../eventstorage/GridEventStorageManager.java | 11 +- .../managers/indexing/GridIndexingManager.java | 14 +- .../affinity/GridAffinityAssignmentCache.java | 11 +- .../processors/cache/CacheEntryImpl.java | 29 +- .../processors/cache/CacheInvokeResult.java | 24 +- .../processors/cache/CacheLockImpl.java | 20 +- .../processors/cache/CacheObjectImpl.java | 2 +- .../processors/cache/CacheOperationContext.java | 170 + .../processors/cache/CacheProjection.java | 1386 ---- .../cache/CacheStoreBalancingWrapper.java | 6 + .../cache/CacheVersionedEntryImpl.java | 29 +- .../cache/DynamicCacheDescriptor.java | 16 +- .../internal/processors/cache/GridCache.java | 223 - .../processors/cache/GridCacheAdapter.java | 747 +- .../cache/GridCacheAffinityManager.java | 12 - .../processors/cache/GridCacheAtomicFuture.java | 7 - .../cache/GridCacheConcurrentMap.java | 24 +- .../processors/cache/GridCacheContext.java | 66 +- .../processors/cache/GridCacheEntryEx.java | 4 + .../cache/GridCacheEvictionManager.java | 13 +- .../processors/cache/GridCacheGateway.java | 119 +- .../processors/cache/GridCacheIoManager.java | 320 +- .../processors/cache/GridCacheMapEntry.java | 51 +- .../processors/cache/GridCacheMessage.java | 8 +- .../processors/cache/GridCacheMvccManager.java | 2 +- .../GridCachePartitionExchangeManager.java | 10 +- .../processors/cache/GridCacheProcessor.java | 213 +- .../processors/cache/GridCacheProjectionEx.java | 351 - .../cache/GridCacheProjectionImpl.java | 766 -- .../processors/cache/GridCacheProxy.java | 27 - .../processors/cache/GridCacheProxyImpl.java | 412 +- .../processors/cache/GridCacheReturn.java | 5 +- .../cache/GridCacheSharedContext.java | 2 +- .../processors/cache/GridCacheSwapManager.java | 250 +- .../processors/cache/GridCacheTtlManager.java | 156 +- .../processors/cache/GridCacheUtils.java | 65 +- .../processors/cache/IgniteCacheProxy.java | 462 +- .../processors/cache/IgniteInternalCache.java | 1789 +++++ .../cache/affinity/GridCacheAffinityImpl.java | 2 +- .../cache/affinity/GridCacheAffinityProxy.java | 30 +- .../CacheDataStructuresManager.java | 8 +- ...ridCacheOptimisticCheckPreparedTxFuture.java | 383 - ...idCacheOptimisticCheckPreparedTxRequest.java | 232 - ...dCacheOptimisticCheckPreparedTxResponse.java | 179 - .../distributed/GridCacheTxRecoveryFuture.java | 506 ++ .../distributed/GridCacheTxRecoveryRequest.java | 261 + .../GridCacheTxRecoveryResponse.java | 182 + .../GridDistributedCacheAdapter.java | 20 +- .../distributed/GridDistributedLockRequest.java | 99 +- .../GridDistributedTxRemoteAdapter.java | 5 +- .../dht/GridDhtAffinityAssignmentResponse.java | 21 +- .../distributed/dht/GridDhtCacheAdapter.java | 8 +- .../cache/distributed/dht/GridDhtGetFuture.java | 9 +- .../distributed/dht/GridDhtLocalPartition.java | 2 +- .../distributed/dht/GridDhtLockFuture.java | 38 +- .../distributed/dht/GridDhtLockRequest.java | 45 +- .../distributed/dht/GridDhtTopologyFuture.java | 8 + .../dht/GridDhtTransactionalCacheAdapter.java | 21 +- .../distributed/dht/GridDhtTxFinishFuture.java | 104 +- .../cache/distributed/dht/GridDhtTxLocal.java | 9 + .../distributed/dht/GridDhtTxLocalAdapter.java | 49 +- .../distributed/dht/GridDhtTxPrepareFuture.java | 18 +- .../cache/distributed/dht/GridDhtTxRemote.java | 14 +- .../dht/GridPartitionedGetFuture.java | 29 +- .../dht/atomic/GridDhtAtomicCache.java | 115 +- .../dht/atomic/GridDhtAtomicUpdateFuture.java | 19 - .../dht/atomic/GridDhtAtomicUpdateResponse.java | 8 + .../dht/atomic/GridNearAtomicUpdateFuture.java | 59 +- .../dht/atomic/GridNearAtomicUpdateRequest.java | 49 +- .../atomic/GridNearAtomicUpdateResponse.java | 18 +- .../dht/colocated/GridDhtColocatedCache.java | 61 +- .../colocated/GridDhtColocatedLockFuture.java | 44 +- .../dht/preloader/GridDhtForceKeysFuture.java | 6 + .../dht/preloader/GridDhtForceKeysResponse.java | 54 +- .../preloader/GridDhtPartitionSupplyPool.java | 2 +- .../GridDhtPartitionsExchangeFuture.java | 25 + .../dht/preloader/GridDhtPreloader.java | 2 +- .../distributed/near/GridNearAtomicCache.java | 13 +- .../distributed/near/GridNearCacheAdapter.java | 20 +- .../distributed/near/GridNearCacheEntry.java | 16 +- .../distributed/near/GridNearGetFuture.java | 37 +- .../distributed/near/GridNearGetResponse.java | 8 +- .../distributed/near/GridNearLockFuture.java | 41 +- .../distributed/near/GridNearLockRequest.java | 61 +- .../near/GridNearTransactionalCache.java | 49 +- .../near/GridNearTxFinishFuture.java | 3 + .../cache/distributed/near/GridNearTxLocal.java | 13 +- .../near/GridNearTxPrepareFuture.java | 20 + .../distributed/near/GridNearTxRemote.java | 7 +- .../processors/cache/local/GridLocalCache.java | 8 +- .../local/atomic/GridLocalAtomicCache.java | 102 +- .../processors/cache/query/CacheQueries.java | 143 - .../processors/cache/query/CacheQuery.java | 12 +- .../query/GridCacheDistributedQueryManager.java | 3 + .../cache/query/GridCacheLocalQueryFuture.java | 3 + .../cache/query/GridCacheQueriesEx.java | 68 - .../cache/query/GridCacheQueriesImpl.java | 217 - .../cache/query/GridCacheQueriesProxy.java | 285 - .../query/GridCacheQueryFutureAdapter.java | 3 + .../cache/query/GridCacheQueryManager.java | 173 +- .../cache/query/GridCacheSqlQuery.java | 137 +- .../cache/query/GridCacheTwoStepQuery.java | 25 +- .../jdbc/GridCacheQueryJdbcMetadataTask.java | 7 +- .../cache/transactions/IgniteInternalTx.java | 10 +- .../cache/transactions/IgniteTxAdapter.java | 21 +- .../cache/transactions/IgniteTxEntry.java | 74 +- .../cache/transactions/IgniteTxHandler.java | 116 +- .../transactions/IgniteTxLocalAdapter.java | 136 +- .../cache/transactions/IgniteTxLocalEx.java | 4 +- .../cache/transactions/IgniteTxManager.java | 330 +- .../IgniteCacheObjectProcessorImpl.java | 2 +- .../closure/GridClosureProcessor.java | 16 +- .../datastreamer/DataStreamProcessor.java | 28 +- .../datastreamer/DataStreamerCacheUpdaters.java | 15 +- .../datastreamer/DataStreamerImpl.java | 84 +- .../datastreamer/DataStreamerRequest.java | 38 +- .../datastreamer/DataStreamerUpdateJob.java | 16 +- .../datastructures/DataStructuresProcessor.java | 24 +- .../datastructures/GridCacheAtomicLongImpl.java | 4 +- .../GridCacheAtomicReferenceImpl.java | 4 +- .../GridCacheAtomicSequenceImpl.java | 4 +- .../GridCacheAtomicStampedImpl.java | 4 +- .../GridCacheCountDownLatchImpl.java | 4 +- .../datastructures/GridCacheSetImpl.java | 2 +- .../dr/IgniteDrDataStreamerCacheUpdater.java | 2 - .../processors/igfs/IgfsDataManager.java | 9 +- .../processors/igfs/IgfsDeleteWorker.java | 4 + .../processors/igfs/IgfsFileWorkerBatch.java | 3 + .../processors/igfs/IgfsMetaManager.java | 10 +- .../internal/processors/igfs/IgfsThread.java | 8 +- .../internal/processors/igfs/IgfsUtils.java | 11 +- .../processors/job/GridJobHoldListener.java | 6 +- .../processors/job/GridJobProcessor.java | 28 +- .../internal/processors/job/GridJobWorker.java | 32 +- .../offheap/GridOffHeapProcessor.java | 17 + .../portable/GridPortableInputStream.java | 26 - .../processors/query/GridQueryIndexing.java | 23 +- .../processors/query/GridQueryProcessor.java | 83 +- .../messages/GridQueryNextPageResponse.java | 68 +- .../h2/twostep/messages/GridQueryRequest.java | 21 +- .../processors/resource/GridResourceField.java | 11 + .../processors/resource/GridResourceIoc.java | 387 +- .../processors/resource/GridResourceMethod.java | 13 + .../resource/GridResourceProcessor.java | 4 +- .../processors/rest/GridRestCommand.java | 11 +- .../processors/rest/GridRestProcessor.java | 12 +- .../message/GridClientCacheQueryRequest.java | 366 - .../cache/GridCacheClientQueryResult.java | 97 - .../handlers/cache/GridCacheCommandHandler.java | 64 +- .../cache/GridCacheQueryCommandHandler.java | 480 -- .../top/GridTopologyCommandHandler.java | 3 +- .../protocols/tcp/GridTcpRestNioListener.java | 24 - .../rest/request/GridRestCacheQueryRequest.java | 143 - .../service/GridServiceProcessor.java | 11 +- .../processors/task/GridTaskProcessor.java | 6 +- .../processors/task/GridTaskWorker.java | 12 + .../timeout/GridTimeoutProcessor.java | 3 + .../ignite/internal/util/GridJavaProcess.java | 4 + .../ignite/internal/util/IgniteUtils.java | 41 +- .../util/ipc/loopback/IpcServerTcpEndpoint.java | 2 +- .../shmem/IpcSharedMemoryServerEndpoint.java | 2 +- .../util/lang/GridFilteredIterator.java | 2 +- .../ignite/internal/util/lang/GridFunc.java | 7218 +++++------------- .../ignite/internal/util/nio/GridNioServer.java | 6 + .../util/offheap/GridOffHeapPartitionedMap.java | 9 + .../unsafe/GridUnsafePartitionedMap.java | 155 +- .../util/spring/IgniteSpringHelper.java | 56 +- .../util/tostring/GridToStringBuilder.java | 2 +- .../apache/ignite/internal/util/typedef/X.java | 2 +- .../ignite/internal/util/worker/GridWorker.java | 3 + .../ignite/internal/visor/cache/VisorCache.java | 92 +- .../visor/cache/VisorCacheConfiguration.java | 7 - .../visor/cache/VisorCacheMetadataTask.java | 9 +- .../internal/visor/cache/VisorCacheMetrics.java | 57 +- .../cache/VisorCacheNearConfiguration.java | 4 +- .../visor/cache/VisorCacheNodesTask.java | 74 + .../visor/cache/VisorCacheRebalanceTask.java | 4 +- .../visor/cache/VisorCacheResetMetricsTask.java | 2 +- .../visor/cache/VisorCacheStartTask.java | 155 + .../cache/VisorCacheStoreConfiguration.java | 35 - .../visor/cache/VisorCacheSwapBackupsTask.java | 2 +- .../cache/VisorCacheTypeFieldMetadata.java | 36 +- .../visor/cache/VisorCacheTypeMetadata.java | 99 +- .../internal/visor/igfs/VisorIgfsMetrics.java | 128 +- .../visor/misc/VisorResolveHostNameTask.java | 2 +- .../visor/node/VisorBasicConfiguration.java | 11 + .../visor/node/VisorNodeDataCollectorJob.java | 8 +- .../node/VisorNodeEventsCollectorTask.java | 58 +- .../internal/visor/query/VisorQueryArg.java | 31 +- .../internal/visor/query/VisorQueryCursor.java | 1 - .../internal/visor/query/VisorQueryJob.java | 11 +- .../internal/visor/query/VisorQueryTask.java | 41 - .../internal/visor/util/VisorEventMapper.java | 13 + .../internal/visor/util/VisorTaskUtils.java | 12 +- .../apache/ignite/lang/IgniteAsyncSupport.java | 4 +- .../apache/ignite/logger/java/JavaLogger.java | 12 +- .../apache/ignite/marshaller/Marshaller.java | 14 +- .../ignite/marshaller/jdk/JdkMarshaller.java | 10 +- .../optimized/OptimizedMarshaller.java | 8 +- .../ignite/messaging/MessagingListenActor.java | 3 + .../apache/ignite/resources/LoggerResource.java | 2 +- .../apache/ignite/resources/SpringResource.java | 2 +- .../org/apache/ignite/services/Service.java | 2 +- .../ignite/services/ServiceConfiguration.java | 2 +- .../org/apache/ignite/spi/IgniteSpiAdapter.java | 24 +- .../org/apache/ignite/spi/IgniteSpiContext.java | 6 + .../org/apache/ignite/spi/IgniteSpiThread.java | 3 + .../checkpoint/cache/CacheCheckpointSpi.java | 2 +- .../spi/checkpoint/jdbc/JdbcCheckpointSpi.java | 2 +- .../sharedfs/SharedFsCheckpointSpi.java | 4 +- .../fifoqueue/FifoQueueCollisionSpi.java | 10 +- .../jobstealing/JobStealingCollisionSpi.java | 14 +- .../PriorityQueueCollisionSpi.java | 6 +- .../communication/tcp/TcpCommunicationSpi.java | 82 +- .../tcp/TcpCommunicationSpiMBean.java | 9 - .../ignite/spi/discovery/DiscoverySpi.java | 7 + .../discovery/tcp/TcpClientDiscoverySpi.java | 16 +- .../spi/discovery/tcp/TcpDiscoverySpi.java | 95 +- .../discovery/tcp/TcpDiscoverySpiAdapter.java | 18 +- .../memory/MemoryEventStorageSpi.java | 10 +- .../spi/failover/always/AlwaysFailoverSpi.java | 10 +- .../jobstealing/JobStealingFailoverSpi.java | 6 +- .../spi/failover/never/NeverFailoverSpi.java | 8 +- .../apache/ignite/spi/indexing/IndexingSpi.java | 4 +- .../adaptive/AdaptiveLoadBalancingSpi.java | 12 +- .../roundrobin/RoundRobinLoadBalancingSpi.java | 10 +- .../WeightedRandomLoadBalancingSpi.java | 10 +- .../spi/swapspace/file/FileSwapSpaceSpi.java | 10 +- .../ignite/startup/BasicWarmupClosure.java | 20 +- .../startup/cmdline/CommandLineStartup.java | 5 +- .../startup/cmdline/CommandLineTransformer.java | 3 + .../TransactionSynchronization.java | 45 - .../resources/META-INF/classnames.properties | 64 +- .../core/src/main/resources/ignite.properties | 2 +- .../src/test/config/load/merge-sort-base.xml | 2 +- .../internal/GridCacheProjectionRemoveTest.java | 41 - .../internal/GridContinuousTaskSelfTest.java | 114 + .../internal/GridDiscoveryEventSelfTest.java | 30 +- ...ridFailFastNodeFailureDetectionSelfTest.java | 117 + .../internal/GridLifecycleBeanSelfTest.java | 36 + .../internal/GridUpdateNotifierSelfTest.java | 30 +- .../internal/IgniteInternalCacheRemoveTest.java | 41 + .../GridDiscoveryManagerAliveCacheSelfTest.java | 6 +- .../processors/cache/CacheGetFromJobTest.java | 110 + .../GridCacheAbstractFailoverSelfTest.java | 48 +- .../cache/GridCacheAbstractFullApiSelfTest.java | 731 +- .../cache/GridCacheAbstractSelfTest.java | 4 +- .../GridCacheConcurrentTxMultiNodeTest.java | 2 +- ...CacheFullTextQueryMultithreadedSelfTest.java | 4 +- .../GridCachePreloadingEvictionsSelfTest.java | 4 +- .../cache/GridCachePutAllFailoverSelfTest.java | 2 +- .../processors/cache/GridCachePutAllTask.java | 56 +- .../cache/GridCacheSwapReloadSelfTest.java | 20 +- .../processors/cache/GridCacheTestEntryEx.java | 2 + ...ProjectionForCachesOnDaemonNodeSelfTest.java | 147 + .../IgniteCacheEntryListenerAbstractTest.java | 4 +- .../IgniteCacheP2pUnmarshallingErrorTest.java | 189 + ...gniteCacheP2pUnmarshallingNearErrorTest.java | 56 + ...CacheP2pUnmarshallingRebalanceErrorTest.java | 80 + .../IgniteCacheP2pUnmarshallingTxErrorTest.java | 109 + .../cache/IgniteCachePeekModesAbstractTest.java | 15 +- .../cache/IgniteCachePutAllRestartTest.java | 203 + .../cache/IgniteCacheTxPreloadNoWriteTest.java | 29 +- .../cache/IgniteDynamicCacheStartSelfTest.java | 4 +- .../cache/IgniteExchangeFutureHistoryTest.java | 77 + ...gniteTopologyValidatorAbstractCacheTest.java | 183 + ...iteTopologyValidatorAbstractTxCacheTest.java | 125 + ...ValidatorNearPartitionedAtomicCacheTest.java | 30 + ...logyValidatorNearPartitionedTxCacheTest.java | 30 + ...logyValidatorPartitionedAtomicCacheTest.java | 49 + ...TopologyValidatorPartitionedTxCacheTest.java | 30 + ...ologyValidatorReplicatedAtomicCacheTest.java | 49 + ...eTopologyValidatorReplicatedTxCacheTest.java | 30 + .../cache/IgniteTxMultiNodeAbstractTest.java | 31 +- .../cache/OffHeapTieredTransactionSelfTest.java | 127 + ...CacheLoadingConcurrentGridStartSelfTest.java | 163 + .../CacheNoValueClassOnServerNodeTest.java | 129 + .../GridCacheAbstractNodeRestartSelfTest.java | 94 +- .../GridCacheAtomicTimeoutSelfTest.java | 314 - ...GridCacheLoadingConcurrentGridStartTest.java | 154 - .../distributed/GridCacheLockAbstractTest.java | 75 + ...ridCachePartitionNotLoadedEventSelfTest.java | 22 +- .../IgniteCacheAtomicMessageRecoveryTest.java | 32 + .../IgniteCacheMessageRecoveryAbstractTest.java | 175 + .../IgniteCachePutGetRestartAbstractTest.java | 234 + .../IgniteCacheTxFairAffinityNodeJoinTest.java | 35 + .../IgniteCacheTxMessageRecoveryTest.java | 32 + ...arDisabledFairAffinityPutGetRestartTest.java | 35 + ...iteCacheTxNearDisabledPutGetRestartTest.java | 30 + ...xOriginatingNodeFailureAbstractSelfTest.java | 8 +- ...cOriginatingNodeFailureAbstractSelfTest.java | 7 +- .../dht/GridCacheDhtPreloadSelfTest.java | 6 +- ...icOffHeapTieredMultiNodeFullApiSelfTest.java | 43 + ...ledFairAffinityMultiNodeFullApiSelfTest.java | 36 + ...ionedNearDisabledOffHeapFullApiSelfTest.java | 8 +- ...DisabledOffHeapMultiNodeFullApiSelfTest.java | 8 +- ...abledOffHeapTieredAtomicFullApiSelfTest.java | 56 + ...earDisabledOffHeapTieredFullApiSelfTest.java | 33 + ...edOffHeapTieredMultiNodeFullApiSelfTest.java | 33 + ...itionedTxOriginatingNodeFailureSelfTest.java | 2 - .../dht/IgniteCacheMultiTxLockSelfTest.java | 2 +- ...rDisabledPrimaryNodeFailureRecoveryTest.java | 31 + ...rtitionedPrimaryNodeFailureRecoveryTest.java | 31 + ...woBackupsPrimaryNodeFailureRecoveryTest.java | 37 + ...ePrimaryNodeFailureRecoveryAbstractTest.java | 533 ++ ...eAtomicInvalidPartitionHandlingSelfTest.java | 14 +- ...nlyFairAffinityMultiNodeFullApiSelfTest.java | 36 + ...micFairAffinityMultiNodeFullApiSelfTest.java | 35 + ...ledFairAffinityMultiNodeFullApiSelfTest.java | 36 + ...CacheAtomicOffHeapTieredFullApiSelfTest.java | 32 + ...icOffHeapTieredMultiNodeFullApiSelfTest.java | 33 + ...derFairAffinityMultiNodeFullApiSelfTest.java | 36 + ...yWriteOrderOffHeapTieredFullApiSelfTest.java | 33 + ...erOffHeapTieredMultiNodeFullApiSelfTest.java | 33 + ...nlyFairAffinityMultiNodeFullApiSelfTest.java | 35 + ...idCacheNearOnlyMultiNodeFullApiSelfTest.java | 1 + ...achePartitionedMultiNodeFullApiSelfTest.java | 15 +- .../GridCachePartitionedNodeRestartTest.java | 4 +- ...dCachePartitionedOffHeapFullApiSelfTest.java | 8 +- ...titionedOffHeapMultiNodeFullApiSelfTest.java | 8 +- ...PartitionedOffHeapTieredFullApiSelfTest.java | 32 + ...edOffHeapTieredMultiNodeFullApiSelfTest.java | 72 + ...ePartitionedOptimisticTxNodeRestartTest.java | 4 +- ...achePartitionedPreloadLifecycleSelfTest.java | 2 +- .../near/IgniteCacheNearTxRollbackTest.java | 133 + ...nedFairAffinityMultiNodeFullApiSelfTest.java | 37 + .../GridCacheReplicatedNodeRestartSelfTest.java | 2 + ...idCacheReplicatedOffHeapFullApiSelfTest.java | 8 +- ...plicatedOffHeapMultiNodeFullApiSelfTest.java | 8 +- ...eReplicatedOffHeapTieredFullApiSelfTest.java | 33 + ...edOffHeapTieredMultiNodeFullApiSelfTest.java | 33 + ...dezvousAffinityMultiNodeFullApiSelfTest.java | 35 - ...CacheReplicatedPreloadLifecycleSelfTest.java | 2 +- ...dCacheSortedBatchEvictionPolicySelfTest.java | 385 + ...acheSortedEvictionPolicyPerformanceTest.java | 135 + .../GridCacheSortedEvictionPolicySelfTest.java | 373 + .../IgniteCacheExpiryPolicyAbstractTest.java | 2 +- .../IgniteCacheExpiryPolicyTestSuite.java | 2 + .../expiry/IgniteCacheTtlCleanupSelfTest.java | 85 + ...LocalAtomicOffHeapTieredFullApiSelfTest.java | 32 + .../GridCacheLocalIsolatedNodesSelfTest.java | 102 + .../GridCacheLocalOffHeapFullApiSelfTest.java | 6 +- ...dCacheLocalOffHeapTieredFullApiSelfTest.java | 32 + .../GridCacheSwapScanQueryAbstractSelfTest.java | 26 +- .../DataStreamerMultiThreadedSelfTest.java | 112 + .../igfs/IgfsClientCacheSelfTest.java | 132 + .../processors/igfs/IgfsOneClientNodeTest.java | 133 + .../processors/igfs/IgfsSizeSelfTest.java | 2 +- .../processors/igfs/IgfsStreamsSelfTest.java | 4 +- .../cache/GridCacheCommandHandlerSelfTest.java | 10 +- .../GridServiceReassignmentSelfTest.java | 2 +- ...idFileSwapSpaceSpiMultithreadedLoadTest.java | 4 +- .../GridContinuousOperationsLoadTest.java | 3 +- .../logger/java/IgniteJavaLoggerTest.java | 65 - .../ignite/logger/java/JavaLoggerTest.java | 65 + .../GridTcpCommunicationSpiConfigSelfTest.java | 1 - .../tcp/TcpClientDiscoverySelfTest.java | 8 + .../spi/discovery/tcp/TcpDiscoverySelfTest.java | 70 +- .../GridSwapSpaceSpiAbstractSelfTest.java | 2 +- .../testframework/GridSpiTestContext.java | 5 + .../testframework/junits/GridAbstractTest.java | 15 + .../junits/common/GridCommonAbstractTest.java | 36 +- .../junits/logger/GridTestLog4jLogger.java | 10 +- .../ignite/testsuites/IgniteBasicTestSuite.java | 5 + .../IgniteCacheEvictionSelfTestSuite.java | 3 + .../IgniteCacheFailoverTestSuite.java | 14 +- .../IgniteCacheFullApiSelfTestSuite.java | 27 +- ...gniteCacheP2pUnmarshallingErrorTestSuit.java | 41 + .../testsuites/IgniteCacheRestartTestSuite.java | 10 +- .../ignite/testsuites/IgniteCacheTestSuite.java | 292 +- .../testsuites/IgniteCacheTestSuite2.java | 141 + .../testsuites/IgniteCacheTestSuite3.java | 140 + .../testsuites/IgniteCacheTestSuite4.java | 131 + .../IgniteCacheTxRecoverySelfTestSuite.java | 4 + .../ignite/testsuites/IgniteIgfsTestSuite.java | 3 + .../testsuites/IgniteLoggingSelfTestSuite.java | 2 +- .../IgniteTopologyValidatorTestSuit.java | 43 + .../ignite/util/TestTcpCommunicationSpi.java | 54 + modules/extdata/p2p/pom.xml | 2 +- .../CacheNoValueClassOnServerTestClient.java | 88 + .../apache/ignite/tests/p2p/cache/Person.java | 42 + .../CacheConfigurationP2PTestClient.java | 1 - modules/extdata/uri/pom.xml | 2 +- modules/gce/README.txt | 32 + modules/gce/licenses/apache-2.0.txt | 202 + modules/gce/pom.xml | 92 + .../gce/TcpDiscoveryGoogleStorageIpFinder.java | 380 + .../tcp/ipfinder/gce/package-info.java | 22 + ...pDiscoveryGoogleStorageIpFinderSelfTest.java | 73 + .../tcp/ipfinder/gce/package-info.java | 22 + .../ignite/testsuites/IgniteGCETestSuite.java | 71 + modules/geospatial/pom.xml | 2 +- .../query/h2/GridH2IndexingGeoSelfTest.java | 52 +- modules/hadoop/pom.xml | 2 +- .../processors/hadoop/HadoopDefaultJobInfo.java | 3 + .../processors/hadoop/HadoopProcessor.java | 2 +- .../processors/hadoop/igfs/HadoopIgfsIpcIo.java | 3 + .../hadoop/jobtracker/HadoopJobTracker.java | 19 +- .../hadoop/taskexecutor/HadoopRunnableTask.java | 3 + .../external/HadoopExternalTaskExecutor.java | 3 + .../processors/hadoop/v2/HadoopV2Job.java | 11 +- .../hadoop/v2/HadoopV2TaskContext.java | 9 + .../HadoopDefaultMapReducePlannerSelfTest.java | 8 +- modules/hibernate/pom.xml | 2 +- .../HibernateAccessStrategyAdapter.java | 10 +- .../hibernate/HibernateCollectionRegion.java | 2 +- .../cache/hibernate/HibernateEntityRegion.java | 2 +- .../hibernate/HibernateGeneralDataRegion.java | 2 +- .../hibernate/HibernateNaturalIdRegion.java | 2 +- .../HibernateNonStrictAccessStrategy.java | 4 +- .../hibernate/HibernateQueryResultsRegion.java | 2 +- .../HibernateReadOnlyAccessStrategy.java | 2 +- .../HibernateReadWriteAccessStrategy.java | 2 +- .../ignite/cache/hibernate/HibernateRegion.java | 4 +- .../cache/hibernate/HibernateRegionFactory.java | 10 +- .../hibernate/HibernateTimestampsRegion.java | 2 +- .../HibernateTransactionalAccessStrategy.java | 4 +- .../HibernateTransactionalDataRegion.java | 2 +- .../HibernateL2CacheConfigurationSelfTest.java | 2 +- .../hibernate/HibernateL2CacheSelfTest.java | 2 +- modules/indexing/pom.xml | 2 +- .../processors/query/h2/IgniteH2Indexing.java | 186 +- .../query/h2/opt/GridH2AbstractKeyValueRow.java | 92 +- .../query/h2/opt/GridH2KeyValueRowOffheap.java | 7 +- .../query/h2/opt/GridH2KeyValueRowOnheap.java | 6 +- .../query/h2/opt/GridH2RowDescriptor.java | 14 +- .../processors/query/h2/opt/GridH2Table.java | 10 +- .../query/h2/opt/GridH2ValueCacheObject.java | 191 + .../query/h2/opt/GridLuceneIndex.java | 84 +- .../query/h2/sql/GridSqlFunction.java | 6 +- .../query/h2/sql/GridSqlPlaceholder.java | 51 + .../processors/query/h2/sql/GridSqlQuery.java | 20 + .../query/h2/sql/GridSqlQueryParser.java | 16 +- .../query/h2/sql/GridSqlQuerySplitter.java | 11 +- .../processors/query/h2/sql/GridSqlSelect.java | 2 +- .../processors/query/h2/sql/GridSqlUnion.java | 2 +- .../query/h2/twostep/GridMapQueryExecutor.java | 61 +- .../query/h2/twostep/GridMergeIndex.java | 6 +- .../h2/twostep/GridMergeIndexUnsorted.java | 4 +- .../h2/twostep/GridReduceQueryExecutor.java | 162 +- .../query/h2/twostep/GridResultPage.java | 80 +- .../query/h2/twostep/msg/GridH2Array.java | 124 + .../query/h2/twostep/msg/GridH2Boolean.java | 112 + .../query/h2/twostep/msg/GridH2Byte.java | 113 + .../query/h2/twostep/msg/GridH2Bytes.java | 113 + .../query/h2/twostep/msg/GridH2CacheObject.java | 148 + .../query/h2/twostep/msg/GridH2Date.java | 115 + .../query/h2/twostep/msg/GridH2Decimal.java | 134 + .../query/h2/twostep/msg/GridH2Double.java | 113 + .../query/h2/twostep/msg/GridH2Float.java | 113 + .../query/h2/twostep/msg/GridH2Geometry.java | 134 + .../query/h2/twostep/msg/GridH2Integer.java | 113 + .../query/h2/twostep/msg/GridH2JavaObject.java | 113 + .../query/h2/twostep/msg/GridH2Long.java | 113 + .../query/h2/twostep/msg/GridH2Null.java | 78 + .../query/h2/twostep/msg/GridH2Short.java | 113 + .../query/h2/twostep/msg/GridH2String.java | 115 + .../query/h2/twostep/msg/GridH2Time.java | 116 + .../query/h2/twostep/msg/GridH2Timestamp.java | 133 + .../query/h2/twostep/msg/GridH2Uuid.java | 133 + .../h2/twostep/msg/GridH2ValueMessage.java | 49 + .../twostep/msg/GridH2ValueMessageFactory.java | 201 + .../GridCacheAbstractFieldsQuerySelfTest.java | 1282 ---- .../cache/GridCacheCrossCacheQuerySelfTest.java | 68 +- .../cache/GridCacheOffHeapAndSwapSelfTest.java | 11 +- .../cache/GridCacheOffHeapSelfTest.java | 11 +- .../cache/GridCacheOffheapIndexGetSelfTest.java | 111 + .../GridCacheQuerySerializationSelfTest.java | 144 + ...idCacheReduceQueryMultithreadedSelfTest.java | 10 +- .../cache/GridIndexingWithNoopSwapSelfTest.java | 17 +- .../IgniteCacheAbstractFieldsQuerySelfTest.java | 256 +- ...niteCacheP2pUnmarshallingQueryErrorTest.java | 56 + ...eQueryMultiThreadedOffHeapTiredSelfTest.java | 37 + .../IgniteCacheQueryMultiThreadedSelfTest.java | 39 +- ...artitionedFieldsQueryP2PEnabledSelfTest.java | 34 - ...GridCachePartitionedFieldsQuerySelfTest.java | 115 - ...rtitionedFieldsQueryP2PDisabledSelfTest.java | 34 - ...artitionedFieldsQueryP2PEnabledSelfTest.java | 34 + ...eReplicatedFieldsQueryP2PEnableSelfTest.java | 34 - .../GridCacheReplicatedFieldsQuerySelfTest.java | 157 - ...eplicatedFieldsQueryP2PDisabledSelfTest.java | 34 - ...ReplicatedFieldsQueryP2PEnabledSelfTest.java | 34 + ...dCacheAbstractReduceFieldsQuerySelfTest.java | 11 +- ...cheReduceFieldsQueryPartitionedSelfTest.java | 5 +- .../query/h2/GridH2IndexRebuildTest.java | 18 +- .../h2/GridIndexingSpiAbstractSelfTest.java | 132 +- .../h2/sql/AbstractH2CompareQueryTest.java | 49 +- .../query/h2/sql/BaseH2CompareQueryTest.java | 32 +- .../query/h2/sql/GridQueryParsingTest.java | 9 + .../query/h2/sql/H2CompareBigQueryTest.java | 2 +- .../IgniteCacheQuerySelfTestSuite.java | 12 +- .../IgniteCacheWithIndexingTestSuite.java | 2 + .../processors/query/h2/sql/bigQuery.sql | 2 +- modules/jcl/pom.xml | 2 +- .../ignite/logger/jcl/IgniteJclLogger.java | 167 - .../org/apache/ignite/logger/jcl/JclLogger.java | 167 + .../ignite/logger/jcl/IgniteJclLoggerTest.java | 48 - .../apache/ignite/logger/jcl/JclLoggerTest.java | 48 + .../ignite/testsuites/IgniteJclTestSuite.java | 2 +- modules/jta/pom.xml | 2 +- modules/log4j/pom.xml | 2 +- .../apache/ignite/logger/log4j/Log4JLogger.java | 8 +- modules/rest-http/pom.xml | 2 +- .../http/jetty/GridJettyRestHandler.java | 3 + modules/scalar/pom.xml | 2 +- .../ignite/scalar/ScalarConversions.scala | 8 - modules/schedule/pom.xml | 2 +- modules/schema-import/pom.xml | 8 +- .../ignite/schema/generator/CodeGenerator.java | 105 +- .../ignite/schema/ui/SchemaImportApp.java | 8 +- modules/slf4j/pom.xml | 2 +- .../ignite/logger/slf4j/GridSlf4jLogger.java | 138 - .../apache/ignite/logger/slf4j/Slf4jLogger.java | 138 + modules/spring/pom.xml | 2 +- .../util/spring/IgniteSpringHelperImpl.java | 217 +- .../ignite/internal/GridFactorySelfTest.java | 3 +- .../IgniteStartFromStreamConfigurationTest.java | 50 + .../testsuites/IgniteSpringTestSuite.java | 2 + modules/ssh/pom.xml | 2 +- ...gniteProjectionStartStopRestartSelfTest.java | 26 +- modules/tools/pom.xml | 2 +- modules/urideploy/pom.xml | 2 +- .../uri/GridUriDeploymentClassLoader.java | 4 +- .../spi/deployment/uri/UriDeploymentSpi.java | 2 +- modules/visor-console/pom.xml | 6 +- .../ignite/visor/commands/VisorConsole.scala | 316 +- .../visor/commands/VisorConsoleCommand.scala | 77 - .../ignite/visor/commands/VisorTextTable.scala | 539 -- .../visor/commands/ack/VisorAckCommand.scala | 42 +- .../commands/alert/VisorAlertCommand.scala | 35 +- .../commands/cache/VisorCacheClearCommand.scala | 51 +- .../commands/cache/VisorCacheCommand.scala | 36 +- .../commands/cache/VisorCacheScanCommand.scala | 60 +- .../commands/cache/VisorCacheStopCommand.scala | 30 +- .../commands/cache/VisorCacheSwapCommand.scala | 66 +- .../commands/common/VisorConsoleCommand.scala | 90 + .../visor/commands/common/VisorTextTable.scala | 543 ++ .../config/VisorConfigurationCommand.scala | 438 +- .../commands/deploy/VisorDeployCommand.scala | 47 +- .../commands/disco/VisorDiscoveryCommand.scala | 58 +- .../commands/events/VisorEventsCommand.scala | 338 +- .../visor/commands/gc/VisorGcCommand.scala | 130 +- .../visor/commands/kill/VisorKillCommand.scala | 53 +- .../visor/commands/node/VisorNodeCommand.scala | 47 +- .../visor/commands/ping/VisorPingCommand.scala | 41 +- .../commands/start/VisorStartCommand.scala | 34 +- .../commands/tasks/VisorTasksCommand.scala | 76 +- .../commands/top/VisorTopologyCommand.scala | 36 +- .../visor/commands/vvm/VisorVvmCommand.scala | 32 +- .../scala/org/apache/ignite/visor/visor.scala | 286 +- .../ignite/visor/VisorTextTableSpec.scala | 3 +- modules/visor-plugins/pom.xml | 2 +- modules/web/pom.xml | 2 +- modules/yardstick/pom.xml | 2 +- .../cache/IgniteCacheAbstractBenchmark.java | 2 +- .../jdbc/IgniteJdbcStoreAbstractBenchmark.java | 4 +- parent/pom.xml | 2 + pom.xml | 329 +- 659 files changed, 28942 insertions(+), 20716 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8796bc59/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java ---------------------------------------------------------------------- diff --cc modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java index e8f107c,e66b32d..e3f63a1 --- a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java @@@ -17,11 -17,12 +17,11 @@@ package org.apache.ignite.cache.query; -import org.apache.ignite.internal.processors.cache.query.*; ++import org.apache.ignite.internal.processors.cache.query.CacheQuery; + /** -- * Cache query metrics used to obtain statistics on query. You can get metrics for - * particular query via {@link org.apache.ignite.internal.processors.cache.query.CacheQuery#metrics()} method - * or accumulated metrics for all queries via - * {@link org.apache.ignite.internal.processors.cache.query.CacheQueries#metrics()}. - * particular query via {@link CacheQuery#metrics()} method or accumulated metrics - * for all queries via {@link GridCacheQueryManager#metrics()}. ++ * Cache query metrics used to obtain statistics on query. You can get metrics for particular query ++ * via {@link CacheQuery#metrics()} method or accumulated metrics for all queries via {@link CacheQuery#metrics()}. */ public interface QueryMetrics { /** http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8796bc59/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java ----------------------------------------------------------------------