Skip to content

Commit

Permalink
Merge branches 'ignite-383' and 'sprint-2' of https://git-wip-us.apac…
Browse files Browse the repository at this point in the history
…he.org/repos/asf/incubator-ignite into ignite-383

Conflicts:
	modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
	modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java
	modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java
  • Loading branch information
Yakov Zhdanov committed Mar 11, 2015
1 parent 6bb9dbc commit 1900787
Show file tree
Hide file tree
Showing 13 changed files with 21 additions and 29 deletions.
6 changes: 3 additions & 3 deletions modules/core/src/main/java/org/apache/ignite/IgniteCache.java
Expand Up @@ -459,20 +459,20 @@ public <T> Map<K, EntryProcessorResult<T>> invokeAll(Set<? extends K> keys,

/**
* This cache node to re-balance its partitions. This method is usually used when
* {@link CacheConfiguration#getRebalancePartitionedDelay()} configuration parameter has non-zero value.
* {@link CacheConfiguration#getRebalanceDelay()} configuration parameter has non-zero value.
* When many nodes are started or stopped almost concurrently, it is more efficient to delay
* rebalancing until the node topology is stable to make sure that no redundant re-partitioning
* happens.
* <p>
* In case of{@link CacheMode#PARTITIONED} caches, for better efficiency user should
* usually make sure that new nodes get placed on the same place of consistent hash ring as
* the left nodes, and that nodes are restarted before
* {@link CacheConfiguration#getRebalancePartitionedDelay() rebalanceDelay} expires. To place nodes
* {@link CacheConfiguration#getRebalanceDelay() rebalanceDelay} expires. To place nodes
* on the same place in consistent hash ring, use
* {@link CacheRendezvousAffinityFunction#setHashIdResolver(CacheAffinityNodeHashResolver)} to make sure that
* a node maps to the same hash ID if re-started.
* <p>
* See {@link CacheConfiguration#getRebalancePartitionedDelay()} for more information on how to configure
* See {@link CacheConfiguration#getRebalanceDelay()} for more information on how to configure
* rebalance re-partition delay.
* <p>
* @return Future that will be completed when rebalancing is finished.
Expand Down
Expand Up @@ -199,20 +199,20 @@ public interface GridCache<K, V> extends CacheProjection<K, V> {

/**
* Forces this cache node to re-balance its partitions. This method is usually used when
* {@link CacheConfiguration#getRebalancePartitionedDelay()} configuration parameter has non-zero value.
* {@link CacheConfiguration#getRebalanceDelay()} configuration parameter has non-zero value.
* When many nodes are started or stopped almost concurrently, it is more efficient to delay
* rebalancing until the node topology is stable to make sure that no redundant re-partitioning
* happens.
* <p>
* In case of{@link CacheMode#PARTITIONED} caches, for better efficiency user should
* usually make sure that new nodes get placed on the same place of consistent hash ring as
* the left nodes, and that nodes are restarted before
* {@link CacheConfiguration#getRebalancePartitionedDelay() rebalanceDelay} expires. To place nodes
* {@link CacheConfiguration#getRebalanceDelay() rebalanceDelay} expires. To place nodes
* on the same place in consistent hash ring, use
* {@link org.apache.ignite.cache.affinity.rendezvous.CacheRendezvousAffinityFunction#setHashIdResolver(CacheAffinityNodeHashResolver)} to make sure that
* a node maps to the same hash ID if re-started.
* <p>
* See {@link org.apache.ignite.configuration.CacheConfiguration#getRebalancePartitionedDelay()} for more information on how to configure
* See {@link org.apache.ignite.configuration.CacheConfiguration#getRebalanceDelay()} for more information on how to configure
* rebalance re-partition delay.
* <p>
* @return Future that will be completed when rebalancing is finished.
Expand Down
Expand Up @@ -382,7 +382,7 @@ public CacheConfiguration(CompleteConfiguration cfg) {
nearEvictPlc = cc.getNearEvictionPolicy();
rebalanceMode = cc.getRebalanceMode();
rebalanceBatchSize = cc.getRebalanceBatchSize();
rebalanceDelay = cc.getRebalancePartitionedDelay();
rebalanceDelay = cc.getRebalanceDelay();
rebalanceOrder = cc.getRebalanceOrder();
rebalancePoolSize = cc.getRebalanceThreadPoolSize();
rebalanceTimeout = cc.getRebalanceTimeout();
Expand Down Expand Up @@ -1321,7 +1321,6 @@ public void setRebalanceTimeout(long rebalanceTimeout) {
* after they leave topology, or if you plan to start multiple nodes at once or one after another
* and don't want to repartition and rebalance until all nodes are started.
* <p>
* Delayed rebalancing is applied to {@link CacheMode#PARTITIONED} caches only.
* For better efficiency user should usually make sure that new nodes get placed on
* the same place of consistent hash ring as the left nodes, and that nodes are
* restarted before this delay expires. To place nodes on the same place in consistent hash ring,
Expand All @@ -1338,16 +1337,16 @@ public void setRebalanceTimeout(long rebalanceTimeout) {
* start rebalancing manually, or positive value to specify delay in milliseconds
* after which rebalancing should start automatically.
*/
public long getRebalancePartitionedDelay() {
public long getRebalanceDelay() {
return rebalanceDelay;
}

/**
* Sets rebalance delay (see {@link #getRebalancePartitionedDelay()} for more information).
* Sets rebalance delay (see {@link #getRebalanceDelay()} for more information).
*
* @param rebalanceDelay Rebalance delay to set.
*/
public void setRebalancePartitionedDelay(long rebalanceDelay) {
public void setRebalanceDelay(long rebalanceDelay) {
this.rebalanceDelay = rebalanceDelay;
}

Expand Down
Expand Up @@ -1873,7 +1873,7 @@ private static CacheConfiguration marshallerSystemCache(boolean client) {
cache.setAtomicityMode(TRANSACTIONAL);
cache.setSwapEnabled(false);
cache.setQueryIndexEnabled(false);
cache.setPreloadMode(SYNC);
cache.setRebalanceMode(SYNC);
cache.setWriteSynchronizationMode(FULL_SYNC);
cache.setAffinity(new CacheRendezvousAffinityFunction(false, 100));

Expand Down
Expand Up @@ -390,14 +390,7 @@ public long defaultLockTimeout() {
}

/**
* @return Flag indicating if cached values should be additionally stored in serialized form.
*/
public boolean storeValueBytes() {
return storeValBytes;
}

/**
* @return Preload batch size.
* @return Rebalance batch size.
*/
public int rebalanceBatchSize() {
return rebalanceBatchSize;
Expand Down
Expand Up @@ -851,7 +851,7 @@ void addFuture(GridDhtPartitionsExchangeFuture exchFut) {
}

for (GridCacheContext<K, V> cacheCtx : cctx.cacheContexts()) {
long delay = cacheCtx.config().getRebalancePartitionedDelay();
long delay = cacheCtx.config().getRebalanceDelay();

GridDhtPreloaderAssignments<K, V> assigns = null;

Expand Down
Expand Up @@ -374,7 +374,7 @@ private void validate(IgniteConfiguration c,
throw new IgniteCheckedException("Cannot enable write-through (writer or store is not provided) " +
"for cache: " + cc.getName());

long delay = cc.getRebalancePartitionedDelay();
long delay = cc.getRebalanceDelay();

if (delay != 0) {
if (cc.getCacheMode() != PARTITIONED)
Expand Down Expand Up @@ -1321,7 +1321,7 @@ private void onKernalStart(GridCacheAdapter<?, ?> cache) throws IgniteCheckedExc

if (cfg.getRebalanceMode() == SYNC) {
if (cfg.getCacheMode() == REPLICATED ||
(cfg.getCacheMode() == PARTITIONED && cfg.getRebalancePartitionedDelay() >= 0))
(cfg.getCacheMode() == PARTITIONED && cfg.getRebalanceDelay() >= 0))
cache.preloader().syncFuture().get();
}
}
Expand Down
Expand Up @@ -330,7 +330,7 @@ void addAssignments(final GridDhtPreloaderAssignments<K, V> assigns, boolean for
if (log.isDebugEnabled())
log.debug("Adding partition assignments: " + assigns);

long delay = cctx.config().getRebalancePartitionedDelay();
long delay = cctx.config().getRebalanceDelay();

if (delay == 0 || force) {
assert assigns != null;
Expand Down
Expand Up @@ -226,7 +226,7 @@ public GridDhtPreloader(GridCacheContext<K, V> cctx) {

final long start = U.currentTimeMillis();

if (cctx.config().getRebalancePartitionedDelay() >= 0) {
if (cctx.config().getRebalanceDelay() >= 0) {
U.log(log, "Starting rebalancing in " + cctx.config().getRebalanceMode() + " mode: " + cctx.name());

demandPool.syncFuture().listen(new CI1<Object>() {
Expand Down
Expand Up @@ -58,7 +58,7 @@ public static VisorCachePreloadConfiguration from(CacheConfiguration ccfg) {
cfg.mode = ccfg.getRebalanceMode();
cfg.batchSize = ccfg.getRebalanceBatchSize();
cfg.threadPoolSize = ccfg.getRebalanceThreadPoolSize();
cfg.partitionedDelay = ccfg.getRebalancePartitionedDelay();
cfg.partitionedDelay = ccfg.getRebalanceDelay();
cfg.throttle = ccfg.getRebalanceThrottle();
cfg.timeout = ccfg.getRebalanceTimeout();

Expand Down
Expand Up @@ -59,7 +59,7 @@ public CacheConfiguration cacheConfiguration(String gridName) {
ccfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
ccfg.setCacheMode(CacheMode.PARTITIONED);

ccfg.setRebalancePartitionedDelay(-1);
ccfg.setRebalanceDelay(-1);

return ccfg;
}
Expand Down
Expand Up @@ -74,7 +74,7 @@ public class GridCacheDhtPreloadDelayedSelfTest extends GridCommonAbstractTest {
cc.setCacheMode(PARTITIONED);
cc.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
cc.setRebalanceMode(preloadMode);
cc.setRebalancePartitionedDelay(delay);
cc.setRebalanceDelay(delay);
cc.setAffinity(new CacheRendezvousAffinityFunction(false, 128));
cc.setBackups(1);
cc.setAtomicityMode(TRANSACTIONAL);
Expand Down
Expand Up @@ -86,7 +86,7 @@ protected CacheConfiguration cacheConfiguration(String gridName) throws Exceptio
cfg.setEvictionPolicy(null);
cfg.setBackups(1);
cfg.setDistributionMode(PARTITIONED_ONLY);
cfg.setRebalancePartitionedDelay(-1);
cfg.setRebalanceDelay(-1);
cfg.setBackups(1);
cfg.setStatisticsEnabled(true);

Expand Down

0 comments on commit 1900787

Please sign in to comment.