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 1D11117678 for ; Sun, 1 Mar 2015 20:02:08 +0000 (UTC) Received: (qmail 88995 invoked by uid 500); 1 Mar 2015 20:02:08 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 88957 invoked by uid 500); 1 Mar 2015 20:02:08 -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 88948 invoked by uid 99); 1 Mar 2015 20:02:07 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Sun, 01 Mar 2015 20:02:07 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED,T_RP_MATCHES_RCVD X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO mail.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with SMTP; Sun, 01 Mar 2015 20:02:05 +0000 Received: (qmail 87826 invoked by uid 99); 1 Mar 2015 20:01:45 -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; Sun, 01 Mar 2015 20:01:45 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 4BE9DE0F5A; Sun, 1 Mar 2015 20:01:45 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sevdokimov@apache.org To: commits@ignite.incubator.apache.org Date: Sun, 01 Mar 2015 20:01:49 -0000 Message-Id: In-Reply-To: <0c0df158b03549a29dfdd08c32b2d879@git.apache.org> References: <0c0df158b03549a29dfdd08c32b2d879@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [05/10] incubator-ignite git commit: Merge commit X-Virus-Checked: Checked by ClamAV on apache.org Merge commit Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/153cbee4 Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/153cbee4 Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/153cbee4 Branch: refs/heads/sprint-2 Commit: 153cbee41b8adbcac1989f7746d1ef51049cda1f Parents: 493775a bf8c048 Author: sevdokimov Authored: Thu Feb 26 17:01:52 2015 +0300 Committer: sevdokimov Committed: Thu Feb 26 17:01:52 2015 +0300 ---------------------------------------------------------------------- examples/config/example-compute.xml | 2 + .../hibernate/example-hibernate-L2-cache.xml | 2 +- .../ComputeFibonacciContinuationExample.java | 17 +- .../examples/datagrid/CacheApiExample.java | 4 +- .../datagrid/CacheContinuousQueryExample.java | 6 +- .../IgniteCountDownLatchExample.java | 3 +- .../MessagingPingPongListenActorExample.java | 3 +- .../examples/ScalarContinuationExample.scala | 20 +- .../checkpoint/s3/S3CheckpointSpiSelfTest.java | 10 +- .../internal/client/ClientStartNodeTask.java | 5 +- .../processors/rest/RestProcessorTest.java | 8 +- .../processors/rest/TestBinaryClient.java | 1 - .../processors/rest/TestMemcacheClient.java | 13 +- .../protocols/tcp/TcpRestParserSelfTest.java | 2 +- .../ignite/codegen/MessageCodeGenerator.java | 28 +- .../java/org/apache/ignite/IgniteCache.java | 69 + .../java/org/apache/ignite/IgniteCluster.java | 3 +- .../main/java/org/apache/ignite/IgniteFs.java | 2 +- .../apache/ignite/IgniteSystemProperties.java | 5 + .../ignite/cache/IgniteEntryProcessor.java | 28 + .../CacheRendezvousAffinityFunction.java | 1 + .../cache/store/jdbc/CacheJdbcPojoStore.java | 28 +- .../ignite/cluster/ClusterNodeLocalMap.java | 60 - .../org/apache/ignite/events/EventType.java | 8 + .../IgfsByteDelimiterRecordResolver.java | 1 + .../java/org/apache/ignite/igfs/package.html | 2 +- .../ignite/internal/ClusterMetricsSnapshot.java | 9 +- .../internal/GridEventConsumeHandler.java | 1 + .../ignite/internal/GridKernalContext.java | 16 +- .../ignite/internal/GridKernalContextImpl.java | 23 +- .../internal/GridMessageListenHandler.java | 1 + .../apache/ignite/internal/IgniteKernal.java | 11 +- .../cluster/ClusterNodeLocalMapImpl.java | 27 +- .../cluster/IgniteClusterAsyncImpl.java | 3 +- .../internal/cluster/IgniteClusterImpl.java | 4 +- .../internal/events/DiscoveryCustomEvent.java | 68 + .../igfs/common/IgfsControlResponse.java | 1 + .../internal/igfs/common/IgfsMarshaller.java | 1 + .../internal/managers/GridManagerAdapter.java | 5 + .../discovery/GridDiscoveryManager.java | 105 +- .../affinity/GridAffinityMessage.java | 1 + .../processors/cache/GridCacheEntryInfo.java | 5 +- .../processors/cache/GridCacheMapEntry.java | 562 ++-- .../processors/cache/GridCacheSwapManager.java | 4 +- .../processors/cache/GridCacheUtils.java | 4 + .../processors/cache/IgniteCacheProxy.java | 63 + .../GridDistributedTxRemoteAdapter.java | 66 +- .../dht/atomic/GridDhtAtomicCache.java | 44 +- .../dht/atomic/GridNearAtomicUpdateFuture.java | 84 +- .../distributed/near/GridNearAtomicCache.java | 3 - .../query/GridCacheDistributedQueryManager.java | 4 +- .../cache/query/GridCacheQueryManager.java | 2 +- .../continuous/CacheContinuousQueryEntry.java | 1 + .../continuous/CacheContinuousQueryHandler.java | 1 + .../cache/transactions/IgniteTxAdapter.java | 58 +- .../cache/transactions/IgniteTxEntry.java | 17 +- .../transactions/IgniteTxLocalAdapter.java | 48 +- .../version/GridCacheRawVersionedEntry.java | 1 + .../cache/version/GridCacheVersion.java | 3 +- .../GridCacheVersionConflictContext.java | 35 +- .../cache/version/GridCacheVersionEx.java | 2 +- .../processors/clock/GridClockMessage.java | 1 + .../closure/GridClosureProcessor.java | 82 +- .../dataload/GridDataLoaderProcessor.java | 2 +- .../datastructures/DataStructuresProcessor.java | 2 +- .../dr/GridDrDataLoadCacheUpdater.java | 6 +- .../processors/igfs/IgfsDataManager.java | 4 +- .../internal/processors/igfs/IgfsImpl.java | 6 +- .../processors/igfs/IgfsOutputStreamImpl.java | 4 +- .../internal/processors/igfs/IgfsServer.java | 1 + .../internal/processors/job/GridJobWorker.java | 2 +- .../processors/rest/GridRestProcessor.java | 85 +- .../processors/rest/GridRestResponse.java | 1 + .../message/GridClientAbstractMessage.java | 1 + .../message/GridClientHandshakeRequest.java | 1 + .../cache/GridCacheQueryCommandHandler.java | 7 +- .../protocols/tcp/GridMemcachedMessage.java | 3 +- .../rest/protocols/tcp/GridTcpRestParser.java | 10 +- .../securesession/GridSecureSession.java | 60 - .../GridSecureSessionProcessor.java | 59 - .../os/GridOsSecureSessionProcessor.java | 61 - .../security/GridSecurityContext.java | 251 -- .../security/GridSecurityProcessor.java | 6 +- .../processors/security/SecurityContext.java | 56 + .../security/os/GridOsSecurityProcessor.java | 146 +- .../processors/task/GridTaskWorker.java | 4 +- .../internal/util/IgniteExceptionRegistry.java | 259 ++ .../ignite/internal/util/IgniteUtils.java | 2569 ++++++++---------- .../nio/GridConnectionBytesVerifyFilter.java | 1 + .../util/nio/GridTcpCommunicationClient.java | 1 + .../apache/ignite/internal/util/typedef/X.java | 3 +- .../VisorComputeToggleMonitoringTask.java | 4 +- .../visor/node/VisorNodeDataCollectorJob.java | 4 +- .../node/VisorNodeEventsCollectorTask.java | 3 +- .../visor/query/VisorQueryCleanupTask.java | 4 +- .../visor/query/VisorQueryNextPageTask.java | 6 +- .../internal/visor/query/VisorQueryTask.java | 4 +- .../internal/visor/util/VisorTaskUtils.java | 45 +- .../ignite/lang/IgniteProductVersion.java | 1 + .../optimized/optimized-classnames.properties | 3 +- .../org/apache/ignite/mxbean/IgniteMXBean.java | 6 + .../org/apache/ignite/spi/IgniteSpiAdapter.java | 15 + .../org/apache/ignite/spi/IgniteSpiContext.java | 8 + .../ignite/spi/IgniteSpiThreadFactory.java | 2 +- .../communication/tcp/TcpCommunicationSpi.java | 54 +- .../ignite/spi/discovery/DiscoverySpi.java | 7 + .../spi/discovery/DiscoverySpiListener.java | 11 +- .../DiscoverySpiNodeAuthenticator.java | 2 +- .../discovery/tcp/TcpClientDiscoverySpi.java | 7 +- .../spi/discovery/tcp/TcpDiscoverySpi.java | 280 +- .../discovery/tcp/TcpDiscoverySpiAdapter.java | 3 + .../tcp/internal/TcpDiscoveryNode.java | 1 + .../TcpDiscoveryMulticastIpFinder.java | 1 + .../messages/TcpDiscoveryAuthFailedMessage.java | 1 + .../TcpDiscoveryCustomEventMessage.java | 66 + .../messages/TcpDiscoveryHeartbeatMessage.java | 1 + .../spi/swapspace/file/FileSwapSpaceSpi.java | 2 +- .../core/src/main/resources/ignite.properties | 2 +- .../internal/GridDiscoveryEventSelfTest.java | 44 + .../internal/GridEventStorageSelfTest.java | 3 +- .../GridJobMasterLeaveAwareSelfTest.java | 24 +- .../internal/GridMultipleJobsSelfTest.java | 2 +- .../ignite/internal/GridNodeLocalSelfTest.java | 4 +- .../GridTaskContinuousMapperSelfTest.java | 3 +- .../GridTaskExecutionContextSelfTest.java | 2 +- .../GridCheckpointManagerAbstractSelfTest.java | 20 +- .../cache/GridCacheAbstractFullApiSelfTest.java | 101 +- .../GridCacheConcurrentTxMultiNodeTest.java | 13 +- .../cache/GridCachePutAllFailoverSelfTest.java | 5 +- .../cache/IgniteTxMultiNodeAbstractTest.java | 33 +- ...cheAtomicReferenceMultiNodeAbstractTest.java | 12 +- .../GridCacheMultiNodeDataStructureTest.java | 3 +- ...dCacheSequenceMultiNodeAbstractSelfTest.java | 2 +- ...titionedAtomicSequenceMultiThreadedTest.java | 16 +- ...dCachePartitionedQueueEntryMoveSelfTest.java | 2 +- .../GridCacheAbstractJobExecutionTest.java | 2 +- .../distributed/GridCacheEventAbstractTest.java | 4 +- .../dht/GridCacheDhtInternalEntrySelfTest.java | 6 +- .../dht/GridCacheDhtMultiBackupTest.java | 2 +- ...idCachePartitionedHitsAndMissesSelfTest.java | 3 +- ...ePartitionedMultiThreadedPutGetSelfTest.java | 2 +- .../near/IgniteCacheNearReadCommittedTest.java | 3 + .../GridCacheRandomEvictionPolicySelfTest.java | 2 +- .../IgniteCacheLoaderWriterAbstractTest.java | 47 + .../closure/GridClosureProcessorSelfTest.java | 14 +- .../processors/igfs/IgfsAbstractSelfTest.java | 5 +- .../igfs/IgfsDataManagerSelfTest.java | 1 + .../processors/igfs/IgfsProcessorSelfTest.java | 3 +- .../streamer/GridStreamerEvictionSelfTest.java | 2 +- .../streamer/GridStreamerSelfTest.java | 2 +- .../util/IgniteExceptionRegistrySelfTest.java | 89 + .../internal/util/IgniteUtilsSelfTest.java | 6 +- .../internal/util/nio/GridRoundTripTest.java | 1 + .../offheap/GridOffHeapMapAbstractSelfTest.java | 1 + .../cache/GridCacheDataStructuresLoadTest.java | 36 +- .../loadtests/cache/GridCacheLoadTest.java | 6 +- .../loadtests/cache/GridCacheSwapLoadTest.java | 5 +- .../loadtests/colocation/GridTestMain.java | 3 +- .../communication/GridIoManagerBenchmark.java | 3 +- .../GridMultiSplitsRedeployLoadTest.java | 5 +- .../loadtests/discovery/GridGcTimeoutTest.java | 3 +- .../ignite/loadtests/dsi/GridDsiPerfJob.java | 16 +- .../job/GridJobExecutionSingleNodeLoadTest.java | 2 +- .../mapper/GridContinuousMapperLoadTest1.java | 3 +- .../loadtests/mapper/GridNodeStartup.java | 3 +- .../mergesort/GridMergeSortLoadTest.java | 3 +- .../streamer/GridStreamerIndexLoadTest.java | 4 +- .../swap/GridSwapEvictAllBenchmark.java | 3 +- .../marshaller/GridMarshallerAbstractTest.java | 6 +- ...idSessionFutureWaitJobAttributeSelfTest.java | 3 +- .../GridSessionSetTaskAttributeSelfTest.java | 3 +- ...GridSessionTaskWaitJobAttributeSelfTest.java | 3 +- .../discovery/AbstractDiscoverySelfTest.java | 22 +- .../tcp/TcpDiscoverySpiStartStopSelfTest.java | 23 - .../roundrobin/GridRoundRobinTestUtils.java | 6 +- .../file/GridFileSwapSpaceSpiSelfTest.java | 1 + .../index/GridStreamerIndexSelfTest.java | 17 +- .../window/GridStreamerWindowSelfTest.java | 3 +- .../testframework/GridSpiTestContext.java | 6 + .../junits/GridTestKernalContext.java | 2 + .../ignite/testframework/junits/IgniteMock.java | 1 - .../cache/GridAbstractCacheStoreSelfTest.java | 3 +- .../junits/spi/GridSpiAbstractTest.java | 12 - .../ignite/testsuites/IgniteCacheTestSuite.java | 3 +- .../testsuites/IgniteUtilSelfTestSuite.java | 1 + .../tests/p2p/GridP2PAwareTestUserResource.java | 5 +- .../tests/p2p/GridTestMessageListener.java | 4 +- modules/hadoop/pom.xml | 10 - .../client/hadoop/GridHadoopClientProtocol.java | 3 +- .../java/org/apache/ignite/igfs/package.html | 2 +- .../internal/igfs/hadoop/IgfsHadoopWrapper.java | 2 +- .../hadoop/GridHadoopDefaultJobInfo.java | 5 +- .../processors/hadoop/GridHadoopSetup.java | 3 +- .../GridHadoopDefaultMapReducePlanner.java | 3 +- .../shuffle/GridHadoopShuffleMessage.java | 1 + .../taskexecutor/GridHadoopExecutorService.java | 2 +- .../hadoop/v2/GridHadoopSplitWrapper.java | 3 +- .../hadoop/GridHadoopGroupingTest.java | 10 +- .../hadoop/GridHadoopJobTrackerSelfTest.java | 2 +- .../hadoop/GridHadoopSortingTest.java | 9 +- .../processors/hadoop/GridHadoopStartup.java | 2 +- .../ignite/loadtests/igfs/IgfsNodeStartup.java | 3 +- .../testsuites/IgniteHadoopTestSuite.java | 7 +- ...idHibernateAbstractRegionAccessStrategy.java | 97 - .../GridHibernateAccessStrategyAdapter.java | 369 --- .../GridHibernateCollectionRegion.java | 99 - .../hibernate/GridHibernateEntityRegion.java | 110 - .../GridHibernateGeneralDataRegion.java | 69 - .../hibernate/GridHibernateNaturalIdRegion.java | 98 - .../GridHibernateNonStrictAccessStrategy.java | 220 -- .../GridHibernateQueryResultsRegion.java | 71 - .../GridHibernateReadOnlyAccessStrategy.java | 106 - .../GridHibernateReadWriteAccessStrategy.java | 282 -- .../cache/hibernate/GridHibernateRegion.java | 101 - .../hibernate/GridHibernateRegionFactory.java | 231 -- .../GridHibernateTimestampsRegion.java | 40 - ...ridHibernateTransactionalAccessStrategy.java | 139 - .../GridHibernateTransactionalDataRegion.java | 96 - .../HibernateAbstractRegionAccessStrategy.java | 97 + .../HibernateAccessStrategyAdapter.java | 369 +++ .../hibernate/HibernateCollectionRegion.java | 99 + .../cache/hibernate/HibernateEntityRegion.java | 110 + .../hibernate/HibernateGeneralDataRegion.java | 69 + .../hibernate/HibernateNaturalIdRegion.java | 98 + .../HibernateNonStrictAccessStrategy.java | 220 ++ .../hibernate/HibernateQueryResultsRegion.java | 71 + .../HibernateReadOnlyAccessStrategy.java | 106 + .../HibernateReadWriteAccessStrategy.java | 282 ++ .../ignite/cache/hibernate/HibernateRegion.java | 101 + .../cache/hibernate/HibernateRegionFactory.java | 231 ++ .../hibernate/HibernateTimestampsRegion.java | 40 + .../HibernateTransactionalAccessStrategy.java | 139 + .../HibernateTransactionalDataRegion.java | 96 + .../apache/ignite/cache/hibernate/package.html | 2 +- ...idHibernateL2CacheConfigurationSelfTest.java | 393 --- .../hibernate/GridHibernateL2CacheSelfTest.java | 1923 ------------- ...idHibernateL2CacheTransactionalSelfTest.java | 135 - .../HibernateL2CacheConfigurationSelfTest.java | 393 +++ .../hibernate/HibernateL2CacheSelfTest.java | 1925 +++++++++++++ .../HibernateL2CacheTransactionalSelfTest.java | 135 + .../testsuites/IgniteHibernateTestSuite.java | 6 +- .../cache/GridCacheCrossCacheQuerySelfTest.java | 2 +- .../GridCacheCrossCacheQuerySelfTestNewApi.java | 2 +- .../http/jetty/GridJettyRestHandler.java | 1 + .../p2p/GridP2PUserVersionChangeSelfTest.java | 4 +- ...gniteProjectionStartStopRestartSelfTest.java | 3 +- pom.xml | 2 +- 247 files changed, 8023 insertions(+), 7780 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/153cbee4/examples/config/example-compute.xml ---------------------------------------------------------------------- diff --cc examples/config/example-compute.xml index c532e17,c532e17..303b4eb --- a/examples/config/example-compute.xml +++ b/examples/config/example-compute.xml @@@ -32,6 -32,6 +32,8 @@@ ++ ++ http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/153cbee4/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/153cbee4/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java ---------------------------------------------------------------------- diff --cc modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java index 3e9c120,ecdc6cc..3fc52b1 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java @@@ -1395,83 -1399,68 +1399,86 @@@ public class TcpDiscoverySpi extends Tc return false; boolean retry = false; - IgniteCheckedException errs = null; + Collection errs = new ArrayList<>(); - for (InetSocketAddress addr : addrs) { - try { - Integer res = sendMessageDirectly(joinReq, addr); + SocketMultiConnector multiConnector = new SocketMultiConnector(addrs, 2); - assert res != null; + try { + GridTuple3 tuple; - noResAddrs.remove(addr); + while ((tuple = multiConnector.next()) != null) { + InetSocketAddress addr = tuple.get1(); + Socket sock = tuple.get2(); + Exception ex = tuple.get3(); - // Address is responsive, reset period start. - noResStart = 0; + if (ex == null) { + assert sock != null; - switch (res) { - case RES_WAIT: - // Concurrent startup, try sending join request again or wait if no success. - retry = true; + try { + Integer res = sendMessageDirectly(joinReq, addr, sock); - break; - case RES_OK: - if (log.isDebugEnabled()) - log.debug("Join request message has been sent to address [addr=" + addr + - ", req=" + joinReq + ']'); + assert res != null; - // Join request sending succeeded, wait for response from topology. - return true; + noResAddrs.remove(addr); - default: - // Concurrent startup, try next node. - if (res == RES_CONTINUE_JOIN) { - if (!fromAddrs.contains(addr)) + // Address is responsive, reset period start. + noResStart = 0; + + switch (res) { + case RES_WAIT: + // Concurrent startup, try sending join request again or wait if no success. retry = true; - } - else { - if (log.isDebugEnabled()) - log.debug("Unexpected response to join request: " + res); - retry = true; - } + break; + case RES_OK: + if (log.isDebugEnabled()) + log.debug("Join request message has been sent to address [addr=" + addr + + ", req=" + joinReq + ']'); - break; + // Join request sending succeeded, wait for response from topology. + return true; + + default: + // Concurrent startup, try next node. + if (res == RES_CONTINUE_JOIN) { + if (!fromAddrs.contains(addr)) + retry = true; + } + else { + if (log.isDebugEnabled()) + log.debug("Unexpected response to join request: " + res); + + retry = true; + } + + break; + } + } + catch (IgniteSpiException e) { + ex = e; + } } - } - catch (IgniteSpiException e) { - if (errs == null) - errs = new IgniteCheckedException("Multiple connection attempts failed."); - errs.addSuppressed(e); + if (ex != null) { + errs.add(ex); - if (log.isDebugEnabled()) { - IOException ioe = X.cause(e, IOException.class); + if (log.isDebugEnabled()) { + IOException ioe = X.cause(ex, IOException.class); - log.debug("Failed to send join request message [addr=" + addr + - ", msg=" + ioe != null ? ioe.getMessage() : e.getMessage() + ']'); + log.debug("Failed to send join request message [addr=" + addr + + ", msg=" + ioe != null ? ioe.getMessage() : ex.getMessage() + ']'); + - onException("Failed to send join request message [addr=" + addr + - ", msg=" + ioe != null ? ioe.getMessage() : e.getMessage() + ']', ioe); - } ++ onException("Failed to send join request message [addr=" + addr + ++ ", msg=" + ioe != null ? ioe.getMessage() : ex.getMessage() + ']', ioe); + } - noResAddrs.add(addr); + noResAddrs.add(addr); + } } } + finally { + multiConnector.close(); + } if (retry) { if (log.isDebugEnabled()) http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/153cbee4/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java ----------------------------------------------------------------------