# ignite-42
Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/550d6033 Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/550d6033 Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/550d6033 Branch: refs/heads/ignite-63 Commit: 550d603363f59df34ce82916e009f16a4dc5ed39 Parents: 7b1a738 Author: sboikov <sboi...@gridgain.com> Authored: Thu Jan 22 18:11:21 2015 +0300 Committer: sboikov <sboi...@gridgain.com> Committed: Thu Jan 22 18:11:32 2015 +0300 ---------------------------------------------------------------------- .../examples/datagrid/CacheQueryExample.java | 1 - .../examples/ScalarCacheQueryExample.scala | 1 - .../spi/checkpoint/s3/GridS3CheckpointSpi.java | 3 +- .../ipfinder/s3/GridTcpDiscoveryS3IpFinder.java | 5 +- .../s3/GridS3CheckpointSpiSelfTest.java | 4 +- .../GridClientPartitionAffinitySelfTest.java | 2 + .../src/main/java/org/apache/ignite/Ignite.java | 1 - .../ignite/IgniteAuthenticationException.java | 35 ++ .../apache/ignite/IgniteBasicWarmupClosure.java | 560 ++++++++++++++++++ .../java/org/apache/ignite/IgniteCache.java | 1 - .../java/org/apache/ignite/IgniteCluster.java | 1 - .../org/apache/ignite/IgniteDataLoader.java | 30 +- .../ignite/IgniteDeploymentException.java | 57 ++ .../ignite/IgniteIllegalStateException.java | 84 +++ .../ignite/IgniteInterruptedException.java | 56 ++ .../org/apache/ignite/IgniteMetadataAware.java | 187 ++++++ .../apache/ignite/IgniteNotPeerDeployable.java | 41 ++ .../java/org/apache/ignite/IgnitePortables.java | 1 - .../org/apache/ignite/IgniteTransactions.java | 1 - .../main/java/org/apache/ignite/Ignition.java | 13 +- .../apache/ignite/cache/CacheConfiguration.java | 2 - .../cache/CachePartialUpdateException.java | 3 - .../java/org/apache/ignite/cache/GridCache.java | 2 - .../org/apache/ignite/cache/GridCacheEntry.java | 3 +- .../org/apache/ignite/cache/GridCacheMode.java | 2 - .../org/apache/ignite/cache/GridCacheName.java | 2 - .../ignite/cache/GridCachePreloadMode.java | 2 - .../ignite/cache/GridCacheProjection.java | 2 - .../affinity/GridCacheAffinityFunction.java | 1 - .../affinity/GridCacheAffinityKeyMapped.java | 2 - ...GridCacheConsistentHashAffinityFunction.java | 4 +- .../fair/GridCachePartitionFairAffinity.java | 1 - .../GridCacheRendezvousAffinityFunction.java | 1 - .../ignite/cache/cloner/GridCacheCloner.java | 1 - .../datastructures/GridCacheCountDownLatch.java | 13 +- .../datastructures/GridCacheDataStructures.java | 1 - .../cache/eviction/GridCacheEvictionFilter.java | 1 - .../cache/eviction/GridCacheEvictionPolicy.java | 1 - .../fifo/GridCacheFifoEvictionPolicy.java | 2 - .../ggfs/GridCacheGgfsEvictionFilter.java | 1 - .../GridCacheGgfsPerBlockLruEvictionPolicy.java | 1 - .../lru/GridCacheLruEvictionPolicy.java | 2 - .../random/GridCacheRandomEvictionPolicy.java | 2 - .../cache/query/GridCacheContinuousQuery.java | 1 - .../ignite/cache/query/GridCacheQuery.java | 2 - .../cache/store/CacheLoadOnlyStoreAdapter.java | 1 - .../apache/ignite/cache/store/CacheStore.java | 4 +- .../ignite/cache/store/CacheStoreAdapter.java | 1 - .../cache/store/jdbc/CacheJdbcBlobStore.java | 3 +- .../impl/GridClientAbstractProjection.java | 5 +- .../org/apache/ignite/cluster/ClusterNode.java | 5 +- .../ignite/cluster/ClusterNodeLocalMap.java | 4 +- .../ignite/compute/ComputeJobSibling.java | 3 +- .../ignite/compute/ComputeLoadBalancer.java | 3 +- .../apache/ignite/hadoop/GridHadoopTask.java | 3 +- .../processors/cache/CacheLockImpl.java | 7 +- .../processors/cache/IgniteCacheProxy.java | 1 - .../org/apache/ignite/jdbc/GridJdbcDriver.java | 1 - .../org/apache/ignite/lang/IgniteFuture.java | 7 +- .../org/apache/ignite/lang/IgnitePredicate.java | 2 - .../IgniteOptimizedObjectStreamRegistry.java | 15 +- .../main/java/org/apache/ignite/package.html | 23 + .../ignite/scheduler/SchedulerFuture.java | 7 +- .../communication/tcp/TcpCommunicationSpi.java | 3 +- .../discovery/tcp/TcpClientDiscoverySpi.java | 11 +- .../spi/discovery/tcp/TcpDiscoverySpi.java | 9 +- .../discovery/tcp/TcpDiscoverySpiAdapter.java | 5 +- .../ipfinder/jdbc/TcpDiscoveryJdbcIpFinder.java | 3 +- .../TcpDiscoveryMulticastIpFinder.java | 3 +- .../sharedfs/TcpDiscoverySharedFsIpFinder.java | 3 +- .../spi/swapspace/file/FileSwapSpaceSpi.java | 7 +- .../StreamerCacheAffinityEventRouter.java | 1 - .../window/StreamerBoundedSizeBatchWindow.java | 11 +- .../window/StreamerBoundedTimeBatchWindow.java | 11 +- .../apache/ignite/transactions/IgniteTx.java | 1 - .../transactions/IgniteTxSynchronization.java | 1 - .../grid/GridAuthenticationException.java | 37 -- .../gridgain/grid/GridBasicWarmupClosure.java | 563 ------------------- .../gridgain/grid/GridDeploymentException.java | 58 -- .../grid/GridIllegalStateException.java | 84 --- .../gridgain/grid/GridInterruptedException.java | 58 -- .../org/gridgain/grid/GridMetadataAware.java | 187 ------ .../gridgain/grid/GridNotPeerDeployable.java | 41 -- .../grid/kernal/ClusterNodeLocalMapImpl.java | 3 +- .../grid/kernal/GridEventConsumeHandler.java | 5 +- .../java/org/gridgain/grid/kernal/GridEx.java | 1 - .../org/gridgain/grid/kernal/GridGainEx.java | 17 +- .../grid/kernal/GridJobContextImpl.java | 2 - .../org/gridgain/grid/kernal/GridKernal.java | 4 +- .../gridgain/grid/kernal/GridKernalContext.java | 3 +- .../grid/kernal/GridMessageListenHandler.java | 5 +- .../gridgain/grid/kernal/IgniteComputeImpl.java | 3 +- .../grid/kernal/IgniteTransactionsEx.java | 1 - .../grid/kernal/ggfs/common/GridGgfsLogger.java | 6 +- .../kernal/managers/GridManagerAdapter.java | 1 - .../managers/communication/GridIoManager.java | 5 +- .../deployment/GridDeploymentCommunication.java | 3 +- .../deployment/GridDeploymentLocalStore.java | 3 +- .../discovery/GridDiscoveryManager.java | 3 +- .../eventstorage/GridEventStorageManager.java | 5 +- .../loadbalancer/GridLoadBalancerManager.java | 2 - .../affinity/GridAffinityAssignmentCache.java | 1 - .../affinity/GridAffinityProcessor.java | 2 - .../processors/affinity/GridAffinityUtils.java | 6 +- .../GridCacheAffinityFunctionContextImpl.java | 1 - .../processors/cache/GridCacheAdapter.java | 10 +- .../processors/cache/GridCacheAttributes.java | 2 - .../cache/GridCacheConcurrentMap.java | 1 - .../processors/cache/GridCacheContext.java | 1 - .../GridCacheDefaultAffinityKeyMapper.java | 1 - .../cache/GridCacheDeploymentManager.java | 1 - .../processors/cache/GridCacheEntryEx.java | 4 +- .../processors/cache/GridCacheEntryImpl.java | 4 +- .../processors/cache/GridCacheEntrySet.java | 1 - .../cache/GridCacheEvictionEntry.java | 4 +- .../cache/GridCacheEvictionManager.java | 4 +- .../cache/GridCacheFilterEvaluationEntry.java | 4 +- .../processors/cache/GridCacheIterator.java | 1 - .../processors/cache/GridCacheKeySet.java | 1 - .../processors/cache/GridCacheMBeanAdapter.java | 1 - .../processors/cache/GridCacheMapAdapter.java | 1 - .../processors/cache/GridCacheMapEntry.java | 4 +- .../processors/cache/GridCacheMessage.java | 1 - .../cache/GridCacheMetricsAdapter.java | 1 - .../processors/cache/GridCacheMvccManager.java | 1 - .../GridCachePartitionExchangeManager.java | 7 +- .../cache/GridCachePreloaderAdapter.java | 1 - .../processors/cache/GridCacheProcessor.java | 3 - .../processors/cache/GridCacheProjectionEx.java | 1 - .../cache/GridCacheProjectionImpl.java | 1 - .../kernal/processors/cache/GridCacheProxy.java | 1 - .../processors/cache/GridCacheProxyImpl.java | 2 - .../cache/GridCacheSharedContext.java | 1 - .../processors/cache/GridCacheStoreManager.java | 7 +- .../processors/cache/GridCacheTtlManager.java | 3 +- .../kernal/processors/cache/GridCacheUtils.java | 1 - .../cache/GridCacheValueCollection.java | 1 - .../cache/GridCacheWriteBehindStore.java | 16 +- .../cache/affinity/GridCacheAffinityImpl.java | 1 - .../cache/affinity/GridCacheAffinityProxy.java | 1 - .../datastructures/GridCacheAtomicLongImpl.java | 1 - .../GridCacheAtomicReferenceImpl.java | 1 - .../GridCacheAtomicSequenceImpl.java | 1 - .../GridCacheAtomicStampedImpl.java | 1 - .../GridCacheCountDownLatchImpl.java | 1 - .../GridCacheDataStructuresManager.java | 1 - .../datastructures/GridCacheQueueAdapter.java | 1 - .../cache/datastructures/GridCacheSetImpl.java | 2 - .../GridDistributedCacheAdapter.java | 1 - .../GridDistributedTxRemoteAdapter.java | 1 - .../GridPartitionedCacheEntryImpl.java | 4 +- .../distributed/dht/GridDhtCacheAdapter.java | 1 - .../distributed/dht/GridDhtCacheEntry.java | 1 - .../distributed/dht/GridDhtCacheEntryImpl.java | 1 - .../cache/distributed/dht/GridDhtGetFuture.java | 1 - .../distributed/dht/GridDhtLocalPartition.java | 3 +- .../distributed/dht/GridDhtLockFuture.java | 1 - .../dht/GridDhtTransactionalCacheAdapter.java | 1 - .../distributed/dht/GridDhtTxLocalAdapter.java | 1 - .../distributed/dht/GridDhtTxPrepareFuture.java | 3 +- .../dht/GridPartitionedGetFuture.java | 1 - .../dht/atomic/GridDhtAtomicCache.java | 1 - .../dht/atomic/GridDhtAtomicCacheEntry.java | 1 - .../dht/atomic/GridDhtAtomicUpdateFuture.java | 1 - .../dht/atomic/GridDhtAtomicUpdateRequest.java | 1 - .../dht/atomic/GridNearAtomicUpdateFuture.java | 1 - .../dht/atomic/GridNearAtomicUpdateRequest.java | 1 - .../dht/colocated/GridDhtColocatedCache.java | 1 - .../colocated/GridDhtColocatedCacheEntry.java | 1 - .../colocated/GridDhtColocatedLockFuture.java | 1 - .../dht/preloader/GridDhtForceKeysFuture.java | 3 +- .../preloader/GridDhtPartitionDemandPool.java | 14 +- .../preloader/GridDhtPartitionSupplyPool.java | 3 +- .../GridDhtPartitionsExchangeFuture.java | 8 +- .../distributed/near/GridNearAtomicCache.java | 1 - .../distributed/near/GridNearCacheAdapter.java | 1 - .../distributed/near/GridNearCacheEntry.java | 1 - .../distributed/near/GridNearGetFuture.java | 1 - .../distributed/near/GridNearGetRequest.java | 1 - .../distributed/near/GridNearLockFuture.java | 1 - .../distributed/near/GridNearLockRequest.java | 1 - .../near/GridNearTransactionalCache.java | 1 - .../processors/cache/local/GridLocalCache.java | 1 - .../cache/local/GridLocalLockFuture.java | 1 - .../local/atomic/GridLocalAtomicCache.java | 1 - .../query/GridCacheDistributedQueryFuture.java | 3 +- .../query/GridCacheDistributedQueryManager.java | 4 +- .../cache/query/GridCacheLocalQueryFuture.java | 1 - .../cache/query/GridCacheQueriesImpl.java | 1 - .../cache/query/GridCacheQueriesProxy.java | 1 - .../cache/query/GridCacheQueryAdapter.java | 1 - .../query/GridCacheQueryFutureAdapter.java | 5 +- .../cache/query/GridCacheQueryInfo.java | 1 - .../cache/query/GridCacheQueryManager.java | 4 +- .../cache/query/GridCacheQueryRequest.java | 1 - .../GridCacheContinuousQueryAdapter.java | 21 +- .../GridCacheContinuousQueryEntry.java | 4 +- .../GridCacheContinuousQueryHandler.java | 6 +- .../GridCacheContinuousQueryManager.java | 1 - .../jdbc/GridCacheQueryJdbcMetadataTask.java | 1 - .../query/jdbc/GridCacheQueryJdbcTask.java | 1 - .../cache/transactions/IgniteTxEntry.java | 1 - .../cache/transactions/IgniteTxEx.java | 4 +- .../transactions/IgniteTxLocalAdapter.java | 1 - .../cache/transactions/IgniteTxLocalEx.java | 1 - .../clock/GridClockDeltaSnapshot.java | 7 +- .../processors/clock/GridClockServer.java | 3 +- .../clock/GridClockSyncProcessor.java | 3 +- .../closure/GridClosureProcessor.java | 2 - .../continuous/GridContinuousProcessor.java | 9 +- .../dataload/GridDataLoadCacheUpdaters.java | 2 - .../dataload/GridDataLoaderProcessor.java | 5 +- .../dataload/IgniteDataLoaderImpl.java | 22 +- .../processors/ggfs/GridGgfsDataManager.java | 9 +- .../processors/ggfs/GridGgfsDeleteWorker.java | 3 +- .../ggfs/GridGgfsFileWorkerBatch.java | 3 +- .../ggfs/GridGgfsFragmentizerManager.java | 7 +- .../processors/ggfs/GridGgfsMetaManager.java | 4 +- .../processors/ggfs/GridGgfsProcessor.java | 2 - .../kernal/processors/ggfs/GridGgfsServer.java | 5 +- .../kernal/processors/job/GridJobProcessor.java | 3 +- .../kernal/processors/job/GridJobWorker.java | 5 +- .../processors/query/GridQueryProcessor.java | 5 +- .../handlers/cache/GridCacheCommandHandler.java | 1 - .../cache/GridCacheQueryCommandHandler.java | 1 - .../service/GridServiceProcessor.java | 6 +- .../processors/streamer/IgniteStreamerImpl.java | 13 +- .../processors/task/GridTaskProcessor.java | 12 +- .../grid/kernal/visor/cache/VisorCache.java | 1 - .../cache/VisorCacheAffinityConfiguration.java | 1 - .../kernal/visor/cache/VisorCacheClearTask.java | 1 - .../visor/cache/VisorCacheCompactTask.java | 1 - .../visor/cache/VisorCacheConfiguration.java | 1 - .../kernal/visor/cache/VisorCacheLoadTask.java | 1 - .../visor/cache/VisorCacheMetadataTask.java | 1 - .../kernal/visor/cache/VisorCacheMetrics.java | 1 - .../cache/VisorCacheMetricsCollectorTask.java | 1 - .../cache/VisorCachePreloadConfiguration.java | 1 - .../visor/cache/VisorCachePreloadTask.java | 1 - .../visor/cache/VisorCacheResetMetricsTask.java | 1 - .../visor/cache/VisorCacheSwapBackupsTask.java | 1 - .../visor/node/VisorNodeDataCollectorJob.java | 1 - .../grid/kernal/visor/query/VisorQueryTask.java | 1 - .../main/java/org/gridgain/grid/package.html | 23 - .../grid/util/GridAtomicInitializer.java | 5 +- .../java/org/gridgain/grid/util/GridUtils.java | 66 ++- .../grid/util/future/GridEmbeddedFuture.java | 5 +- .../grid/util/future/GridFutureAdapter.java | 5 +- .../grid/util/future/GridFutureAdapterEx.java | 5 +- .../GridIpcSharedMemoryServerEndpoint.java | 7 +- .../grid/util/lang/GridComputeJobWrapper.java | 9 +- .../org/gridgain/grid/util/lang/GridFunc.java | 46 +- .../util/lang/GridMetadataAwareAdapter.java | 7 +- .../gridgain/grid/util/nio/GridNioFuture.java | 7 +- .../grid/util/nio/GridNioFutureImpl.java | 5 +- .../gridgain/grid/util/nio/GridNioServer.java | 11 +- .../org/gridgain/grid/util/typedef/PCE.java | 1 - .../gridgain/grid/util/worker/GridWorker.java | 7 +- ...eCacheEntryListenerEagerTtlDisabledTest.java | 1 - ...cpCommunicationSpiMultithreadedSelfTest.java | 3 +- .../index/GridStreamerIndexSelfTest.java | 3 +- .../kernal/GridJobMasterLeaveAwareSelfTest.java | 9 +- .../GridCheckpointManagerAbstractSelfTest.java | 5 +- .../cache/GridCacheGenericTestStore.java | 3 +- .../GridCachePartitionedAffinitySpreadTest.java | 2 + .../GridCachePreloadingEvictionsSelfTest.java | 5 +- .../processors/cache/GridCacheTestStore.java | 3 +- ...BehindStorePartitionedMultiNodeSelfTest.java | 5 +- ...ridCacheQueueJoinedNodeSelfAbstractTest.java | 11 +- .../dht/GridCacheDhtPreloadDelayedSelfTest.java | 4 +- ...eAtomicInvalidPartitionHandlingSelfTest.java | 3 +- .../GridCachePartitionedAffinitySelfTest.java | 1 + .../GridCacheEmptyEntriesAbstractSelfTest.java | 6 +- ...ridCacheContinuousQueryAbstractSelfTest.java | 114 ++-- ...heGgfsPerBlockLruEvictionPolicySelfTest.java | 4 +- .../ggfs/GridGgfsAbstractSelfTest.java | 5 +- .../streamer/GridStreamerSelfTest.java | 3 +- .../gridgain/grid/lang/GridTupleSelfTest.java | 5 +- .../communication/GridIoManagerBenchmark.java | 5 +- .../GridContinuousOperationsLoadTest.java | 11 +- ...JobExecutionSingleNodeSemaphoreLoadTest.java | 3 +- .../streamer/GridStreamerIndexLoadTest.java | 3 +- .../loadtests/streamer/QueryClosure.java | 5 +- .../GridTestSafeThreadFactory.java | 3 +- .../gridgain/testframework/GridTestUtils.java | 13 +- .../bamboo/GridDataGridTestSuite.java | 2 +- .../tests/p2p/GridCacheDeploymentTestTask1.java | 1 - .../tests/p2p/GridExternalAffinityFunction.java | 1 - .../p2p/GridExternalAffinityKeyMapper.java | 1 - .../GridUriDeploymentAbstractTestTask.java | 1 - .../kernal/ggfs/hadoop/GridGgfsHadoopIpcIo.java | 3 +- .../hadoop/jobtracker/GridHadoopJobTracker.java | 1 - .../hadoop/shuffle/GridHadoopShuffleJob.java | 5 +- .../taskexecutor/GridHadoopExecutorService.java | 3 +- .../hadoop/v2/GridHadoopV2CleanupTask.java | 3 +- .../hadoop/v2/GridHadoopV2MapTask.java | 3 +- .../hadoop/v2/GridHadoopV2ReduceTask.java | 3 +- .../hadoop/v2/GridHadoopV2SetupTask.java | 3 +- .../hadoop/v2/GridHadoopV2Splitter.java | 3 +- .../hibernate/GridCacheHibernateBlobStore.java | 3 +- .../processors/query/h2/GridH2Indexing.java | 1 - .../cache/GridCacheAbstractQuerySelfTest.java | 2 + .../query/h2/GridH2IndexRebuildTest.java | 3 +- .../grid/cache/jta/GridCacheTmLookup.java | 2 - .../cache/jta/jndi/GridCacheJndiTmLookup.java | 6 +- .../reflect/GridCacheReflectionTmLookup.java | 2 +- .../GridTmLookupLifecycleAwareSelfTest.java | 2 +- .../org/gridgain/scalar/ScalarConversions.scala | 1 - .../gridgain/scalar/pimps/ScalarCachePimp.scala | 1 - .../pimps/ScalarCacheProjectionPimp.scala | 1 - .../scalar/pimps/ScalarProjectionPimp.scala | 4 +- .../main/scala/org/gridgain/scalar/scalar.scala | 4 - .../processors/schedule/ScheduleFutureImpl.java | 5 +- .../util/nodestart/GridNodeCallableImpl.java | 7 +- .../deployment/uri/GridUriDeploymentSpi.java | 3 +- .../main/scala/org/gridgain/visor/visor.scala | 2 +- .../GridServletContextListenerStartup.java | 3 +- 317 files changed, 1496 insertions(+), 1831 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/examples/src/main/java/org/gridgain/examples/datagrid/CacheQueryExample.java ---------------------------------------------------------------------- diff --git a/examples/src/main/java/org/gridgain/examples/datagrid/CacheQueryExample.java b/examples/src/main/java/org/gridgain/examples/datagrid/CacheQueryExample.java index 3de6932..a8f906f 100644 --- a/examples/src/main/java/org/gridgain/examples/datagrid/CacheQueryExample.java +++ b/examples/src/main/java/org/gridgain/examples/datagrid/CacheQueryExample.java @@ -22,7 +22,6 @@ import org.apache.ignite.cache.*; import org.apache.ignite.cache.affinity.*; import org.apache.ignite.cache.query.*; import org.apache.ignite.lang.*; -import org.gridgain.grid.cache.affinity.*; import java.io.*; import java.util.*; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/examples/src/main/scala/org/gridgain/scalar/examples/ScalarCacheQueryExample.scala ---------------------------------------------------------------------- diff --git a/examples/src/main/scala/org/gridgain/scalar/examples/ScalarCacheQueryExample.scala b/examples/src/main/scala/org/gridgain/scalar/examples/ScalarCacheQueryExample.scala index a938d8e..759d850 100644 --- a/examples/src/main/scala/org/gridgain/scalar/examples/ScalarCacheQueryExample.scala +++ b/examples/src/main/scala/org/gridgain/scalar/examples/ScalarCacheQueryExample.scala @@ -24,7 +24,6 @@ import scalar._ import org.apache.ignite._ import org.gridgain.grid.cache._ import GridCacheMode._ -import affinity._ import java.util._ /** http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpi.java ---------------------------------------------------------------------- diff --git a/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpi.java b/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpi.java index be4f950..7d0c8bd 100644 --- a/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpi.java +++ b/modules/aws/src/main/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpi.java @@ -25,7 +25,6 @@ import org.apache.ignite.*; import org.apache.ignite.marshaller.*; import org.apache.ignite.resources.*; import org.apache.ignite.spi.*; -import org.gridgain.grid.*; import org.apache.ignite.spi.checkpoint.*; import org.gridgain.grid.util.typedef.*; import org.gridgain.grid.util.typedef.internal.*; @@ -282,7 +281,7 @@ public class GridS3CheckpointSpi extends IgniteSpiAdapter implements CheckpointS try { U.sleep(200); } - catch (GridInterruptedException e) { + catch (IgniteInterruptedException e) { throw new IgniteSpiException("Thread has been interrupted.", e); } } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/GridTcpDiscoveryS3IpFinder.java ---------------------------------------------------------------------- diff --git a/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/GridTcpDiscoveryS3IpFinder.java b/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/GridTcpDiscoveryS3IpFinder.java index 1b1cefe..3975ebb 100644 --- a/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/GridTcpDiscoveryS3IpFinder.java +++ b/modules/aws/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/GridTcpDiscoveryS3IpFinder.java @@ -24,7 +24,6 @@ import com.amazonaws.services.s3.model.*; import org.apache.ignite.*; import org.apache.ignite.resources.*; import org.apache.ignite.spi.*; -import org.gridgain.grid.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; import org.gridgain.grid.util.typedef.*; import org.gridgain.grid.util.typedef.internal.*; @@ -250,7 +249,7 @@ public class GridTcpDiscoveryS3IpFinder extends TcpDiscoveryIpFinderAdapter { try { U.sleep(200); } - catch (GridInterruptedException e) { + catch (IgniteInterruptedException e) { throw new IgniteSpiException("Thread has been interrupted.", e); } } @@ -270,7 +269,7 @@ public class GridTcpDiscoveryS3IpFinder extends TcpDiscoveryIpFinderAdapter { try { U.await(initLatch); } - catch (GridInterruptedException e) { + catch (IgniteInterruptedException e) { throw new IgniteSpiException("Thread has been interrupted.", e); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpiSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpiSelfTest.java b/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpiSelfTest.java index ebf96d0..ac1f3e8 100644 --- a/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpiSelfTest.java +++ b/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/GridS3CheckpointSpiSelfTest.java @@ -207,9 +207,9 @@ public class GridS3CheckpointSpiSelfTest extends GridSpiAbstractTest<GridS3Check * Wrapper around {@link GridTestUtils#retryAssert(org.apache.ignite.IgniteLogger, int, long, GridAbsClosure)}. * Provides s3-specific timeouts. * @param assertion Closure with assertion inside. - * @throws GridInterruptedException If was interrupted. + * @throws org.apache.ignite.IgniteInterruptedException If was interrupted. */ - private void assertWithRetries(GridAbsClosureX assertion) throws GridInterruptedException { + private void assertWithRetries(GridAbsClosureX assertion) throws IgniteInterruptedException { GridTestUtils.retryAssert(log, 6, 5000, assertion); } } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/clients/src/test/java/org/apache/ignite/client/impl/GridClientPartitionAffinitySelfTest.java ---------------------------------------------------------------------- diff --git a/modules/clients/src/test/java/org/apache/ignite/client/impl/GridClientPartitionAffinitySelfTest.java b/modules/clients/src/test/java/org/apache/ignite/client/impl/GridClientPartitionAffinitySelfTest.java index 6827eff..af644ac 100644 --- a/modules/clients/src/test/java/org/apache/ignite/client/impl/GridClientPartitionAffinitySelfTest.java +++ b/modules/clients/src/test/java/org/apache/ignite/client/impl/GridClientPartitionAffinitySelfTest.java @@ -28,6 +28,8 @@ import org.gridgain.testframework.junits.common.*; import java.util.*; +import static org.apache.ignite.cache.affinity.consistenthash.GridCacheConsistentHashAffinityFunction.*; + /** * Client's partitioned affinity tests. */ http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/Ignite.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java index e023c15..ad6848b 100644 --- a/modules/core/src/main/java/org/apache/ignite/Ignite.java +++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java @@ -23,7 +23,6 @@ import org.apache.ignite.configuration.*; import org.apache.ignite.fs.IgniteFsConfiguration; import org.apache.ignite.plugin.*; import org.apache.ignite.product.*; -import org.gridgain.grid.cache.*; import org.apache.ignite.hadoop.*; import org.apache.ignite.plugin.security.*; import org.gridgain.grid.util.typedef.*; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteAuthenticationException.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteAuthenticationException.java b/modules/core/src/main/java/org/apache/ignite/IgniteAuthenticationException.java new file mode 100644 index 0000000..e3ebd60 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/IgniteAuthenticationException.java @@ -0,0 +1,35 @@ +/* + * 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; + +/** + * Exception that represents authentication failure. + */ +public class IgniteAuthenticationException extends IgniteCheckedException { + /** */ + private static final long serialVersionUID = 0L; + + /** + * Creates authentication exception with given error message. + * + * @param msg Error message. + */ + public IgniteAuthenticationException(String msg) { + super(msg); + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteBasicWarmupClosure.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteBasicWarmupClosure.java b/modules/core/src/main/java/org/apache/ignite/IgniteBasicWarmupClosure.java new file mode 100644 index 0000000..10632c6 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/IgniteBasicWarmupClosure.java @@ -0,0 +1,560 @@ +/* + * 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; + +import org.apache.ignite.cache.*; +import org.apache.ignite.configuration.*; +import org.apache.ignite.lang.*; +import org.apache.ignite.logger.*; +import org.apache.ignite.spi.discovery.tcp.*; +import org.apache.ignite.spi.discovery.tcp.ipfinder.*; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*; +import org.gridgain.grid.util.typedef.*; +import org.gridgain.grid.util.typedef.internal.*; + +import java.text.*; +import java.util.*; +import java.util.concurrent.*; + +/** + * Basic warm-up closure which warm-ups cache operations. + */ +public class IgniteBasicWarmupClosure implements IgniteInClosure<IgniteConfiguration> { + /** */ + private static final long serialVersionUID = 9175346848249957458L; + + /** Default grid count to warm up. */ + public static final int DFLT_GRID_CNT = 2; + + /** Default iteration count per thread. */ + public static final int DFLT_ITERATION_CNT = 30_000; + + /** Default key range. */ + public static final int DFLT_KEY_RANGE = 10_000; + + /** Grid count. */ + private int gridCnt = DFLT_GRID_CNT; + + /** Warmup date format. */ + private static final SimpleDateFormat WARMUP_DATE_FMT = new SimpleDateFormat("HH:mm:ss"); + + /** Warmup thread count. */ + private int threadCnt = Runtime.getRuntime().availableProcessors() * 2; + + /** Per thread iteration count. */ + private int iterCnt = DFLT_ITERATION_CNT; + + /** Key range. */ + private int keyRange = DFLT_KEY_RANGE; + + /** Warmup discovery port. */ + private int discoveryPort = 27000; + + /** Methods to warmup. */ + private String[] warmupMethods = {"put", "putx", "get", "remove", "removex", "putIfAbsent", "replace"}; + + /** + * Gets number of grids to start and run warmup. + * + * @return Number of grids. + */ + public int getGridCount() { + return gridCnt; + } + + /** + * Sets number of grids to start and run the warmup. + * + * @param gridCnt Number of grids. + */ + public void setGridCount(int gridCnt) { + this.gridCnt = gridCnt; + } + + /** + * Gets warmup methods to use for cache warmup. + * + * @return Warmup methods. + */ + public String[] getWarmupMethods() { + return warmupMethods; + } + + /** + * Sets warmup methods to use for cache warmup. + * + * @param warmupMethods Array of warmup methods. + */ + public void setWarmupMethods(String... warmupMethods) { + this.warmupMethods = warmupMethods; + } + + /** + * Gets thread count for warmup. + * + * @return Thread count. + */ + public int getThreadCount() { + return threadCnt; + } + + /** + * Sets thread count for warmup. + * + * @param threadCnt Thread count. + */ + public void setThreadCount(int threadCnt) { + this.threadCnt = threadCnt; + } + + /** + * Gets iteration count for warmup. + * + * @return Iteration count. + */ + public int getIterationCount() { + return iterCnt; + } + + /** + * Sets iteration count for warmup. + * + * @param iterCnt Iteration count for warmup. + */ + public void setIterationCount(int iterCnt) { + this.iterCnt = iterCnt; + } + + /** + * Gets key range. + * + * @return Key range. + */ + public int getKeyRange() { + return keyRange; + } + + /** + * Sets key range. + * + * @param keyRange Key range. + */ + public void setKeyRange(int keyRange) { + this.keyRange = keyRange; + } + + /** + * Gets discovery port for warmup. + * + * @return Discovery port. + */ + public int getDiscoveryPort() { + return discoveryPort; + } + + /** + * Sets discovery port for warmup. + * + * @param discoveryPort Discovery port. + */ + public void setDiscoveryPort(int discoveryPort) { + this.discoveryPort = discoveryPort; + } + + /** {@inheritDoc} */ + @Override public void apply(IgniteConfiguration gridCfg) { + // Remove cache duplicates, clean up the rest, etc. + IgniteConfiguration cfg = prepareConfiguration(gridCfg); + + // Do nothing if no caches found. + if (cfg == null) + return; + + out("Starting grids to warmup caches [gridCnt=" + gridCnt + + ", caches=" + cfg.getCacheConfiguration().length + ']'); + + Collection<Ignite> ignites = new LinkedList<>(); + + String old = System.getProperty(IgniteSystemProperties.GG_UPDATE_NOTIFIER); + + try { + System.setProperty(IgniteSystemProperties.GG_UPDATE_NOTIFIER, "false"); + + TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + for (int i = 0; i < gridCnt; i++) { + IgniteConfiguration cfg0 = new IgniteConfiguration(cfg); + + TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); + + discoSpi.setIpFinder(ipFinder); + + discoSpi.setLocalPort(discoveryPort); + + cfg0.setDiscoverySpi(discoSpi); + + cfg0.setGridLogger(new IgniteNullLogger()); + + cfg0.setGridName("gridgain-warmup-grid-" + i); + + ignites.add(Ignition.start(cfg0)); + } + + doWarmup(ignites); + } + catch (Exception e) { + throw new IgniteException(e); + } + finally { + for (Ignite ignite : ignites) + Ignition.stop(ignite.name(), false); + + out("Stopped warmup grids."); + + if (old == null) + old = "false"; + + System.setProperty(IgniteSystemProperties.GG_UPDATE_NOTIFIER, old); + } + } + + /** + * @param grids Grids to warmup. + */ + private void doWarmup(Iterable<Ignite> grids) throws Exception { + Ignite first = F.first(grids); + + ExecutorService svc = Executors.newFixedThreadPool(threadCnt); + + try { + for (GridCache<?, ?> cache : first.caches()) { + GridCache<Object, Object> cache0 = first.cache(cache.name()); + + for (String warmupMethod : warmupMethods) { + Collection<Future> futs = new ArrayList<>(threadCnt); + + for (int i = 0; i < threadCnt; i++) { + Callable call; + + switch (warmupMethod) { + case "get": { + call = new GetCallable(cache0); + + break; + } + + case "put": { + call = new PutCallable(cache0); + + break; + } + + case "putx": { + call = new PutxCallable(cache0); + + break; + } + + case "remove": { + call = new RemoveCallable(cache0); + + break; + } + + case "removex": { + call = new RemovexCallable(cache0); + + break; + } + + case "putIfAbsent": { + call = new PutIfAbsentCallable(cache0); + + break; + } + + case "replace": { + call = new ReplaceCallable(cache0); + + break; + } + + default: + throw new IgniteCheckedException("Unsupported warmup method: " + warmupMethod); + } + + futs.add(svc.submit(call)); + } + + out("Running warmup [cacheName=" + cache.name() + ", method=" + warmupMethod + ']'); + + for (Future fut : futs) + fut.get(); + + for (int key = 0; key < keyRange; key++) + cache0.remove(key); + } + } + } + finally { + svc.shutdownNow(); + } + } + + /** + * Output for warmup messages. + * + * @param msg Format message. + */ + private static void out(String msg) { + System.out.println('[' + WARMUP_DATE_FMT.format(new Date(System.currentTimeMillis())) + "][WARMUP][" + + Thread.currentThread().getName() + ']' + ' ' + msg); + } + + /** + * Prepares configuration for warmup. + * + * @param gridCfg Original grid configuration. + * @return Prepared configuration or {@code null} if no caches found. + */ + private IgniteConfiguration prepareConfiguration(IgniteConfiguration gridCfg) { + if (F.isEmpty(gridCfg.getCacheConfiguration())) + return null; + + IgniteConfiguration cp = new IgniteConfiguration(); + + cp.setClientConnectionConfiguration(null); + + Collection<CacheConfiguration> reduced = new ArrayList<>(); + + for (CacheConfiguration ccfg : gridCfg.getCacheConfiguration()) { + if (CU.isSystemCache(ccfg.getName())) + continue; + + if (!matches(reduced, ccfg)) { + CacheConfiguration ccfgCp = new CacheConfiguration(ccfg); + + if (ccfgCp.getDistributionMode() == GridCacheDistributionMode.CLIENT_ONLY) + ccfgCp.setDistributionMode(GridCacheDistributionMode.PARTITIONED_ONLY); + else if (ccfgCp.getDistributionMode() == GridCacheDistributionMode.NEAR_ONLY) + ccfgCp.setDistributionMode(GridCacheDistributionMode.NEAR_PARTITIONED); + + ccfgCp.setCacheStoreFactory(null); + ccfgCp.setWriteBehindEnabled(false); + + reduced.add(ccfgCp); + } + } + + if (F.isEmpty(reduced)) + return null; + + CacheConfiguration[] res = new CacheConfiguration[reduced.size()]; + + reduced.toArray(res); + + cp.setCacheConfiguration(res); + + return cp; + } + + /** + * Checks if passed configuration matches one of the configurations in the list. + * + * @param reduced Reduced configurations. + * @param ccfg Cache configuration to match. + * @return {@code True} if matching configuration is found, {@code false} otherwise. + */ + private boolean matches(Iterable<CacheConfiguration> reduced, CacheConfiguration ccfg) { + for (CacheConfiguration ccfg0 : reduced) { + if (matches(ccfg0, ccfg)) + return true; + } + + return false; + } + + /** + * Checks if cache configurations are alike for warmup. + * + * @param ccfg0 First configuration. + * @param ccfg1 Second configuration. + * @return {@code True} if configurations match. + */ + private boolean matches(CacheConfiguration ccfg0, CacheConfiguration ccfg1) { + return + F.eq(ccfg0.getCacheMode(), ccfg1.getCacheMode()) && + F.eq(ccfg0.getBackups(), ccfg1.getBackups()) && + F.eq(ccfg0.getAtomicityMode(), ccfg1.getAtomicityMode()) && + F.eq(ccfg0.getAtomicWriteOrderMode(), ccfg1.getAtomicWriteOrderMode()) && + F.eq(ccfg0.getMemoryMode(), ccfg1.getMemoryMode()) && + F.eq(ccfg0.getDistributionMode(), ccfg1.getDistributionMode()); + } + + /** + * Base class for all warmup callables. + */ + private abstract class BaseWarmupCallable implements Callable<Object> { + /** Cache. */ + protected final GridCache<Object, Object> cache; + + /** + * @param cache Cache. + */ + protected BaseWarmupCallable(GridCache<Object, Object> cache) { + this.cache = cache; + } + + /** {@inheritDoc} */ + @Override public Object call() throws Exception { + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + for (int i = 0; i < iterCnt; i++) + operation(rnd.nextInt(keyRange)); + + return null; + } + + /** + * Runs operation. + * + * @param key Key. + * @throws Exception If failed. + */ + protected abstract void operation(int key) throws Exception; + } + + /** + * + */ + private class GetCallable extends BaseWarmupCallable { + /** + * @param cache Cache. + */ + private GetCallable(GridCache<Object, Object> cache) { + super(cache); + } + + /** {@inheritDoc} */ + @Override protected void operation(int key) throws Exception { + cache.get(key); + } + } + + /** + * + */ + private class PutCallable extends BaseWarmupCallable { + /** + * @param cache Cache. + */ + private PutCallable(GridCache<Object, Object> cache) { + super(cache); + } + + /** {@inheritDoc} */ + @Override protected void operation(int key) throws Exception { + cache.put(key, key); + } + } + + /** + * + */ + private class PutxCallable extends BaseWarmupCallable { + /** + * @param cache Cache. + */ + private PutxCallable(GridCache<Object, Object> cache) { + super(cache); + } + + /** {@inheritDoc} */ + @Override protected void operation(int key) throws Exception { + cache.putx(key, key); + } + } + + /** + * + */ + private class RemoveCallable extends BaseWarmupCallable { + /** + * @param cache Cache. + */ + private RemoveCallable(GridCache<Object, Object> cache) { + super(cache); + } + + /** {@inheritDoc} */ + @Override protected void operation(int key) throws Exception { + cache.remove(key); + } + } + + /** + * + */ + private class RemovexCallable extends BaseWarmupCallable { + /** + * @param cache Cache. + */ + private RemovexCallable(GridCache<Object, Object> cache) { + super(cache); + } + + /** {@inheritDoc} */ + @Override protected void operation(int key) throws Exception { + cache.removex(key); + } + } + + /** + * + */ + private class PutIfAbsentCallable extends BaseWarmupCallable { + /** + * @param cache Cache. + */ + private PutIfAbsentCallable(GridCache<Object, Object> cache) { + super(cache); + } + + /** {@inheritDoc} */ + @Override protected void operation(int key) throws Exception { + cache.putIfAbsent(key, key); + } + } + + /** + * + */ + private class ReplaceCallable extends BaseWarmupCallable { + /** + * @param cache Cache. + */ + private ReplaceCallable(GridCache<Object, Object> cache) { + super(cache); + } + + /** {@inheritDoc} */ + @Override protected void operation(int key) throws Exception { + cache.replace(key, key, key); + } + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteCache.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java index c7c125d..b16f9df 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java @@ -21,7 +21,6 @@ import org.apache.ignite.cache.*; import org.apache.ignite.cache.query.*; import org.apache.ignite.cache.store.*; import org.apache.ignite.lang.*; -import org.gridgain.grid.cache.*; import org.jetbrains.annotations.*; import javax.cache.*; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java index 042897c..bde9f9c 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java @@ -19,7 +19,6 @@ package org.apache.ignite; import org.apache.ignite.cluster.*; import org.apache.ignite.lang.*; -import org.gridgain.grid.cache.affinity.*; import org.gridgain.grid.util.lang.*; import org.jetbrains.annotations.*; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java b/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java index 127f86c..a4b41dd 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java @@ -19,8 +19,6 @@ package org.apache.ignite; import org.apache.ignite.dataload.*; import org.apache.ignite.lang.*; -import org.gridgain.grid.*; -import org.gridgain.grid.cache.*; import org.jetbrains.annotations.*; import java.util.*; @@ -238,11 +236,11 @@ public interface IgniteDataLoader<K, V> extends AutoCloseable { * @param key Key. * @return Future fo this operation. * @throws IgniteCheckedException If failed to map key to node. - * @throws GridInterruptedException If thread has been interrupted. + * @throws IgniteInterruptedException If thread has been interrupted. * @throws IllegalStateException If grid has been concurrently stopped or * {@link #close(boolean)} has already been called on loader. */ - public IgniteFuture<?> removeData(K key) throws IgniteCheckedException, GridInterruptedException, IllegalStateException; + public IgniteFuture<?> removeData(K key) throws IgniteCheckedException, IgniteInterruptedException, IllegalStateException; /** * Adds data for loading on remote node. This method can be called from multiple @@ -256,11 +254,11 @@ public interface IgniteDataLoader<K, V> extends AutoCloseable { * @param val Value or {@code null} if respective entry must be removed from cache. * @return Future fo this operation. * @throws IgniteCheckedException If failed to map key to node. - * @throws GridInterruptedException If thread has been interrupted. + * @throws IgniteInterruptedException If thread has been interrupted. * @throws IllegalStateException If grid has been concurrently stopped or * {@link #close(boolean)} has already been called on loader. */ - public IgniteFuture<?> addData(K key, @Nullable V val) throws IgniteCheckedException, GridInterruptedException, + public IgniteFuture<?> addData(K key, @Nullable V val) throws IgniteCheckedException, IgniteInterruptedException, IllegalStateException; /** @@ -274,11 +272,11 @@ public interface IgniteDataLoader<K, V> extends AutoCloseable { * @param entry Entry. * @return Future fo this operation. * @throws IgniteCheckedException If failed to map key to node. - * @throws GridInterruptedException If thread has been interrupted. + * @throws IgniteInterruptedException If thread has been interrupted. * @throws IllegalStateException If grid has been concurrently stopped or * {@link #close(boolean)} has already been called on loader. */ - public IgniteFuture<?> addData(Map.Entry<K, V> entry) throws IgniteCheckedException, GridInterruptedException, + public IgniteFuture<?> addData(Map.Entry<K, V> entry) throws IgniteCheckedException, IgniteInterruptedException, IllegalStateException; /** @@ -321,33 +319,33 @@ public interface IgniteDataLoader<K, V> extends AutoCloseable { * use {@link #tryFlush()} method. * * @throws IgniteCheckedException If failed to map key to node. - * @throws GridInterruptedException If thread has been interrupted. + * @throws IgniteInterruptedException If thread has been interrupted. * @throws IllegalStateException If grid has been concurrently stopped or * {@link #close(boolean)} has already been called on loader. * @see #tryFlush() */ - public void flush() throws IgniteCheckedException, GridInterruptedException, IllegalStateException; + public void flush() throws IgniteCheckedException, IgniteInterruptedException, IllegalStateException; /** * Makes an attempt to load remaining data. This method is mostly similar to {@link #flush}, * with the difference that it won't wait and will exit immediately. * * @throws IgniteCheckedException If failed to map key to node. - * @throws GridInterruptedException If thread has been interrupted. + * @throws IgniteInterruptedException If thread has been interrupted. * @throws IllegalStateException If grid has been concurrently stopped or * {@link #close(boolean)} has already been called on loader. * @see #flush() */ - public void tryFlush() throws IgniteCheckedException, GridInterruptedException, IllegalStateException; + public void tryFlush() throws IgniteCheckedException, IgniteInterruptedException, IllegalStateException; /** * Loads any remaining data and closes this loader. * * @param cancel {@code True} to cancel ongoing loading operations. * @throws IgniteCheckedException If failed to map key to node. - * @throws GridInterruptedException If thread has been interrupted. + * @throws IgniteInterruptedException If thread has been interrupted. */ - public void close(boolean cancel) throws IgniteCheckedException, GridInterruptedException; + public void close(boolean cancel) throws IgniteCheckedException, IgniteInterruptedException; /** * Closes data loader. This method is identical to calling {@link #close(boolean) close(false)} method. @@ -356,7 +354,7 @@ public interface IgniteDataLoader<K, V> extends AutoCloseable { * {@code try-with-resources} statement. * * @throws IgniteCheckedException If failed to close data loader. - * @throws GridInterruptedException If thread has been interrupted. + * @throws IgniteInterruptedException If thread has been interrupted. */ - @Override public void close() throws IgniteCheckedException, GridInterruptedException; + @Override public void close() throws IgniteCheckedException, IgniteInterruptedException; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteDeploymentException.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteDeploymentException.java b/modules/core/src/main/java/org/apache/ignite/IgniteDeploymentException.java new file mode 100644 index 0000000..e5fd803 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/IgniteDeploymentException.java @@ -0,0 +1,57 @@ +/* + * 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; + +import org.jetbrains.annotations.*; + +/** + * Deployment or re-deployment failed. + */ +public class IgniteDeploymentException extends IgniteCheckedException { + /** */ + private static final long serialVersionUID = 0L; + + /** + * Creates new exception with given error message. + * + * @param msg Error message. + */ + public IgniteDeploymentException(String msg) { + super(msg); + } + + /** + * Creates new exception with given throwable as a nested cause and + * source of error message. + * + * @param cause Non-null throwable cause. + */ + public IgniteDeploymentException(Throwable cause) { + this(cause.getMessage(), cause); + } + + /** + * Creates a new exception with given error message and optional nested cause exception. + * + * @param msg Error message. + * @param cause Optional nested exception (can be {@code null}). + */ + public IgniteDeploymentException(String msg, @Nullable Throwable cause) { + super(msg, cause); + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteIllegalStateException.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteIllegalStateException.java b/modules/core/src/main/java/org/apache/ignite/IgniteIllegalStateException.java new file mode 100644 index 0000000..6dc44bb --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/IgniteIllegalStateException.java @@ -0,0 +1,84 @@ +/* + * 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; + +import org.gridgain.grid.util.typedef.*; +import org.jetbrains.annotations.*; + +/** + * This exception indicates the ignite access in invalid state. + */ +public class IgniteIllegalStateException extends IllegalStateException { + /** */ + private static final long serialVersionUID = 0L; + + /** + * Constructs exception with given message and cause. + * + * @param msg Exception message. + * @param cause Exception cause. + */ + public IgniteIllegalStateException(String msg, @Nullable Throwable cause) { + super(msg, cause); + } + + /** + * Creates exception given throwable as a cause and + * source of error message. + * + * @param cause Non-null throwable cause. + */ + public IgniteIllegalStateException(Throwable cause) { + this(cause.getMessage(), cause); + } + + /** + * Constructs exception with given message. + * + * @param msg Exception message. + */ + public IgniteIllegalStateException(String msg) { + super(msg); + } + + /** + * Checks if this exception has given class in {@code 'cause'} hierarchy. + * + * @param cls Cause class to check (if {@code null}, {@code false} is returned).. + * @return {@code True} if one of the causing exception is an instance of passed in class, + * {@code false} otherwise. + */ + public boolean hasCause(@Nullable Class<? extends Throwable>... cls) { + return X.hasCause(this, cls); + } + + /** + * Gets first exception of given class from {@code 'cause'} hierarchy if any. + * + * @param cls Cause class to get cause (if {@code null}, {@code null} is returned). + * @return First causing exception of passed in class, {@code null} otherwise. + */ + @Nullable public <T extends Throwable> T getCause(@Nullable Class<T> cls) { + return X.cause(this, cls); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return getClass() + ": " + getMessage(); + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteInterruptedException.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteInterruptedException.java b/modules/core/src/main/java/org/apache/ignite/IgniteInterruptedException.java new file mode 100644 index 0000000..1e7f267 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/IgniteInterruptedException.java @@ -0,0 +1,56 @@ +/* + * 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; + +/** + * This exception is used to wrap standard {@link InterruptedException} into {@link IgniteCheckedException}. + */ +@SuppressWarnings({"TypeMayBeWeakened"}) +public class IgniteInterruptedException extends IgniteCheckedException { + /** */ + private static final long serialVersionUID = 0L; + + /** + * Creates new exception with given throwable as a nested cause and + * source of error message. + * + * @param cause Non-null throwable cause. + */ + public IgniteInterruptedException(InterruptedException cause) { + this(cause.getMessage(), cause); + } + + /** + * Creates a new exception with given error message and optional nested cause exception. + * + * @param msg Error message. + */ + public IgniteInterruptedException(String msg) { + super(msg); + } + + /** + * Creates a new exception with given error message and optional nested cause exception. + * + * @param msg Error message. + * @param cause Optional nested exception (can be {@code null}). + */ + public IgniteInterruptedException(String msg, InterruptedException cause) { + super(msg, cause); + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteMetadataAware.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteMetadataAware.java b/modules/core/src/main/java/org/apache/ignite/IgniteMetadataAware.java new file mode 100644 index 0000000..a49520e --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/IgniteMetadataAware.java @@ -0,0 +1,187 @@ +/* + * 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; + +import org.jetbrains.annotations.*; +import java.io.*; +import java.util.*; +import java.util.concurrent.*; + +/** + * Allows to attach metadata attributes to any entity that implements + * this interface. This promotes <i>meta-programming technique</i> throughout the + * GridGain APIs by allowing to attach and query the <b>metadata information</b> to top-level + * entities in GridGain APIs. + * <p> + * Note that unlike other types of attributes the metadata is explicitly + * local to VM. Local metadata does not have any distribution semantic and is not + * specifically distributed in any way. Note, however, that if object that implements this + * interface gets serialized and deserialized on the remote node the attachments may or may not + * be carried over (depending on the implementation). All classes that come with GridGain + * and implement this interface support proper serialization and deserialization. + * <p> + * For example, this may become useful for cache entries or + * cache transactions. Cache entry attachment can be used whenever entry + * needs to carry additional context and it is too expensive to keep + * looking that context up from a separate map by a key. For example, + * an expiration policy used by caches may add some expiration + * context to cache entries to properly expire them. + */ +public interface IgniteMetadataAware extends Serializable { + /** + * Copies all metadata from another instance. + * + * @param from Metadata aware instance to copy metadata from. + */ + public void copyMeta(IgniteMetadataAware from); + + /** + * Copies all metadata from given map. + * + * @param data Map to copy metadata from. + */ + public void copyMeta(Map<String, ?> data); + + /** + * Adds a new metadata. + * + * @param name Metadata name. + * @param val Metadata value. + * @param <V> Type of the value. + * @return Metadata previously associated with given name, or + * {@code null} if there was none. + */ + @Nullable public <V> V addMeta(String name, V val); + + /** + * Adds given metadata value only if it was absent. + * + * @param name Metadata name. + * @param val Value to add if it's not attached already. + * @param <V> Type of the value. + * @return {@code null} if new value was put, or current value if put didn't happen. + */ + @Nullable public <V> V putMetaIfAbsent(String name, V val); + + /** + * Adds given metadata value only if it was absent. + * + * @param name Metadata name. + * @param c Factory closure to produce value to add if it's not attached already. + * Not that unlike {@link #addMeta(String, Object)} method the factory closure will + * not be called unless the value is required and therefore value will only be created + * when it is actually needed. + * @param <V> Type of the value. + * @return {@code null} if new value was put, or current value if put didn't happen. + */ + @Nullable public <V> V putMetaIfAbsent(String name, Callable<V> c); + + /** + * Adds given metadata value only if it was absent. Unlike + * {@link #putMetaIfAbsent(String, Callable)}, this method always returns + * the latest value and never previous one. + * + * @param name Metadata name. + * @param val Value to add if it's not attached already. + * @param <V> Type of the value. + * @return The value of the metadata after execution of this method. + */ + public <V> V addMetaIfAbsent(String name, V val); + + /** + * Adds given metadata value only if it was absent. + * + * @param name Metadata name. + * @param c Factory closure to produce value to add if it's not attached already. + * Not that unlike {@link #addMeta(String, Object)} method the factory closure will + * not be called unless the value is required and therefore value will only be created + * when it is actually needed. If {@code null} and metadata value is missing - {@code null} + * will be returned from this method. + * @param <V> Type of the value. + * @return The value of the metadata after execution of this method. + */ + @Nullable public <V> V addMetaIfAbsent(String name, @Nullable Callable<V> c); + + /** + * Gets metadata by name. + * + * @param name Metadata name. + * @param <V> Type of the value. + * @return Metadata value or {@code null}. + */ + public <V> V meta(String name); + + /** + * Removes metadata by name. + * + * @param name Name of the metadata to remove. + * @param <V> Type of the value. + * @return Value of removed metadata or {@code null}. + */ + public <V> V removeMeta(String name); + + /** + * Removes metadata only if its current value is equal to {@code val} passed in. + * + * @param name Name of metadata attribute. + * @param val Value to compare. + * @param <V> Value type. + * @return {@code True} if value was removed, {@code false} otherwise. + */ + public <V> boolean removeMeta(String name, V val); + + /** + * Gets all metadata in this entry. + * + * @param <V> Type of the value. + * @return All metadata in this entry. + */ + public <V> Map<String, V> allMeta(); + + /** + * Tests whether or not given metadata is set. + * + * @param name Name of the metadata to test. + * @return Whether or not given metadata is set. + */ + public boolean hasMeta(String name); + + /** + * Tests whether or not metadata with given name and value is set. Returns {@code true} if + * attachment with given name it set and its value is equals to provided value. Otherwise returns + * {@code false}. + * + * @param name Name of the metadata. + * @param val Value to compare. Cannot be {@code null}. + * @param <V> Type of the value. + * @return {@code true} if metadata with given name is set and its value is equals to provided value - + * otherwise returns {@code false}. + */ + public <V> boolean hasMeta(String name, V val); + + /** + * Replaces given metadata with new {@code newVal} value only if its current value + * is equal to {@code curVal}. Otherwise, it is no-op. + * + * @param name Name of the metadata. + * @param curVal Current value to check. + * @param newVal New value. + * @return {@code true} if replacement occurred, {@code false} otherwise. + */ + public <V> boolean replaceMeta(String name, V curVal, V newVal); +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteNotPeerDeployable.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteNotPeerDeployable.java b/modules/core/src/main/java/org/apache/ignite/IgniteNotPeerDeployable.java new file mode 100644 index 0000000..05cf0c8 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/IgniteNotPeerDeployable.java @@ -0,0 +1,41 @@ +/* + * 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; + +import java.lang.annotation.*; + +/** + * This annotations should be used to mark any type that should not be + * peer deployable. Peer deployment will fail for this object as if + * class could not be found. + * <p> + * This annotation is used as <b>non-distribution assertion</b> and should be + * applied to classes and interfaces that should never be distributed via + * peer-to-peer deployment. + * <p> + * Note, however, that if class is already available on the remote node it + * will not be peer-loaded but will simply be locally class loaded. It may appear + * as if it was successfully peer-loaded when in fact it was simply already + * available on the remote node. + */ +@Documented +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE}) +public @interface IgniteNotPeerDeployable { + // No-op. +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java b/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java index a204386..9f8542a 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java +++ b/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java @@ -19,7 +19,6 @@ package org.apache.ignite; import org.apache.ignite.cache.*; import org.apache.ignite.portables.*; -import org.gridgain.grid.cache.*; import org.jetbrains.annotations.*; import java.sql.*; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/IgniteTransactions.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteTransactions.java b/modules/core/src/main/java/org/apache/ignite/IgniteTransactions.java index bc85829..05277a1 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteTransactions.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteTransactions.java @@ -20,7 +20,6 @@ package org.apache.ignite; import org.apache.ignite.cache.*; import org.apache.ignite.configuration.*; import org.apache.ignite.transactions.*; -import org.gridgain.grid.cache.*; import org.jetbrains.annotations.*; /** http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/Ignition.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/Ignition.java b/modules/core/src/main/java/org/apache/ignite/Ignition.java index 1df3e0a..7abc5ac 100644 --- a/modules/core/src/main/java/org/apache/ignite/Ignition.java +++ b/modules/core/src/main/java/org/apache/ignite/Ignition.java @@ -19,7 +19,6 @@ package org.apache.ignite; import org.apache.ignite.configuration.*; import org.apache.ignite.lifecycle.*; -import org.gridgain.grid.*; import org.gridgain.grid.kernal.*; import org.jetbrains.annotations.*; @@ -338,10 +337,10 @@ public class Ignition { * * @return An instance of default no-name grid. This method never returns * {@code null}. - * @throws GridIllegalStateException Thrown if default grid was not properly + * @throws IgniteIllegalStateException Thrown if default grid was not properly * initialized or grid instance was stopped or was not started. */ - public static Ignite ignite() throws GridIllegalStateException { + public static Ignite ignite() throws IgniteIllegalStateException { return GridGainEx.grid(); } @@ -363,10 +362,10 @@ public class Ignition { * @param locNodeId ID of local node the requested grid instance is managing. * @return An instance of named grid. This method never returns * {@code null}. - * @throws GridIllegalStateException Thrown if grid was not properly + * @throws IgniteIllegalStateException Thrown if grid was not properly * initialized or grid instance was stopped or was not started. */ - public static Ignite ignite(UUID locNodeId) throws GridIllegalStateException { + public static Ignite ignite(UUID locNodeId) throws IgniteIllegalStateException { return GridGainEx.grid(locNodeId); } @@ -383,10 +382,10 @@ public class Ignition { * then grid instance belonging to a default no-name grid will be returned. * @return An instance of named grid. This method never returns * {@code null}. - * @throws GridIllegalStateException Thrown if default grid was not properly + * @throws IgniteIllegalStateException Thrown if default grid was not properly * initialized or grid instance was stopped or was not started. */ - public static Ignite ignite(@Nullable String name) throws GridIllegalStateException { + public static Ignite ignite(@Nullable String name) throws IgniteIllegalStateException { return GridGainEx.grid(name); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/CacheConfiguration.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheConfiguration.java index 0f6d59b..68675a0 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/CacheConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheConfiguration.java @@ -27,8 +27,6 @@ import org.apache.ignite.cache.store.*; import org.apache.ignite.configuration.*; import org.apache.ignite.portables.PortableObject; import org.apache.ignite.spi.indexing.*; -import org.gridgain.grid.cache.*; -import org.gridgain.grid.cache.affinity.*; import org.gridgain.grid.util.typedef.internal.*; import org.jetbrains.annotations.*; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/CachePartialUpdateException.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CachePartialUpdateException.java b/modules/core/src/main/java/org/apache/ignite/cache/CachePartialUpdateException.java index bd4499a..cdaeb05 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/CachePartialUpdateException.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/CachePartialUpdateException.java @@ -17,9 +17,6 @@ package org.apache.ignite.cache; -import org.apache.ignite.cache.*; -import org.gridgain.grid.cache.*; - import javax.cache.*; import java.util.*; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java b/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java index 3a613a4..3c099ff 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java @@ -18,13 +18,11 @@ package org.apache.ignite.cache; import org.apache.ignite.*; -import org.apache.ignite.cache.*; import org.apache.ignite.cache.affinity.*; import org.apache.ignite.cache.datastructures.*; import org.apache.ignite.cache.store.CacheStore; import org.apache.ignite.lang.*; import org.apache.ignite.transactions.*; -import org.gridgain.grid.cache.affinity.*; import org.jetbrains.annotations.*; import java.util.*; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/GridCacheEntry.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/GridCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/cache/GridCacheEntry.java index 120d59e..02bdddb 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/GridCacheEntry.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/GridCacheEntry.java @@ -20,7 +20,6 @@ package org.apache.ignite.cache; import org.apache.ignite.*; import org.apache.ignite.lang.*; import org.apache.ignite.transactions.*; -import org.gridgain.grid.*; import org.jetbrains.annotations.*; import javax.cache.*; @@ -88,7 +87,7 @@ import java.util.Map.*; * @param <K> Key type. * @param <V> Value type. */ -public interface GridCacheEntry<K, V> extends Map.Entry<K, V>, GridMetadataAware, Cache.Entry<K, V> { +public interface GridCacheEntry<K, V> extends Map.Entry<K, V>, IgniteMetadataAware, Cache.Entry<K, V> { /** * Cache projection to which this entry belongs. Note that entry and its * parent projections have same flags and filters. http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/GridCacheMode.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/GridCacheMode.java b/modules/core/src/main/java/org/apache/ignite/cache/GridCacheMode.java index 128d0e8..27fc454 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/GridCacheMode.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/GridCacheMode.java @@ -17,8 +17,6 @@ package org.apache.ignite.cache; -import org.apache.ignite.cache.*; -import org.gridgain.grid.cache.affinity.*; import org.jetbrains.annotations.*; /** http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/GridCacheName.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/GridCacheName.java b/modules/core/src/main/java/org/apache/ignite/cache/GridCacheName.java index 24068ee..367f4b9 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/GridCacheName.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/GridCacheName.java @@ -17,8 +17,6 @@ package org.apache.ignite.cache; -import org.gridgain.grid.cache.affinity.*; - import java.lang.annotation.*; import java.util.concurrent.*; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/GridCachePreloadMode.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/GridCachePreloadMode.java b/modules/core/src/main/java/org/apache/ignite/cache/GridCachePreloadMode.java index ea596e6..4ee32d9 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/GridCachePreloadMode.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/GridCachePreloadMode.java @@ -17,8 +17,6 @@ package org.apache.ignite.cache; -import org.apache.ignite.cache.*; -import org.gridgain.grid.cache.affinity.*; import org.jetbrains.annotations.*; /** http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/GridCacheProjection.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/GridCacheProjection.java b/modules/core/src/main/java/org/apache/ignite/cache/GridCacheProjection.java index 3f681b1..15977f3 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/GridCacheProjection.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/GridCacheProjection.java @@ -18,13 +18,11 @@ package org.apache.ignite.cache; import org.apache.ignite.*; -import org.apache.ignite.cache.*; import org.apache.ignite.cache.query.*; import org.apache.ignite.cache.store.*; import org.apache.ignite.cluster.*; import org.apache.ignite.lang.*; import org.apache.ignite.transactions.*; -import org.gridgain.grid.cache.affinity.*; import org.jetbrains.annotations.*; import java.sql.*; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityFunction.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityFunction.java index 439c231..c33bcad 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityFunction.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityFunction.java @@ -19,7 +19,6 @@ package org.apache.ignite.cache.affinity; import org.apache.ignite.cache.*; import org.apache.ignite.cluster.*; -import org.gridgain.grid.cache.*; import java.io.*; import java.util.*; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityKeyMapped.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityKeyMapped.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityKeyMapped.java index 4b38e01..3cbac33 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityKeyMapped.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/GridCacheAffinityKeyMapped.java @@ -17,8 +17,6 @@ package org.apache.ignite.cache.affinity; -import org.gridgain.grid.cache.*; - import java.lang.annotation.*; import java.util.concurrent.*; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/GridCacheConsistentHashAffinityFunction.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/GridCacheConsistentHashAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/GridCacheConsistentHashAffinityFunction.java index 628e852..01b470a 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/GridCacheConsistentHashAffinityFunction.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/consistenthash/GridCacheConsistentHashAffinityFunction.java @@ -23,8 +23,6 @@ import org.apache.ignite.cache.affinity.*; import org.apache.ignite.cluster.*; import org.apache.ignite.lang.*; import org.apache.ignite.resources.*; -import org.gridgain.grid.*; -import org.gridgain.grid.cache.affinity.*; import org.gridgain.grid.util.*; import org.gridgain.grid.util.tostring.*; import org.gridgain.grid.util.typedef.*; @@ -591,7 +589,7 @@ public class GridCacheConsistentHashAffinityFunction implements GridCacheAffinit try { U.await(initLatch); } - catch (GridInterruptedException ignored) { + catch (IgniteInterruptedException ignored) { // Recover interrupted state flag. Thread.currentThread().interrupt(); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinity.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinity.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinity.java index cfcf16b..d3d5c28 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinity.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinity.java @@ -22,7 +22,6 @@ import org.apache.ignite.cache.affinity.*; import org.apache.ignite.cluster.*; import org.apache.ignite.events.*; import org.apache.ignite.lang.*; -import org.gridgain.grid.cache.affinity.*; import org.gridgain.grid.util.typedef.*; import org.gridgain.grid.util.typedef.internal.*; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/GridCacheRendezvousAffinityFunction.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/GridCacheRendezvousAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/GridCacheRendezvousAffinityFunction.java index 088c389..b6e41a6 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/GridCacheRendezvousAffinityFunction.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/GridCacheRendezvousAffinityFunction.java @@ -24,7 +24,6 @@ import org.apache.ignite.cluster.*; import org.apache.ignite.lang.*; import org.apache.ignite.marshaller.*; import org.apache.ignite.marshaller.optimized.*; -import org.gridgain.grid.cache.affinity.*; import org.gridgain.grid.kernal.*; import org.gridgain.grid.util.typedef.*; import org.gridgain.grid.util.typedef.internal.*; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/550d6033/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheCloner.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheCloner.java b/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheCloner.java index e873d4a..892ee71 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheCloner.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/cloner/GridCacheCloner.java @@ -19,7 +19,6 @@ package org.apache.ignite.cache.cloner; import org.apache.ignite.*; import org.apache.ignite.cache.*; -import org.gridgain.grid.cache.*; import org.jetbrains.annotations.*; /**