Repository: ignite
Updated Branches:
refs/heads/master ad868dab1 -> 42048c2bf
IGNITE-2925: ATOMIC cache: added dedicated methods for single key-value pair add.
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/42048c2b
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/42048c2b
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/42048c2b
Branch: refs/heads/master
Commit: 42048c2bf8e66d4eb584fef440b15038ea214985
Parents: ad868da
Author: vozerov-gridgain <vozerov@gridgain.com>
Authored: Fri Apr 1 11:12:04 2016 +0300
Committer: vozerov-gridgain <vozerov@gridgain.com>
Committed: Fri Apr 1 11:12:04 2016 +0300
----------------------------------------------------------------------
.../processors/cache/GridCacheAdapter.java | 117 ----------
.../processors/cache/GridCacheProxyImpl.java | 48 ----
.../processors/cache/IgniteInternalCache.java | 79 -------
.../dht/atomic/GridDhtAtomicCache.java | 224 +++++++++++++------
.../distributed/near/GridNearAtomicCache.java | 22 --
.../local/atomic/GridLocalAtomicCache.java | 57 -----
6 files changed, 158 insertions(+), 389 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/ignite/blob/42048c2b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 44c8b57..d6571cc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -3045,32 +3045,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K,
V
}
/** {@inheritDoc} */
- @Override public GridCacheReturn removex(final K key, final V val) throws IgniteCheckedException
{
- A.notNull(key, "key", val, "val");
-
- if (keyCheck)
- validateCacheKey(key);
-
- return syncOp(new SyncOp<GridCacheReturn>(true) {
- @Override public GridCacheReturn op(IgniteTxLocalAdapter tx) throws IgniteCheckedException
{
- // Register before hiding in the filter.
- if (ctx.deploymentEnabled())
- ctx.deploy().registerClass(val);
-
- return tx.removeAllAsync(ctx,
- Collections.singletonList(key),
- /*retval*/true,
- ctx.equalsValArray(val),
- /*singleRmv*/false).get();
- }
-
- @Override public String toString() {
- return "remove [key=" + key + ", val=" + val + ']';
- }
- });
- }
-
- /** {@inheritDoc} */
@Override public void removeAllConflict(final Map<KeyCacheObject, GridCacheVersion>
drMap)
throws IgniteCheckedException {
if (F.isEmpty(drMap))
@@ -3109,97 +3083,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K,
V
}
/** {@inheritDoc} */
- @Override public GridCacheReturn replacex(final K key, final V oldVal, final V newVal)
- throws IgniteCheckedException
- {
- A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal");
-
- if (keyCheck)
- validateCacheKey(key);
-
- return syncOp(new SyncOp<GridCacheReturn>(true) {
- @Override public GridCacheReturn op(IgniteTxLocalAdapter tx) throws IgniteCheckedException
{
- // Register before hiding in the filter.
- if (ctx.deploymentEnabled())
- ctx.deploy().registerClass(oldVal);
-
- return tx.putAsync(ctx,
- key,
- newVal,
- true,
- ctx.equalsValArray(oldVal)).get();
- }
-
- @Override public String toString() {
- return "replace [key=" + key + ", oldVal=" + oldVal + ", newVal=" + newVal
+ ']';
- }
- });
- }
-
- /** {@inheritDoc} */
- @Override public IgniteInternalFuture<GridCacheReturn> removexAsync(final K key,
final V val) {
- A.notNull(key, "key", val, "val");
-
- if (keyCheck)
- validateCacheKey(key);
-
- return asyncOp(new AsyncOp<GridCacheReturn>() {
- @Override public IgniteInternalFuture<GridCacheReturn> op(IgniteTxLocalAdapter
tx) {
- // Register before hiding in the filter.
- try {
- if (ctx.deploymentEnabled())
- ctx.deploy().registerClass(val);
- }
- catch (IgniteCheckedException e) {
- return new GridFinishedFuture<>(e);
- }
-
- return (IgniteInternalFuture)tx.removeAllAsync(ctx,
- Collections.singletonList(key),
- /*retval*/true,
- ctx.equalsValArray(val),
- /*singleRmv*/false);
- }
-
- @Override public String toString() {
- return "removeAsync [key=" + key + ", val=" + val + ']';
- }
- });
- }
-
- /** {@inheritDoc} */
- @Override public IgniteInternalFuture<GridCacheReturn> replacexAsync(final K key,
final V oldVal, final V newVal)
- {
- A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal");
-
- if (keyCheck)
- validateCacheKey(key);
-
- return asyncOp(new AsyncOp<GridCacheReturn>() {
- @Override public IgniteInternalFuture<GridCacheReturn> op(IgniteTxLocalAdapter
tx) {
- // Register before hiding in the filter.
- try {
- if (ctx.deploymentEnabled())
- ctx.deploy().registerClass(oldVal);
- }
- catch (IgniteCheckedException e) {
- return new GridFinishedFuture<>(e);
- }
-
- return (IgniteInternalFuture)tx.putAsync(ctx,
- key,
- newVal,
- true,
- ctx.equalsValArray(oldVal));
- }
-
- @Override public String toString() {
- return "replaceAsync [key=" + key + ", oldVal=" + oldVal + ", newVal=" +
newVal + ']';
- }
- });
- }
-
- /** {@inheritDoc} */
@Override public boolean remove(final K key, final V val) throws IgniteCheckedException
{
boolean statsEnabled = ctx.config().isStatisticsEnabled();
http://git-wip-us.apache.org/repos/asf/ignite/blob/42048c2b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
index cbe16c6..66304e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
@@ -1226,54 +1226,6 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K,
V>, Exte
}
/** {@inheritDoc} */
- @Override public IgniteInternalFuture<GridCacheReturn> replacexAsync(K key, V oldVal,
V newVal) {
- CacheOperationContext prev = gate.enter(opCtx);
-
- try {
- return delegate.replacexAsync(key, oldVal, newVal);
- }
- finally {
- gate.leave(prev);
- }
- }
-
- /** {@inheritDoc} */
- @Override public GridCacheReturn replacex(K key, V oldVal, V newVal) throws IgniteCheckedException
{
- CacheOperationContext prev = gate.enter(opCtx);
-
- try {
- return delegate.replacex(key, oldVal, newVal);
- }
- finally {
- gate.leave(prev);
- }
- }
-
- /** {@inheritDoc} */
- @Override public GridCacheReturn removex(K key, V val) throws IgniteCheckedException
{
- CacheOperationContext prev = gate.enter(opCtx);
-
- try {
- return delegate.removex(key, val);
- }
- finally {
- gate.leave(prev);
- }
- }
-
- /** {@inheritDoc} */
- @Override public IgniteInternalFuture<GridCacheReturn> removexAsync(K key, V val)
{
- CacheOperationContext prev = gate.enter(opCtx);
-
- try {
- return delegate.removexAsync(key, val);
- }
- finally {
- gate.leave(prev);
- }
- }
-
- /** {@inheritDoc} */
@Override public boolean remove(K key, V val) throws IgniteCheckedException {
CacheOperationContext prev = gate.enter(opCtx);
http://git-wip-us.apache.org/repos/asf/ignite/blob/42048c2b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
index d2e0101..a9f4538 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
@@ -1683,85 +1683,6 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K,
V>> {
public IgniteInternalFuture<?> removeAllConflictAsync(Map<KeyCacheObject, GridCacheVersion>
drMap) throws IgniteCheckedException;
/**
- * Asynchronously stores given key-value pair in cache only if only if the previous value
is equal to the
- * {@code 'oldVal'} passed in.
- * <p>
- * This method will return {@code true} if value is stored in cache and {@code false}
otherwise.
- * <p>
- * If write-through is enabled, the stored value will be persisted to {@link CacheStore}
- * via {@link CacheStore#write(javax.cache.Cache.Entry)} method.
- * <h2 class="header">Transactions</h2>
- * This method is transactional and will enlist the entry into ongoing transaction
- * if there is one.
- *
- * @param key Key to store in cache.
- * @param oldVal Old value to match.
- * @param newVal Value to be associated with the given key.
- * @return Future for the replace operation. The future will return object containing
actual old value and success
- * flag.
- * @throws NullPointerException If either key or value are {@code null}.
- */
- public IgniteInternalFuture<GridCacheReturn> replacexAsync(K key, V oldVal, V newVal);
-
- /**
- * Stores given key-value pair in cache only if only if the previous value is equal to
the
- * {@code 'oldVal'} passed in.
- * <p>
- * This method will return {@code true} if value is stored in cache and {@code false}
otherwise.
- * <p>
- * If write-through is enabled, the stored value will be persisted to {@link CacheStore}
- * via {@link CacheStore#write(javax.cache.Cache.Entry)} method.
- * <h2 class="header">Transactions</h2>
- * This method is transactional and will enlist the entry into ongoing transaction
- * if there is one.
- *
- * @param key Key to store in cache.
- * @param oldVal Old value to match.
- * @param newVal Value to be associated with the given key.
- * @return Object containing actual old value and success flag.
- * @throws NullPointerException If either key or value are {@code null}.
- * @throws IgniteCheckedException If replace operation failed.
- */
- public GridCacheReturn replacex(K key, V oldVal, V newVal) throws IgniteCheckedException;
-
- /**
- * Removes given key mapping from cache if one exists and value is equal to the passed
in value.
- * <p>
- * If write-through is enabled, the value will be removed from {@link CacheStore}
- * via {@link CacheStore#delete(Object)} method.
- * <h2 class="header">Transactions</h2>
- * This method is transactional and will enlist the entry into ongoing transaction
- * if there is one.
- *
- * @param key Key whose mapping is to be removed from cache.
- * @param val Value to match against currently cached value.
- * @return Object containing actual old value and success flag.
- * @throws NullPointerException if the key or value is {@code null}.
- * @throws IgniteCheckedException If remove failed.
- */
- public GridCacheReturn removex(K key, V val) throws IgniteCheckedException;
-
- /**
- * Asynchronously removes given key mapping from cache if one exists and value is equal
to the passed in value.
- * <p>
- * This method will return {@code true} if remove did occur, which means that all optionally
- * provided filters have passed and there was something to remove, {@code false} otherwise.
- * <p>
- * If write-through is enabled, the value will be removed from {@link CacheStore}
- * via {@link CacheStore#delete(Object)} method.
- * <h2 class="header">Transactions</h2>
- * This method is transactional and will enlist the entry into ongoing transaction
- * if there is one.
- *
- * @param key Key whose mapping is to be removed from cache.
- * @param val Value to match against currently cached value.
- * @return Future for the remove operation. The future will return object containing
actual old value and success
- * flag.
- * @throws NullPointerException if the key or value is {@code null}.
- */
- public IgniteInternalFuture<GridCacheReturn> removexAsync(K key, V val);
-
- /**
* Gets value from cache. Will go to primary node even if this is a backup.
*
* @param key Key to get value for.
http://git-wip-us.apache.org/repos/asf/ignite/blob/42048c2b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 1f5c817..f85862d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -82,7 +82,6 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext;
import org.apache.ignite.internal.processors.cache.version.GridCacheVersionEx;
import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
-import org.apache.ignite.internal.util.F0;
import org.apache.ignite.internal.util.GridUnsafe;
import org.apache.ignite.internal.util.future.GridEmbeddedFuture;
import org.apache.ignite.internal.util.future.GridFinishedFuture;
@@ -443,51 +442,45 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K,
V> {
/** {@inheritDoc} */
@SuppressWarnings("unchecked")
@Override public IgniteInternalFuture<V> getAndPutAsync0(K key, V val, @Nullable
CacheEntryPredicate... filter) {
- A.notNull(key, "key");
+ A.notNull(key, "key", val, "val");
- return updateAllAsync0(F0.asMap(key, val),
- null,
- null,
+ return updateAsync0(
+ key,
+ val,
null,
null,
true,
- false,
filter,
- true,
- UPDATE);
+ true);
}
/** {@inheritDoc} */
@SuppressWarnings("unchecked")
@Override public IgniteInternalFuture<Boolean> putAsync0(K key, V val, @Nullable
CacheEntryPredicate... filter) {
- A.notNull(key, "key");
+ A.notNull(key, "key", val, "val");
- return updateAllAsync0(F0.asMap(key, val),
- null,
- null,
+ return updateAsync0(
+ key,
+ val,
null,
null,
false,
- false,
filter,
- true,
- UPDATE);
+ true);
}
/** {@inheritDoc} */
@Override public V tryPutIfAbsent(K key, V val) throws IgniteCheckedException {
A.notNull(key, "key", val, "val");
- return (V)updateAllAsync0(F0.asMap(key, val),
- null,
- null,
+ return (V)updateAsync0(
+ key,
+ val,
null,
null,
true,
- false,
ctx.noValArray(),
- false,
- UPDATE).get();
+ false).get();
}
/** {@inheritDoc} */
@@ -551,39 +544,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K,
V> {
}
/** {@inheritDoc} */
- @Override public GridCacheReturn removex(K key, V val) throws IgniteCheckedException
{
- return removexAsync(key, val).get();
- }
-
- /** {@inheritDoc} */
- @Override public GridCacheReturn replacex(K key, V oldVal, V newVal) throws IgniteCheckedException
{
- return replacexAsync(key, oldVal, newVal).get();
- }
-
- /** {@inheritDoc} */
- @SuppressWarnings("unchecked")
- @Override public IgniteInternalFuture<GridCacheReturn> removexAsync(K key, V val)
{
- A.notNull(key, "key", val, "val");
-
- return removeAllAsync0(F.asList(key), null, true, true, ctx.equalsValArray(val));
- }
-
- /** {@inheritDoc} */
- @SuppressWarnings("unchecked")
- @Override public IgniteInternalFuture<GridCacheReturn> replacexAsync(K key, V oldVal,
V newVal) {
- return updateAllAsync0(F.asMap(key, newVal),
- null,
- null,
- null,
- null,
- true,
- true,
- ctx.equalsValArray(oldVal),
- true,
- UPDATE);
- }
-
- /** {@inheritDoc} */
@Override public void putAll(Map<? extends K, ? extends V> m) throws IgniteCheckedException
{
putAllAsync(m).get();
}
@@ -634,7 +594,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K,
V> {
@Override public IgniteInternalFuture<V> getAndRemoveAsync(K key) {
A.notNull(key, "key");
- return removeAllAsync0(Collections.singletonList(key), null, true, false, CU.empty0());
+ return removeAsync0(key, true, CU.empty0());
}
/** {@inheritDoc} */
@@ -659,7 +619,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K,
V> {
@Override public IgniteInternalFuture<Boolean> removeAsync(K key, @Nullable CacheEntryPredicate...
filter) {
A.notNull(key, "key");
- return removeAllAsync0(Collections.singletonList(key), null, false, false, filter);
+ return removeAsync0(key, false, filter);
}
/** {@inheritDoc} */
@@ -786,23 +746,18 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K,
V> {
if (keyCheck)
validateCacheKey(key);
- Map<? extends K, EntryProcessor> invokeMap =
- Collections.singletonMap(key, (EntryProcessor)entryProcessor);
-
CacheOperationContext opCtx = ctx.operationContextPerCall();
final boolean keepBinary = opCtx != null && opCtx.isKeepBinary();
- IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> fut = updateAllAsync0(null,
- invokeMap,
- args,
- null,
+ IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> fut = updateAsync0(
+ key,
null,
- false,
+ entryProcessor,
+ args,
false,
null,
- true,
- TRANSFORM);
+ true);
return fut.chain(new CX1<IgniteInternalFuture<Map<K, EntryProcessorResult<T>>>,
EntryProcessorResult<T>>() {
@Override public EntryProcessorResult<T> applyx(IgniteInternalFuture<Map<K,
EntryProcessorResult<T>>> fut)
@@ -1011,6 +966,143 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K,
V> {
}
/**
+ * Entry point for update/invoke with a single key.
+ *
+ * @param key Key.
+ * @param val Value.
+ * @param proc Entry processor.
+ * @param invokeArgs Invoke arguments.
+ * @param retval Return value flag.
+ * @param filter Filter.
+ * @param waitTopFut Whether to wait for topology future.
+ * @return Future.
+ */
+ private IgniteInternalFuture updateAsync0(
+ K key,
+ @Nullable V val,
+ @Nullable EntryProcessor proc,
+ @Nullable Object[] invokeArgs,
+ final boolean retval,
+ @Nullable final CacheEntryPredicate[] filter,
+ final boolean waitTopFut
+ ) {
+ assert val == null || proc == null;
+
+ assert ctx.updatesAllowed();
+
+ validateCacheKey(key);
+
+ ctx.checkSecurity(SecurityPermission.CACHE_PUT);
+
+ final GridNearAtomicUpdateFuture updateFut =
+ createSingleUpdateFuture(key, val, proc, invokeArgs, retval, filter, waitTopFut);
+
+ return asyncOp(new CO<IgniteInternalFuture<Object>>() {
+ @Override public IgniteInternalFuture<Object> apply() {
+ updateFut.map();
+
+ return updateFut;
+ }
+ });
+ }
+
+ /**
+ * Entry point for remove with single key.
+ *
+ * @param key Key.
+ * @param retval Whether to return
+ * @param filter Filter.
+ * @return Future.
+ */
+ private IgniteInternalFuture removeAsync0(K key, final boolean retval,
+ @Nullable final CacheEntryPredicate[] filter) {
+ final boolean statsEnabled = ctx.config().isStatisticsEnabled();
+
+ final long start = statsEnabled ? System.nanoTime() : 0L;
+
+ assert ctx.updatesAllowed();
+
+ validateCacheKey(key);
+
+ ctx.checkSecurity(SecurityPermission.CACHE_REMOVE);
+
+ final GridNearAtomicUpdateFuture updateFut =
+ createSingleUpdateFuture(key, null, null, null, retval, filter, true);
+
+ if (statsEnabled)
+ updateFut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start));
+
+ return asyncOp(new CO<IgniteInternalFuture<Object>>() {
+ @Override public IgniteInternalFuture<Object> apply() {
+ updateFut.map();
+
+ return updateFut;
+ }
+ });
+ }
+
+ /**
+ * Craete future for single key-val pair update.
+ *
+ * @param key Key.
+ * @param val Value.
+ * @param proc Processor.
+ * @param invokeArgs Invoke arguments.
+ * @param retval Return value flag.
+ * @param filter Filter.
+ * @param waitTopFut Whether to wait for topology future.
+ * @return Future.
+ */
+ private GridNearAtomicUpdateFuture createSingleUpdateFuture(
+ K key,
+ @Nullable V val,
+ @Nullable EntryProcessor proc,
+ @Nullable Object[] invokeArgs,
+ boolean retval,
+ @Nullable final CacheEntryPredicate[] filter,
+ boolean waitTopFut
+ ) {
+ GridCacheOperation op;
+ Collection vals;
+
+ if (val != null) {
+ op = UPDATE;
+ vals = Collections.singletonList(val);
+ }
+ else if (proc != null) {
+ op = TRANSFORM;
+ vals = Collections.singletonList(proc);
+ }
+ else {
+ op = DELETE;
+ vals = null;
+ }
+
+ CacheOperationContext opCtx = ctx.operationContextPerCall();
+
+ return new GridNearAtomicUpdateFuture(
+ ctx,
+ this,
+ ctx.config().getWriteSynchronizationMode(),
+ op,
+ Collections.singletonList(key),
+ vals,
+ invokeArgs,
+ null,
+ null,
+ retval,
+ false,
+ opCtx != null ? opCtx.expiry() : null,
+ filter,
+ ctx.subjectIdPerCall(null, opCtx),
+ ctx.kernalContext().job().currentTaskNameHash(),
+ opCtx != null && opCtx.skipStore(),
+ opCtx != null && opCtx.isKeepBinary(),
+ opCtx != null && opCtx.noRetries() ? 1 : MAX_RETRIES,
+ waitTopFut);
+ }
+
+ /**
* Entry point for all public API remove methods.
*
* @param keys Keys to remove.
http://git-wip-us.apache.org/repos/asf/ignite/blob/42048c2b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
index 63c073d..6cd7745 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
@@ -506,28 +506,6 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K,
V> {
}
/** {@inheritDoc} */
- @Override public GridCacheReturn removex(K key, V val) throws IgniteCheckedException
{
- return dht.removex(key, val);
- }
-
- /** {@inheritDoc} */
- @Override public GridCacheReturn replacex(K key, V oldVal, V newVal) throws IgniteCheckedException
{
- return dht.replacex(key, oldVal, newVal);
- }
-
- /** {@inheritDoc} */
- @SuppressWarnings("unchecked")
- @Override public IgniteInternalFuture<GridCacheReturn> removexAsync(K key, V val)
{
- return dht.removexAsync(key, val);
- }
-
- /** {@inheritDoc} */
- @SuppressWarnings("unchecked")
- @Override public IgniteInternalFuture<GridCacheReturn> replacexAsync(K key, V oldVal,
V newVal) {
- return dht.replacexAsync(key, oldVal, newVal);
- }
-
- /** {@inheritDoc} */
@Override public void putAll(Map<? extends K, ? extends V> m)
throws IgniteCheckedException {
dht.putAll(m);
http://git-wip-us.apache.org/repos/asf/ignite/blob/42048c2b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
index 07b70cc..2df7340 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -262,63 +262,6 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K,
V> {
}
/** {@inheritDoc} */
- @SuppressWarnings("unchecked")
- @Override public GridCacheReturn replacex(K key, V oldVal, V newVal) throws IgniteCheckedException
{
- A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal");
-
- return (GridCacheReturn)updateAllInternal(UPDATE,
- Collections.singleton(key),
- Collections.singleton(newVal),
- null,
- expiryPerCall(),
- true,
- true,
- ctx.equalsValArray(oldVal),
- ctx.writeThrough(),
- ctx.readThrough(),
- ctx.operationContextPerCall().isKeepBinary());
- }
-
- /** {@inheritDoc} */
- @SuppressWarnings("unchecked")
- @Override public GridCacheReturn removex(K key, V val) throws IgniteCheckedException
{
- A.notNull(key, "key", val, "val");
-
- return (GridCacheReturn)updateAllInternal(DELETE,
- Collections.singleton(key),
- null,
- null,
- expiryPerCall(),
- true,
- true,
- ctx.equalsValArray(val),
- ctx.writeThrough(),
- ctx.readThrough(),
- ctx.operationContextPerCall().isKeepBinary());
- }
-
- /** {@inheritDoc} */
- @SuppressWarnings("unchecked")
- @Override public IgniteInternalFuture<GridCacheReturn> removexAsync(K key, V val)
{
- A.notNull(key, "key", val, "val");
-
- return removeAllAsync0(F.asList(key), true, true, ctx.equalsValArray(val));
- }
-
- /** {@inheritDoc} */
- @SuppressWarnings("unchecked")
- @Override public IgniteInternalFuture<GridCacheReturn> replacexAsync(K key, V oldVal,
V newVal) {
- A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal");
-
- return updateAllAsync0(F.asMap(key, newVal),
- null,
- null,
- true,
- true,
- ctx.equalsValArray(oldVal));
- }
-
- /** {@inheritDoc} */
@Override public void putAll(Map<? extends K, ? extends V> m) throws IgniteCheckedException
{
boolean statsEnabled = ctx.config().isStatisticsEnabled();
|