Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id D0F1618D5B for ; Fri, 26 Jun 2015 08:31:48 +0000 (UTC) Received: (qmail 10124 invoked by uid 500); 26 Jun 2015 08:31:48 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 10094 invoked by uid 500); 26 Jun 2015 08:31:48 -0000 Mailing-List: contact commits-help@ignite.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.incubator.apache.org Delivered-To: mailing list commits@ignite.incubator.apache.org Received: (qmail 10084 invoked by uid 99); 26 Jun 2015 08:31:48 -0000 Received: from Unknown (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 26 Jun 2015 08:31:48 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 1D0B1D0672 for ; Fri, 26 Jun 2015 08:31:48 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 0.373 X-Spam-Level: X-Spam-Status: No, score=0.373 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, KAM_LAZY_DOMAIN_SECURITY=1, RP_MATCHES_RCVD=-1.428, URIBL_BLOCKED=0.001] autolearn=disabled Received: from mx1-us-west.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id BmGod5uLRLhG for ; Fri, 26 Jun 2015 08:31:37 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-us-west.apache.org (ASF Mail Server at mx1-us-west.apache.org) with SMTP id 75B8A2092B for ; Fri, 26 Jun 2015 08:31:37 +0000 (UTC) Received: (qmail 1589 invoked by uid 99); 26 Jun 2015 08:30:22 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 26 Jun 2015 08:30:22 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 2B495DFE11; Fri, 26 Jun 2015 08:30:22 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sboikov@apache.org To: commits@ignite.incubator.apache.org Date: Fri, 26 Jun 2015 08:30:23 -0000 Message-Id: In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [2/2] incubator-ignite git commit: Merge branch 'ignite-sprint-7' into ignite-1006 Merge branch 'ignite-sprint-7' into ignite-1006 Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/a8232be6 Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/a8232be6 Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/a8232be6 Branch: refs/heads/ignite-1006 Commit: a8232be601851ca9658502de12f38ac86c1e6758 Parents: c6f66c6 3e8ddb4 Author: ashutak Authored: Thu Jun 25 21:12:09 2015 +0300 Committer: ashutak Committed: Thu Jun 25 21:12:09 2015 +0300 ---------------------------------------------------------------------- DEVNOTES.txt | 21 + RELEASE_NOTES.txt | 12 + assembly/dependencies-fabric.xml | 1 + examples/config/example-cache.xml | 2 + examples/pom.xml | 36 +- idea/ignite_codeStyle.xml | 147 ++++ modules/aop/pom.xml | 2 +- modules/aws/pom.xml | 2 +- .../s3/S3CheckpointManagerSelfTest.java | 2 +- .../checkpoint/s3/S3CheckpointSpiSelfTest.java | 4 +- .../s3/S3CheckpointSpiStartStopSelfTest.java | 2 +- .../s3/S3SessionCheckpointSelfTest.java | 2 +- .../s3/TcpDiscoveryS3IpFinderSelfTest.java | 2 +- modules/clients/pom.xml | 2 +- .../ClientAbstractConnectivitySelfTest.java | 4 +- .../client/router/TcpSslRouterSelfTest.java | 5 + .../client/suite/IgniteClientTestSuite.java | 3 +- modules/cloud/pom.xml | 2 +- .../cloud/TcpDiscoveryCloudIpFinder.java | 25 +- .../TcpDiscoveryCloudIpFinderSelfTest.java | 3 +- modules/codegen/pom.xml | 2 +- modules/core/pom.xml | 3 +- .../apache/ignite/IgniteSystemProperties.java | 6 + .../apache/ignite/cache/query/ScanQuery.java | 48 +- .../cache/store/jdbc/CacheJdbcBlobStore.java | 22 +- .../store/jdbc/CacheJdbcBlobStoreFactory.java | 290 +++++++ .../cache/store/jdbc/CacheJdbcPojoStore.java | 6 +- .../store/jdbc/CacheJdbcPojoStoreFactory.java | 148 ++++ .../org/apache/ignite/cluster/ClusterGroup.java | 9 + .../org/apache/ignite/cluster/ClusterNode.java | 2 + .../ignite/compute/ComputeTaskSplitAdapter.java | 2 +- .../configuration/CacheConfiguration.java | 4 +- .../configuration/IgniteReflectionFactory.java | 81 +- .../ignite/internal/GridKernalContextImpl.java | 8 +- .../ignite/internal/GridPluginContext.java | 6 + .../apache/ignite/internal/IgniteKernal.java | 13 +- .../internal/MarshallerContextAdapter.java | 48 +- .../ignite/internal/MarshallerContextImpl.java | 36 +- .../client/GridClientConfiguration.java | 2 +- .../GridClientOptimizedMarshaller.java | 26 + .../impl/GridTcpRouterNioListenerAdapter.java | 2 +- .../internal/cluster/ClusterGroupAdapter.java | 38 + .../cluster/IgniteClusterAsyncImpl.java | 5 + .../internal/interop/InteropBootstrap.java | 3 +- .../internal/interop/InteropIgnition.java | 57 +- .../internal/interop/InteropProcessor.java | 8 + .../internal/managers/GridManagerAdapter.java | 8 +- .../discovery/GridDiscoveryManager.java | 39 +- .../affinity/AffinityTopologyVersion.java | 7 - .../affinity/GridAffinityAssignmentCache.java | 5 +- .../processors/cache/GridCacheAdapter.java | 19 +- .../processors/cache/GridCacheContext.java | 2 +- .../processors/cache/GridCacheIoManager.java | 64 +- .../processors/cache/GridCacheMessage.java | 51 -- .../GridCachePartitionExchangeManager.java | 73 +- .../processors/cache/GridCacheProcessor.java | 91 ++- .../processors/cache/GridCacheSwapManager.java | 67 +- .../processors/cache/GridCacheUtils.java | 9 + .../processors/cache/IgniteCacheProxy.java | 23 +- .../processors/cache/KeyCacheObjectImpl.java | 11 +- .../processors/cache/QueryCursorImpl.java | 23 +- .../distributed/GridCacheTxRecoveryRequest.java | 26 +- .../GridCacheTxRecoveryResponse.java | 14 +- .../distributed/GridDistributedBaseMessage.java | 77 +- .../distributed/GridDistributedLockRequest.java | 54 +- .../GridDistributedLockResponse.java | 14 +- .../GridDistributedTxFinishRequest.java | 46 +- .../GridDistributedTxPrepareRequest.java | 62 +- .../GridDistributedTxPrepareResponse.java | 64 +- .../GridDistributedUnlockRequest.java | 6 +- .../distributed/dht/GridDhtLocalPartition.java | 66 +- .../distributed/dht/GridDhtLockFuture.java | 2 +- .../distributed/dht/GridDhtLockRequest.java | 72 +- .../distributed/dht/GridDhtLockResponse.java | 18 +- .../dht/GridDhtPartitionTopologyImpl.java | 4 +- .../dht/GridDhtPartitionsReservation.java | 292 +++++++ .../dht/GridDhtTransactionalCacheAdapter.java | 2 +- .../distributed/dht/GridDhtTxFinishRequest.java | 38 +- .../dht/GridDhtTxPrepareRequest.java | 54 +- .../dht/GridDhtTxPrepareResponse.java | 22 +- .../distributed/dht/GridDhtUnlockRequest.java | 6 +- .../cache/distributed/dht/GridReservable.java | 35 + .../dht/atomic/GridDhtAtomicCache.java | 9 +- .../dht/preloader/GridDhtPartitionMap.java | 26 +- .../GridDhtPartitionsExchangeFuture.java | 95 ++- .../dht/preloader/GridDhtPreloader.java | 2 +- .../distributed/near/GridNearLockRequest.java | 58 +- .../distributed/near/GridNearLockResponse.java | 26 +- .../near/GridNearTxFinishRequest.java | 26 +- .../near/GridNearTxPrepareRequest.java | 50 +- .../near/GridNearTxPrepareResponse.java | 46 +- .../distributed/near/GridNearUnlockRequest.java | 2 +- .../processors/cache/query/CacheQuery.java | 2 +- .../query/GridCacheDistributedQueryManager.java | 3 + .../cache/query/GridCacheQueryAdapter.java | 147 +++- .../cache/query/GridCacheQueryManager.java | 243 +++--- .../cache/query/GridCacheQueryRequest.java | 47 +- .../cache/query/GridCacheTwoStepQuery.java | 22 +- .../processors/cache/query/QueryCursorEx.java | 8 + .../continuous/CacheContinuousQueryHandler.java | 8 + .../cache/transactions/IgniteTxHandler.java | 5 +- .../transactions/IgniteTxLocalAdapter.java | 12 +- .../cacheobject/IgniteCacheObjectProcessor.java | 9 +- .../IgniteCacheObjectProcessorImpl.java | 12 +- .../continuous/GridContinuousProcessor.java | 15 +- .../datastreamer/DataStreamerCacheUpdaters.java | 2 +- .../datastreamer/DataStreamerImpl.java | 8 +- .../datastructures/DataStructuresProcessor.java | 129 +-- .../datastructures/GridCacheSetImpl.java | 4 +- .../dr/IgniteDrDataStreamerCacheUpdater.java | 7 +- .../processors/hadoop/HadoopJobInfo.java | 4 +- .../hadoop/counter/HadoopCounterWriter.java | 5 +- .../offheap/GridOffHeapProcessor.java | 19 +- .../processors/plugin/CachePluginManager.java | 10 +- .../plugin/IgnitePluginProcessor.java | 16 +- .../portable/GridPortableInputStream.java | 10 + .../processors/query/GridQueryIndexing.java | 18 +- .../processors/query/GridQueryProcessor.java | 129 ++- .../messages/GridQueryNextPageResponse.java | 35 +- .../h2/twostep/messages/GridQueryRequest.java | 111 ++- .../rest/client/message/GridRouterRequest.java | 18 + .../rest/client/message/GridRouterResponse.java | 18 + .../rest/protocols/tcp/GridTcpRestProtocol.java | 3 +- .../service/GridServiceProcessor.java | 2 +- .../processors/task/GridTaskProcessor.java | 23 +- .../internal/util/GridConfigurationFinder.java | 55 +- .../apache/ignite/internal/util/GridDebug.java | 48 +- .../ignite/internal/util/GridJavaProcess.java | 2 +- .../ignite/internal/util/IgniteUtils.java | 34 +- .../shmem/IpcSharedMemoryClientEndpoint.java | 2 +- .../ipc/shmem/IpcSharedMemoryNativeLoader.java | 151 +++- .../shmem/IpcSharedMemoryServerEndpoint.java | 2 +- .../util/ipc/shmem/IpcSharedMemoryUtils.java | 4 +- .../ignite/internal/util/nio/GridNioServer.java | 64 +- .../util/nio/GridShmemCommunicationClient.java | 146 ++++ .../util/spring/IgniteSpringHelper.java | 10 + .../internal/visor/VisorMultiNodeTask.java | 2 +- .../ignite/marshaller/MarshallerContext.java | 8 + .../org/apache/ignite/plugin/PluginContext.java | 6 + .../SpringApplicationContextResource.java | 4 +- .../apache/ignite/resources/SpringResource.java | 6 +- .../org/apache/ignite/spi/IgniteSpiAdapter.java | 35 +- .../org/apache/ignite/spi/IgniteSpiContext.java | 9 +- .../communication/tcp/TcpCommunicationSpi.java | 561 +++++++++++-- .../tcp/TcpCommunicationSpiMBean.java | 27 + .../ignite/spi/discovery/DiscoverySpi.java | 3 +- .../ignite/spi/discovery/tcp/ClientImpl.java | 562 ++++++++----- .../ignite/spi/discovery/tcp/ServerImpl.java | 275 ++++--- .../spi/discovery/tcp/TcpDiscoveryImpl.java | 69 +- .../spi/discovery/tcp/TcpDiscoverySpi.java | 48 +- .../tcp/internal/TcpDiscoveryNode.java | 18 + .../ipfinder/TcpDiscoveryIpFinderAdapter.java | 34 +- .../TcpDiscoveryMulticastIpFinder.java | 19 +- .../messages/TcpDiscoveryAbstractMessage.java | 10 +- .../messages/TcpDiscoveryNodeFailedMessage.java | 18 + .../java/org/jsr166/ConcurrentHashMap8.java | 8 +- .../java/org/jsr166/ConcurrentLinkedDeque8.java | 586 +++++--------- .../src/main/java/org/jsr166/LongAdder8.java | 35 +- .../core/src/main/java/org/jsr166/README.txt | 11 + .../src/main/java/org/jsr166/Striped64_8.java | 22 +- .../java/org/jsr166/ThreadLocalRandom8.java | 19 +- .../src/main/java/org/jsr166/package-info.java | 12 +- .../core/src/main/resources/ignite.properties | 2 +- modules/core/src/test/config/tests.properties | 2 +- .../ignite/GridSuppressedExceptionSelfTest.java | 4 +- .../internal/ClusterForHostsSelfTest.java | 113 +++ .../internal/ClusterGroupAbstractTest.java | 777 ++++++++++++++++++ .../ignite/internal/ClusterGroupSelfTest.java | 273 +++++++ .../internal/GridDiscoveryEventSelfTest.java | 6 +- ...ridFailFastNodeFailureDetectionSelfTest.java | 15 +- .../GridFailoverTaskWithPredicateSelfTest.java | 3 - .../GridJobMasterLeaveAwareSelfTest.java | 2 - .../internal/GridJobStealingSelfTest.java | 3 - .../internal/GridProjectionAbstractTest.java | 784 ------------------- ...ectionLocalJobMultipleArgumentsSelfTest.java | 2 - .../ignite/internal/GridProjectionSelfTest.java | 273 ------- .../apache/ignite/internal/GridSelfTest.java | 30 +- .../GridTaskExecutionContextSelfTest.java | 9 - .../GridTaskFailoverAffinityRunTest.java | 170 ++++ .../IgniteComputeEmptyClusterGroupTest.java | 3 - .../IgniteComputeTopologyExceptionTest.java | 9 - .../IgniteSlowClientDetectionSelfTest.java | 187 +++++ .../GridDiscoveryManagerAliveCacheSelfTest.java | 22 +- .../cache/CacheClientStoreSelfTest.java | 228 ++++++ ...acheReadOnlyTransactionalClientSelfTest.java | 327 -------- .../CacheReadThroughAtomicRestartSelfTest.java | 32 + ...heReadThroughLocalAtomicRestartSelfTest.java | 32 + .../CacheReadThroughLocalRestartSelfTest.java | 32 + ...dThroughReplicatedAtomicRestartSelfTest.java | 32 + ...cheReadThroughReplicatedRestartSelfTest.java | 32 + .../cache/CacheReadThroughRestartSelfTest.java | 133 ++++ .../CacheStoreUsageMultinodeAbstractTest.java | 305 ++++++++ ...eUsageMultinodeDynamicStartAbstractTest.java | 169 ++++ ...oreUsageMultinodeDynamicStartAtomicTest.java | 32 + ...heStoreUsageMultinodeDynamicStartTxTest.java | 32 + ...reUsageMultinodeStaticStartAbstractTest.java | 158 ++++ ...toreUsageMultinodeStaticStartAtomicTest.java | 32 + ...cheStoreUsageMultinodeStaticStartTxTest.java | 32 + .../GridCacheAbstractFailoverSelfTest.java | 8 +- .../cache/GridCacheAbstractFullApiSelfTest.java | 39 +- .../cache/GridCacheAbstractSelfTest.java | 5 +- .../cache/GridCacheAffinityRoutingSelfTest.java | 4 +- .../cache/GridCacheDaemonNodeStopSelfTest.java | 119 +++ .../cache/GridCacheDeploymentSelfTest.java | 3 - .../cache/GridCacheEntryMemorySizeSelfTest.java | 91 ++- .../cache/GridCacheMemoryModeSelfTest.java | 2 - ...ridCacheMultinodeUpdateAbstractSelfTest.java | 9 + ...inodeUpdateNearEnabledNoBackupsSelfTest.java | 2 +- ...CacheMultinodeUpdateNearEnabledSelfTest.java | 2 +- .../processors/cache/GridCacheOffHeapTest.java | 28 +- .../GridCacheReferenceCleanupSelfTest.java | 3 - .../processors/cache/GridCacheStopSelfTest.java | 5 + .../cache/GridCacheVersionMultinodeTest.java | 6 +- .../IgniteCacheAbstractStopBusySelfTest.java | 30 +- .../cache/IgniteCacheAbstractTest.java | 3 - .../IgniteCacheAtomicStopBusySelfTest.java | 8 +- .../IgniteCacheEntryListenerAbstractTest.java | 14 +- .../IgniteCacheInterceptorSelfTestSuite.java | 2 +- .../cache/IgniteCacheInvokeReadThroughTest.java | 5 + ...CacheP2pUnmarshallingRebalanceErrorTest.java | 15 +- .../IgniteCacheP2pUnmarshallingTxErrorTest.java | 25 +- ...gniteCacheTransactionalStopBusySelfTest.java | 13 +- ...eDynamicCacheStartNoExchangeTimeoutTest.java | 466 +++++++++++ .../cache/IgniteDynamicCacheStartSelfTest.java | 56 ++ ...teStartCacheInTransactionAtomicSelfTest.java | 32 + .../IgniteStartCacheInTransactionSelfTest.java | 254 ++++++ .../IgniteTxMultiThreadedAbstractTest.java | 4 +- ...cheAtomicReferenceMultiNodeAbstractTest.java | 11 - ...GridCacheQueueMultiNodeAbstractSelfTest.java | 6 +- ...dCacheQueueMultiNodeConsistencySelfTest.java | 5 + ...CacheQueueRotativeMultiNodeAbstractTest.java | 10 - .../GridCacheSetAbstractSelfTest.java | 31 +- .../IgniteDataStructureWithJobTest.java | 111 +++ ...omicOffheapQueueCreateMultiNodeSelfTest.java | 5 + ...ionedAtomicQueueCreateMultiNodeSelfTest.java | 5 + ...rtitionedDataStructuresFailoverSelfTest.java | 5 + ...edOffheapDataStructuresFailoverSelfTest.java | 5 + ...PartitionedQueueCreateMultiNodeSelfTest.java | 5 + ...dCachePartitionedQueueEntryMoveSelfTest.java | 5 + ...nedQueueFailoverDataConsistencySelfTest.java | 5 + ...eplicatedDataStructuresFailoverSelfTest.java | 5 + ...CacheLoadingConcurrentGridStartSelfTest.java | 5 + .../GridCacheAbstractJobExecutionTest.java | 3 - ...ridCachePartitionNotLoadedEventSelfTest.java | 82 ++ .../GridCachePreloadLifecycleAbstractTest.java | 2 - .../distributed/IgniteCache150ClientsTest.java | 189 +++++ .../IgniteCacheClientNodeConcurrentStart.java | 14 +- ...teCacheClientNodePartitionsExchangeTest.java | 1 + .../distributed/IgniteCacheManyClientsTest.java | 318 ++++++++ .../IgniteCacheMessageRecoveryAbstractTest.java | 1 + .../IgniteCacheTxMessageRecoveryTest.java | 5 + ...heAbstractTransformWriteThroughSelfTest.java | 3 - .../GridCacheColocatedTxExceptionSelfTest.java | 5 + ...ePartitionedNearDisabledMetricsSelfTest.java | 4 +- ...dCachePartitionedTopologyChangeSelfTest.java | 5 + .../near/GridCacheNearEvictionSelfTest.java | 3 - ...idCacheNearOnlyMultiNodeFullApiSelfTest.java | 5 - .../near/GridCacheNearTxExceptionSelfTest.java | 5 + ...PartitionedFullApiMultithreadedSelfTest.java | 5 + ...idCachePartitionedHitsAndMissesSelfTest.java | 3 - ...achePartitionedMultiNodeFullApiSelfTest.java | 49 +- .../GridCachePartitionedNodeRestartTest.java | 5 + ...ePartitionedOptimisticTxNodeRestartTest.java | 5 + ...achePartitionedPreloadLifecycleSelfTest.java | 2 +- ...CachePartitionedTxMultiThreadedSelfTest.java | 5 + .../GridCachePartitionedTxSalvageSelfTest.java | 37 +- .../GridCacheReplicatedFailoverSelfTest.java | 5 + ...eReplicatedFullApiMultithreadedSelfTest.java | 5 + .../GridCacheReplicatedInvalidateSelfTest.java | 4 +- ...ridCacheReplicatedMultiNodeLockSelfTest.java | 5 + .../GridCacheReplicatedMultiNodeSelfTest.java | 5 + .../GridCacheReplicatedNodeRestartSelfTest.java | 5 + .../GridCacheReplicatedTxExceptionSelfTest.java | 5 + .../replicated/GridReplicatedTxPreloadTest.java | 2 + ...acheAtomicReplicatedNodeRestartSelfTest.java | 5 + ...CacheReplicatedPreloadLifecycleSelfTest.java | 6 +- .../GridCacheEvictionFilterSelfTest.java | 4 +- ...cheSynchronousEvictionsFailoverSelfTest.java | 5 + .../IgniteCacheExpiryPolicyAbstractTest.java | 10 +- ...eCacheExpiryPolicyWithStoreAbstractTest.java | 4 +- .../IgniteCacheTxStoreSessionTest.java | 4 + ...dCacheLocalFullApiMultithreadedSelfTest.java | 5 + .../GridCacheLocalTxExceptionSelfTest.java | 5 + .../GridCacheSwapScanQueryAbstractSelfTest.java | 115 ++- ...ridCacheContinuousQueryAbstractSelfTest.java | 2 - ...CacheClientWriteBehindStoreAbstractTest.java | 104 +++ ...teCacheClientWriteBehindStoreAtomicTest.java | 38 + .../IgnteCacheClientWriteBehindStoreTxTest.java | 32 + .../closure/GridClosureProcessorSelfTest.java | 29 +- .../continuous/GridEventConsumeSelfTest.java | 2 - .../DataStreamProcessorSelfTest.java | 47 +- .../DataStreamerMultiThreadedSelfTest.java | 3 + .../processors/igfs/IgfsModesSelfTest.java | 4 +- .../internal/util/IgniteUtilsSelfTest.java | 22 + .../ipc/shmem/IgfsSharedMemoryTestServer.java | 2 + .../IpcSharedMemoryCrashDetectionSelfTest.java | 2 +- .../ipc/shmem/IpcSharedMemorySpaceSelfTest.java | 2 +- .../ipc/shmem/IpcSharedMemoryUtilsSelfTest.java | 2 +- .../LoadWithCorruptedLibFileTestRunner.java | 2 +- .../IpcSharedMemoryBenchmarkReader.java | 2 +- .../IpcSharedMemoryBenchmarkWriter.java | 2 +- .../internal/util/nio/GridNioSelfTest.java | 13 +- .../internal/util/nio/GridNioSslSelfTest.java | 2 + .../unsafe/GridUnsafeMemorySelfTest.java | 4 +- .../tostring/GridToStringBuilderSelfTest.java | 4 +- .../communication/GridIoManagerBenchmark0.java | 1 + .../marshaller/MarshallerContextTestImpl.java | 29 +- .../ignite/messaging/GridMessagingSelfTest.java | 3 - .../GridP2PContinuousDeploymentSelfTest.java | 2 + .../p2p/GridP2PLocalDeploymentSelfTest.java | 6 +- .../p2p/GridP2PRemoteClassLoadersSelfTest.java | 31 +- .../spi/GridTcpSpiForwardingSelfTest.java | 4 +- .../GridTcpCommunicationSpiAbstractTest.java | 13 + ...mmunicationSpiConcurrentConnectSelfTest.java | 4 +- .../GridTcpCommunicationSpiConfigSelfTest.java | 1 - ...cpCommunicationSpiMultithreadedSelfTest.java | 21 +- ...pCommunicationSpiMultithreadedShmemTest.java | 28 + ...dTcpCommunicationSpiRecoveryAckSelfTest.java | 1 + ...GridTcpCommunicationSpiRecoverySelfTest.java | 1 + .../GridTcpCommunicationSpiShmemSelfTest.java | 38 + .../tcp/GridTcpCommunicationSpiTcpSelfTest.java | 7 + .../tcp/TcpClientDiscoverySpiSelfTest.java | 183 ++++- .../spi/discovery/tcp/TcpDiscoverySelfTest.java | 44 +- .../testframework/GridSpiTestContext.java | 7 +- .../ignite/testframework/GridTestUtils.java | 14 + .../config/GridTestProperties.java | 14 +- .../testframework/junits/GridAbstractTest.java | 2 +- .../junits/GridTestKernalContext.java | 3 +- .../junits/IgniteTestResources.java | 16 +- .../junits/common/GridCommonAbstractTest.java | 15 +- .../ignite/testsuites/IgniteBasicTestSuite.java | 32 +- .../IgniteCacheDataStructuresSelfTestSuite.java | 25 +- .../IgniteCacheEvictionSelfTestSuite.java | 3 +- .../IgniteCacheFailoverTestSuite.java | 24 +- .../IgniteCacheFailoverTestSuite2.java | 47 ++ .../IgniteCacheFullApiSelfTestSuite.java | 8 +- ...niteCacheP2pUnmarshallingErrorTestSuite.java | 20 +- .../testsuites/IgniteCacheRestartTestSuite.java | 10 +- .../ignite/testsuites/IgniteCacheTestSuite.java | 48 +- .../testsuites/IgniteCacheTestSuite2.java | 4 +- .../testsuites/IgniteCacheTestSuite3.java | 15 +- .../testsuites/IgniteCacheTestSuite4.java | 29 +- .../IgniteCacheWriteBehindTestSuite.java | 2 + .../testsuites/IgniteClientTestSuite.java | 38 + .../testsuites/IgniteComputeGridTestSuite.java | 1 + .../testsuites/IgniteKernalSelfTestSuite.java | 16 +- .../IgniteMarshallerSelfTestSuite.java | 28 +- .../IgniteSpiCommunicationSelfTestSuite.java | 2 + .../testsuites/IgniteUtilSelfTestSuite.java | 18 +- .../apache/ignite/util/GridRandomSelfTest.java | 4 +- .../ignite/util/TestTcpCommunicationSpi.java | 21 + modules/extdata/p2p/pom.xml | 2 +- .../p2p/GridP2PContinuousDeploymentTask1.java | 2 +- .../tests/p2p/P2PTestTaskExternalPath1.java | 10 +- .../tests/p2p/P2PTestTaskExternalPath2.java | 8 +- modules/extdata/uri/pom.xml | 2 +- modules/gce/pom.xml | 2 +- modules/geospatial/pom.xml | 2 +- modules/hadoop/pom.xml | 81 +- .../fs/IgniteHadoopFileSystemCounterWriter.java | 9 +- .../processors/hadoop/HadoopClassLoader.java | 29 + .../processors/hadoop/HadoopDefaultJobInfo.java | 27 +- .../internal/processors/hadoop/HadoopUtils.java | 237 ------ .../hadoop/SecondaryFileSystemProvider.java | 3 +- .../hadoop/fs/HadoopFileSystemCacheUtils.java | 241 ++++++ .../hadoop/fs/HadoopFileSystemsUtils.java | 11 + .../hadoop/fs/HadoopLazyConcurrentMap.java | 5 + .../hadoop/jobtracker/HadoopJobTracker.java | 25 +- .../child/HadoopChildProcessRunner.java | 3 +- .../processors/hadoop/v2/HadoopV2Job.java | 84 +- .../hadoop/v2/HadoopV2JobResourceManager.java | 22 +- .../hadoop/v2/HadoopV2TaskContext.java | 37 +- .../HadoopIgfs20FileSystemAbstractSelfTest.java | 17 +- ...oopSecondaryFileSystemConfigurationTest.java | 14 + .../apache/ignite/igfs/IgfsEventsTestSuite.java | 5 +- .../IgniteHadoopFileSystemAbstractSelfTest.java | 2 +- ...IgniteHadoopFileSystemHandshakeSelfTest.java | 7 + .../IgniteHadoopFileSystemIpcCacheSelfTest.java | 7 + .../hadoop/HadoopAbstractSelfTest.java | 7 + .../processors/hadoop/HadoopMapReduceTest.java | 23 +- .../processors/hadoop/HadoopTasksV1Test.java | 7 +- .../processors/hadoop/HadoopTasksV2Test.java | 7 +- .../processors/hadoop/HadoopV2JobSelfTest.java | 6 +- .../collections/HadoopAbstractMapTest.java | 3 +- .../collections/HadoopHashMapSelfTest.java | 4 +- .../HadoopExternalTaskExecutionSelfTest.java | 2 + .../HadoopExternalCommunicationSelfTest.java | 5 + .../testsuites/IgniteHadoopTestSuite.java | 9 +- .../IgniteIgfsLinuxAndMacOSTestSuite.java | 3 +- modules/hibernate/pom.xml | 16 +- .../hibernate/CacheHibernateBlobStore.java | 87 +- .../CacheHibernateBlobStoreFactory.java | 235 ++++++ .../hibernate/src/test/config/factory-cache.xml | 59 ++ .../src/test/config/factory-cache1.xml | 61 ++ .../config/factory-incorrect-store-cache.xml | 56 ++ .../hibernate/HibernateL2CacheSelfTest.java | 5 + .../HibernateL2CacheTransactionalSelfTest.java | 5 + .../CacheHibernateStoreFactorySelfTest.java | 273 +++++++ .../testsuites/IgniteHibernateTestSuite.java | 6 +- modules/indexing/pom.xml | 18 +- .../processors/query/h2/IgniteH2Indexing.java | 125 ++- .../query/h2/sql/GridSqlQuerySplitter.java | 49 +- .../query/h2/twostep/GridMapQueryExecutor.java | 321 ++++++-- .../query/h2/twostep/GridMergeIndex.java | 17 +- .../h2/twostep/GridMergeIndexUnsorted.java | 7 +- .../h2/twostep/GridReduceQueryExecutor.java | 656 +++++++++++++--- .../query/h2/twostep/GridResultPage.java | 21 +- .../CacheAbstractQueryMetricsSelfTest.java | 205 +++++ .../CachePartitionedQueryMetricsSelfTest.java | 32 + .../CacheReplicatedQueryMetricsSelfTest.java | 32 + ...CacheScanPartitionQueryFallbackSelfTest.java | 408 ++++++++++ .../cache/GridCacheCrossCacheQuerySelfTest.java | 25 +- .../cache/GridCacheOffHeapSelfTest.java | 1 - .../cache/GridCacheQueryMetricsSelfTest.java | 206 ----- ...idCacheReduceQueryMultithreadedSelfTest.java | 10 - .../processors/cache/GridCacheSwapSelfTest.java | 3 - .../IgniteCacheAbstractFieldsQuerySelfTest.java | 13 +- .../cache/IgniteCacheAbstractQuerySelfTest.java | 79 +- .../cache/IgniteCacheOffheapEvictQueryTest.java | 196 +++++ ...hePartitionedQueryMultiThreadedSelfTest.java | 40 +- .../IgniteCacheQueryMultiThreadedSelfTest.java | 2 - ...QueryOffheapEvictsMultiThreadedSelfTest.java | 5 + ...lientQueryReplicatedNodeRestartSelfTest.java | 419 ++++++++++ .../IgniteCacheQueryNodeRestartSelfTest.java | 41 +- .../IgniteCacheQueryNodeRestartSelfTest2.java | 383 +++++++++ ...dCacheAbstractReduceFieldsQuerySelfTest.java | 1 - .../h2/GridIndexingSpiAbstractSelfTest.java | 4 +- .../query/h2/sql/BaseH2CompareQueryTest.java | 6 +- .../query/h2/sql/GridQueryParsingTest.java | 5 +- .../IgniteCacheQuerySelfTestSuite.java | 13 +- modules/jcl/pom.xml | 2 +- modules/jta/pom.xml | 2 +- .../cache/jta/GridCacheXAResource.java | 18 +- .../processors/cache/GridCacheJtaSelfTest.java | 2 +- modules/log4j/pom.xml | 2 +- modules/mesos/README.txt | 2 +- modules/mesos/pom.xml | 2 +- .../apache/ignite/mesos/ClusterProperties.java | 15 + .../apache/ignite/mesos/IgniteScheduler.java | 10 +- modules/rest-http/pom.xml | 2 +- modules/scalar-2.10/README.txt | 4 + modules/scalar-2.10/licenses/apache-2.0.txt | 202 +++++ .../scalar-2.10/licenses/scala-bsd-license.txt | 18 + modules/scalar-2.10/pom.xml | 197 +++++ modules/scalar/pom.xml | 2 +- modules/schedule/pom.xml | 2 +- modules/schema-import/pom.xml | 2 +- modules/slf4j/pom.xml | 2 +- modules/spark-2.10/README.txt | 4 + modules/spark-2.10/licenses/apache-2.0.txt | 202 +++++ .../spark-2.10/licenses/scala-bsd-license.txt | 18 + modules/spark-2.10/pom.xml | 120 +++ modules/spark/README.txt | 8 + modules/spark/licenses/apache-2.0.txt | 202 +++++ modules/spark/licenses/scala-bsd-license.txt | 18 + modules/spark/pom.xml | 110 +++ .../org/apache/ignite/spark/IgniteContext.scala | 119 +++ .../org/apache/ignite/spark/IgniteRDD.scala | 244 ++++++ .../apache/ignite/spark/JavaIgniteContext.scala | 63 ++ .../org/apache/ignite/spark/JavaIgniteRDD.scala | 99 +++ .../ignite/spark/impl/IgniteAbstractRDD.scala | 39 + .../ignite/spark/impl/IgnitePartition.scala | 24 + .../ignite/spark/impl/IgniteQueryIterator.scala | 27 + .../apache/ignite/spark/impl/IgniteSqlRDD.scala | 41 + .../spark/impl/JavaIgniteAbstractRDD.scala | 34 + .../ignite/spark/JavaIgniteRDDSelfTest.java | 298 +++++++ .../scala/org/apache/ignite/spark/Entity.scala | 28 + .../org/apache/ignite/spark/IgniteRddSpec.scala | 231 ++++++ modules/spring/pom.xml | 9 +- .../GridResourceSpringBeanInjector.java | 2 +- .../util/spring/IgniteSpringHelperImpl.java | 17 + .../src/test/config/incorrect-store-cache.xml | 57 ++ modules/spring/src/test/config/node.xml | 43 + modules/spring/src/test/config/node1.xml | 45 ++ .../test/config/pojo-incorrect-store-cache.xml | 56 ++ modules/spring/src/test/config/store-cache.xml | 59 ++ modules/spring/src/test/config/store-cache1.xml | 62 ++ .../jdbc/CacheJdbcBlobStoreFactorySelfTest.java | 172 ++++ .../jdbc/CacheJdbcPojoStoreFactorySelfTest.java | 193 +++++ .../testsuites/IgniteSpringTestSuite.java | 5 + modules/ssh/pom.xml | 2 +- modules/tools/pom.xml | 2 +- .../ignite/tools/classgen/ClassesGenerator.java | 30 +- modules/urideploy/pom.xml | 2 +- modules/visor-console-2.10/README.txt | 4 + modules/visor-console-2.10/pom.xml | 174 ++++ modules/visor-console/pom.xml | 2 +- .../commands/cache/VisorCacheCommand.scala | 7 +- modules/visor-plugins/pom.xml | 2 +- modules/web/pom.xml | 2 +- .../IgniteWebSessionSelfTestSuite.java | 2 +- modules/yardstick/pom.xml | 2 +- parent/pom.xml | 5 + pom.xml | 22 +- scripts/git-patch-prop.sh | 2 +- 495 files changed, 18532 insertions(+), 4895 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8232be6/modules/core/src/main/java/org/apache/ignite/cluster/ClusterGroup.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8232be6/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8232be6/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterAsyncImpl.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8232be6/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java ---------------------------------------------------------------------- diff --cc modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java index 0000000,ceb9bef..37a6e72 mode 000000,100644..100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/ClusterGroupSelfTest.java @@@ -1,0 -1,251 +1,273 @@@ + /* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + package org.apache.ignite.internal; + + import org.apache.ignite.*; + import org.apache.ignite.cluster.*; + import org.apache.ignite.internal.util.typedef.*; + import org.apache.ignite.lang.*; + import org.apache.ignite.marshaller.*; + import org.apache.ignite.testframework.junits.common.*; + + import java.util.*; + + /** + * Test for {@link ClusterGroup}. + */ + @GridCommonTest(group = "Kernal Self") + public class ClusterGroupSelfTest extends ClusterGroupAbstractTest { + /** Nodes count. */ + private static final int NODES_CNT = 4; + + /** Projection node IDs. */ + private static Collection ids; + + /** */ + private static Ignite ignite; + + /** {@inheritDoc} */ + @SuppressWarnings({"ConstantConditions"}) + @Override protected void beforeTestsStarted() throws Exception { + assert NODES_CNT > 2; + + ids = new LinkedList<>(); + + try { + for (int i = 0; i < NODES_CNT; i++) { + Ignition.setClientMode(i > 1); + + Ignite g = startGrid(i); + + ids.add(g.cluster().localNode().id()); + + if (i == 0) + ignite = g; + } + } + finally { + Ignition.setClientMode(false); + } + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + for (int i = 0; i < NODES_CNT; i++) + stopGrid(i); + } + + /** {@inheritDoc} */ + @Override protected ClusterGroup projection() { + return grid(0).cluster().forPredicate(F.nodeForNodeIds(ids)); + } + + /** {@inheritDoc} */ + @Override protected UUID localNodeId() { + return grid(0).localNode().id(); + } + + /** + * @throws Exception If failed. + */ + public void testRandom() throws Exception { + assertTrue(ignite.cluster().nodes().contains(ignite.cluster().forRandom().node())); + } + + /** + * @throws Exception If failed. + */ + public void testOldest() throws Exception { + ClusterGroup oldest = ignite.cluster().forOldest(); + + ClusterNode node = null; + + long minOrder = Long.MAX_VALUE; + + for (ClusterNode n : ignite.cluster().nodes()) { + if (n.order() < minOrder) { + node = n; + + minOrder = n.order(); + } + } + + assertEquals(oldest.node(), ignite.cluster().forNode(node).node()); + } + + /** + * @throws Exception If failed. + */ + public void testYoungest() throws Exception { + ClusterGroup youngest = ignite.cluster().forYoungest(); + + ClusterNode node = null; + + long maxOrder = Long.MIN_VALUE; + + for (ClusterNode n : ignite.cluster().nodes()) { + if (n.order() > maxOrder) { + node = n; + + maxOrder = n.order(); + } + } + + assertEquals(youngest.node(), ignite.cluster().forNode(node).node()); + } + + /** + * @throws Exception If failed. + */ + public void testNewNodes() throws Exception { + ClusterGroup youngest = ignite.cluster().forYoungest(); + ClusterGroup oldest = ignite.cluster().forOldest(); + + ClusterNode old = oldest.node(); + ClusterNode last = youngest.node(); + + assertNotNull(last); + + try (Ignite g = startGrid(NODES_CNT)) { + ClusterNode n = g.cluster().localNode(); + + ClusterNode latest = youngest.node(); + + assertNotNull(latest); + assertEquals(latest.id(), n.id()); + assertEquals(oldest.node(), old); + } + } + + /** + * @throws Exception If failed. + */ + public void testForPredicate() throws Exception { + IgnitePredicate evenP = new IgnitePredicate() { + @Override public boolean apply(ClusterNode node) { + return node.order() % 2 == 0; + } + }; + + IgnitePredicate oddP = new IgnitePredicate() { + @Override public boolean apply(ClusterNode node) { + return node.order() % 2 == 1; + } + }; + + ClusterGroup remotes = ignite.cluster().forRemotes(); + + ClusterGroup evenYoungest = remotes.forPredicate(evenP).forYoungest(); + ClusterGroup evenOldest = remotes.forPredicate(evenP).forOldest(); + + ClusterGroup oddYoungest = remotes.forPredicate(oddP).forYoungest(); + ClusterGroup oddOldest = remotes.forPredicate(oddP).forOldest(); + + int clusterSize = ignite.cluster().nodes().size(); + + assertEquals(grid(gridMaxOrder(clusterSize, true)).localNode().id(), evenYoungest.node().id()); + assertEquals(grid(1).localNode().id(), evenOldest.node().id()); + + assertEquals(grid(gridMaxOrder(clusterSize, false)).localNode().id(), oddYoungest.node().id()); + assertEquals(grid(2).localNode().id(), oddOldest.node().id()); + + try (Ignite g4 = startGrid(NODES_CNT); + Ignite g5 = startGrid(NODES_CNT + 1)) + { + clusterSize = g4.cluster().nodes().size(); + + assertEquals(grid(gridMaxOrder(clusterSize, true)).localNode().id(), evenYoungest.node().id()); + assertEquals(grid(1).localNode().id(), evenOldest.node().id()); + + assertEquals(grid(gridMaxOrder(clusterSize, false)).localNode().id(), oddYoungest.node().id()); + assertEquals(grid(2).localNode().id(), oddOldest.node().id()); + } + } + + /** + * @throws Exception If failed. + */ + public void testAgeClusterGroupSerialization() throws Exception { + Marshaller marshaller = getConfiguration().getMarshaller(); + + ClusterGroup grp = ignite.cluster().forYoungest(); + ClusterNode node = grp.node(); + + byte[] arr = marshaller.marshal(grp); + + ClusterGroup obj = marshaller.unmarshal(arr, null); + + assertEquals(node.id(), obj.node().id()); + + try (Ignite ignore = startGrid()) { + obj = marshaller.unmarshal(arr, null); + + assertEquals(grp.node().id(), obj.node().id()); + assertFalse(node.id().equals(obj.node().id())); + } + } + + /** + * @throws Exception If failed. + */ + public void testClientServer() throws Exception { + ClusterGroup srv = ignite.cluster().forServers(); + + assertEquals(2, srv.nodes().size()); + assertTrue(srv.nodes().contains(ignite(0).cluster().localNode())); + assertTrue(srv.nodes().contains(ignite(1).cluster().localNode())); + + ClusterGroup cli = ignite.cluster().forClients(); + + assertEquals(2, srv.nodes().size()); + assertTrue(cli.nodes().contains(ignite(2).cluster().localNode())); + assertTrue(cli.nodes().contains(ignite(3).cluster().localNode())); + } + + /** + * @param cnt Count. + * @param even Even. + */ + private static int gridMaxOrder(int cnt, boolean even) { + assert cnt > 2; + + cnt = cnt - (cnt % 2); + + return even ? cnt - 1 : cnt - 2; + } ++ ++ /** ++ * @throws Exception If failed. ++ */ ++ public void testHostNames() throws Exception { ++ Collection inputHostNames = ignite.cluster().hostNames(); ++ Collection localNodeHostNames = ignite.cluster().localNode().hostNames(); ++ Collection randomNodeHostNames = ignite.cluster().forRandom().node().hostNames(); ++ Collection allNodes = ignite.cluster().nodes(); ++ Collection checkHostNames = new HashSet (); ++ ++ for (ClusterNode currentNode : allNodes) ++ Collections.addAll(checkHostNames, currentNode.hostNames().toArray(new String[0])); ++ ++ assert(checkHostNames.equals(inputHostNames)); ++ ++ if (!(localNodeHostNames.isEmpty()) && !(inputHostNames.isEmpty())) ++ assert((inputHostNames.containsAll(localNodeHostNames)) == true); ++ ++ if (!(randomNodeHostNames.isEmpty()) && !(inputHostNames.isEmpty())) ++ assert((inputHostNames.containsAll(randomNodeHostNames)) == true); ++ } + }