http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchImpl.java
index 00cc7cd..7ee45bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchImpl.java
@@ -61,7 +61,7 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
private GridCacheInternalKey key;
/** Latch projection. */
- private GridCacheProjection<GridCacheInternalKey, GridCacheCountDownLatchValue> latchView;
+ private CacheProjection<GridCacheInternalKey, GridCacheCountDownLatchValue> latchView;
/** Cache context. */
private GridCacheContext ctx;
@@ -103,7 +103,7 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
* @param ctx Cache context.
*/
public GridCacheCountDownLatchImpl(String name, int cnt, int initCnt, boolean autoDel, GridCacheInternalKey key,
- GridCacheProjection<GridCacheInternalKey, GridCacheCountDownLatchValue> latchView, GridCacheContext ctx) {
+ CacheProjection<GridCacheInternalKey, GridCacheCountDownLatchValue> latchView, GridCacheContext ctx) {
assert name != null;
assert cnt >= 0;
assert initCnt >= 0;
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresImpl.java
index be9df8b..9008aca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresImpl.java
@@ -25,7 +25,7 @@ import org.jetbrains.annotations.*;
/**
* Data structures implementation object.
*/
-public class GridCacheDataStructuresImpl<K, V> implements GridCacheDataStructures {
+public class GridCacheDataStructuresImpl<K, V> implements CacheDataStructures {
/** Data structures manager. */
private GridCacheDataStructuresManager<K, V> dsMgr;
@@ -37,7 +37,7 @@ public class GridCacheDataStructuresImpl<K, V> implements GridCacheDataStructure
}
/** {@inheritDoc} */
- @Override public GridCacheAtomicSequence atomicSequence(String name, long initVal, boolean create)
+ @Override public CacheAtomicSequence atomicSequence(String name, long initVal, boolean create)
throws IgniteCheckedException {
return dsMgr.sequence(name, initVal, create);
}
@@ -48,7 +48,7 @@ public class GridCacheDataStructuresImpl<K, V> implements GridCacheDataStructure
}
/** {@inheritDoc} */
- @Override public GridCacheAtomicLong atomicLong(String name, long initVal, boolean create) throws IgniteCheckedException {
+ @Override public CacheAtomicLong atomicLong(String name, long initVal, boolean create) throws IgniteCheckedException {
return dsMgr.atomicLong(name, initVal, create);
}
@@ -58,7 +58,7 @@ public class GridCacheDataStructuresImpl<K, V> implements GridCacheDataStructure
}
/** {@inheritDoc} */
- @Override public <T> GridCacheAtomicReference<T> atomicReference(String name, T initVal, boolean create)
+ @Override public <T> CacheAtomicReference<T> atomicReference(String name, T initVal, boolean create)
throws IgniteCheckedException {
return dsMgr.atomicReference(name, initVal, create);
}
@@ -69,7 +69,7 @@ public class GridCacheDataStructuresImpl<K, V> implements GridCacheDataStructure
}
/** {@inheritDoc} */
- @Override public <T, S> GridCacheAtomicStamped<T, S> atomicStamped(String name, T initVal, S initStamp,
+ @Override public <T, S> CacheAtomicStamped<T, S> atomicStamped(String name, T initVal, S initStamp,
boolean create) throws IgniteCheckedException {
return dsMgr.atomicStamped(name, initVal, initStamp, create);
}
@@ -80,7 +80,7 @@ public class GridCacheDataStructuresImpl<K, V> implements GridCacheDataStructure
}
/** {@inheritDoc} */
- @Override public <T> GridCacheQueue<T> queue(String name, int cap, boolean collocated, boolean create)
+ @Override public <T> CacheQueue<T> queue(String name, int cap, boolean collocated, boolean create)
throws IgniteCheckedException {
return dsMgr.queue(name, cap <= 0 ? Integer.MAX_VALUE : cap, collocated, create);
}
@@ -96,7 +96,7 @@ public class GridCacheDataStructuresImpl<K, V> implements GridCacheDataStructure
}
/** {@inheritDoc} */
- @Nullable @Override public <T> GridCacheSet<T> set(String name, boolean collocated, boolean create)
+ @Nullable @Override public <T> CacheSet<T> set(String name, boolean collocated, boolean create)
throws IgniteCheckedException {
return dsMgr.set(name, collocated, create);
}
@@ -107,7 +107,7 @@ public class GridCacheDataStructuresImpl<K, V> implements GridCacheDataStructure
}
/** {@inheritDoc} */
- @Nullable @Override public GridCacheCountDownLatch countDownLatch(String name, int cnt, boolean autoDel,
+ @Nullable @Override public CacheCountDownLatch countDownLatch(String name, int cnt, boolean autoDel,
boolean create) throws IgniteCheckedException {
return dsMgr.countDownLatch(name, cnt, autoDel, create);
}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java
index 5eb1ae8..c556a3e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java
@@ -41,7 +41,7 @@ import java.util.concurrent.*;
import java.util.concurrent.atomic.*;
import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.*;
-import static org.apache.ignite.cache.GridCacheFlag.*;
+import static org.apache.ignite.cache.CacheFlag.*;
import static org.apache.ignite.cache.CacheMode.*;
import static org.apache.ignite.transactions.IgniteTxConcurrency.*;
import static org.apache.ignite.transactions.IgniteTxIsolation.*;
@@ -62,7 +62,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
private static final long RETRY_DELAY = 1;
/** Cache contains only {@code GridCacheInternal,GridCacheInternal}. */
- private GridCacheProjection<GridCacheInternal, GridCacheInternal> dsView;
+ private CacheProjection<GridCacheInternal, GridCacheInternal> dsView;
/** Internal storage of all dataStructures items (sequence, queue , atomic long etc.). */
private final ConcurrentMap<GridCacheInternal, GridCacheRemovable> dsMap;
@@ -77,22 +77,22 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
private final AtomicBoolean queueQryGuard = new AtomicBoolean();
/** Cache contains only {@code GridCacheAtomicValue}. */
- private GridCacheProjection<GridCacheInternalKey, GridCacheAtomicLongValue> atomicLongView;
+ private CacheProjection<GridCacheInternalKey, GridCacheAtomicLongValue> atomicLongView;
/** Cache contains only {@code GridCacheCountDownLatchValue}. */
- private GridCacheProjection<GridCacheInternalKey, GridCacheCountDownLatchValue> cntDownLatchView;
+ private CacheProjection<GridCacheInternalKey, GridCacheCountDownLatchValue> cntDownLatchView;
/** Cache contains only {@code GridCacheAtomicReferenceValue}. */
- private GridCacheProjection<GridCacheInternalKey, GridCacheAtomicReferenceValue> atomicRefView;
+ private CacheProjection<GridCacheInternalKey, GridCacheAtomicReferenceValue> atomicRefView;
/** Cache contains only {@code GridCacheAtomicStampedValue}. */
- private GridCacheProjection<GridCacheInternalKey, GridCacheAtomicStampedValue> atomicStampedView;
+ private CacheProjection<GridCacheInternalKey, GridCacheAtomicStampedValue> atomicStampedView;
/** Cache contains only entry {@code GridCacheSequenceValue}. */
- private GridCacheProjection<GridCacheInternalKey, GridCacheAtomicSequenceValue> seqView;
+ private CacheProjection<GridCacheInternalKey, GridCacheAtomicSequenceValue> seqView;
/** Cache contains only entry {@code GridCacheQueueHeader}. */
- private GridCacheProjection<GridCacheQueueHeaderKey, GridCacheQueueHeader> queueHdrView;
+ private CacheProjection<GridCacheQueueHeaderKey, GridCacheQueueHeader> queueHdrView;
/** Busy lock. */
private final GridSpinBusyLock busyLock = new GridSpinBusyLock();
@@ -181,7 +181,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
* @return Sequence.
* @throws IgniteCheckedException If loading failed.
*/
- public final GridCacheAtomicSequence sequence(final String name, final long initVal,
+ public final CacheAtomicSequence sequence(final String name, final long initVal,
final boolean create) throws IgniteCheckedException {
waitInitialization();
@@ -191,14 +191,14 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
try {
// Check type of structure received by key from local cache.
- GridCacheAtomicSequence val = cast(dsMap.get(key), GridCacheAtomicSequence.class);
+ CacheAtomicSequence val = cast(dsMap.get(key), CacheAtomicSequence.class);
if (val != null)
return val;
- return CU.outTx(new Callable<GridCacheAtomicSequence>() {
+ return CU.outTx(new Callable<CacheAtomicSequence>() {
@Override
- public GridCacheAtomicSequence call() throws Exception {
+ public CacheAtomicSequence call() throws Exception {
try (IgniteTx tx = CU.txStartInternal(cctx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
GridCacheAtomicSequenceValue seqVal = cast(dsView.get(key),
GridCacheAtomicSequenceValue.class);
@@ -299,7 +299,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
* @return Atomic long.
* @throws IgniteCheckedException If loading failed.
*/
- public final GridCacheAtomicLong atomicLong(final String name, final long initVal,
+ public final CacheAtomicLong atomicLong(final String name, final long initVal,
final boolean create) throws IgniteCheckedException {
waitInitialization();
@@ -309,14 +309,14 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
try {
// Check type of structure received by key from local cache.
- GridCacheAtomicLong atomicLong = cast(dsMap.get(key), GridCacheAtomicLong.class);
+ CacheAtomicLong atomicLong = cast(dsMap.get(key), CacheAtomicLong.class);
if (atomicLong != null)
return atomicLong;
- return CU.outTx(new Callable<GridCacheAtomicLong>() {
+ return CU.outTx(new Callable<CacheAtomicLong>() {
@Override
- public GridCacheAtomicLong call() throws Exception {
+ public CacheAtomicLong call() throws Exception {
try (IgniteTx tx = CU.txStartInternal(cctx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
GridCacheAtomicLongValue val = cast(dsView.get(key),
GridCacheAtomicLongValue.class);
@@ -395,7 +395,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
* @throws IgniteCheckedException If loading failed.
*/
@SuppressWarnings("unchecked")
- public final <T> GridCacheAtomicReference<T> atomicReference(final String name, final T initVal,
+ public final <T> CacheAtomicReference<T> atomicReference(final String name, final T initVal,
final boolean create) throws IgniteCheckedException {
waitInitialization();
@@ -405,14 +405,14 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
try {
// Check type of structure received by key from local cache.
- GridCacheAtomicReference atomicRef = cast(dsMap.get(key), GridCacheAtomicReference.class);
+ CacheAtomicReference atomicRef = cast(dsMap.get(key), CacheAtomicReference.class);
if (atomicRef != null)
return atomicRef;
- return CU.outTx(new Callable<GridCacheAtomicReference<T>>() {
+ return CU.outTx(new Callable<CacheAtomicReference<T>>() {
@Override
- public GridCacheAtomicReference<T> call() throws Exception {
+ public CacheAtomicReference<T> call() throws Exception {
try (IgniteTx tx = CU.txStartInternal(cctx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
GridCacheAtomicReferenceValue val = cast(dsView.get(key),
GridCacheAtomicReferenceValue.class);
@@ -494,7 +494,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
* @throws IgniteCheckedException If loading failed.
*/
@SuppressWarnings("unchecked")
- public final <T, S> GridCacheAtomicStamped<T, S> atomicStamped(final String name, final T initVal,
+ public final <T, S> CacheAtomicStamped<T, S> atomicStamped(final String name, final T initVal,
final S initStamp, final boolean create) throws IgniteCheckedException {
waitInitialization();
@@ -504,14 +504,14 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
try {
// Check type of structure received by key from local cache.
- GridCacheAtomicStamped atomicStamped = cast(dsMap.get(key), GridCacheAtomicStamped.class);
+ CacheAtomicStamped atomicStamped = cast(dsMap.get(key), CacheAtomicStamped.class);
if (atomicStamped != null)
return atomicStamped;
- return CU.outTx(new Callable<GridCacheAtomicStamped<T, S>>() {
+ return CU.outTx(new Callable<CacheAtomicStamped<T, S>>() {
@Override
- public GridCacheAtomicStamped<T, S> call() throws Exception {
+ public CacheAtomicStamped<T, S> call() throws Exception {
try (IgniteTx tx = CU.txStartInternal(cctx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
GridCacheAtomicStampedValue val = cast(dsView.get(key),
GridCacheAtomicStampedValue.class);
@@ -590,7 +590,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
* @return Instance of queue.
* @throws IgniteCheckedException If failed.
*/
- public final <T> GridCacheQueue<T> queue(final String name, final int cap, boolean colloc,
+ public final <T> CacheQueue<T> queue(final String name, final int cap, boolean colloc,
final boolean create) throws IgniteCheckedException {
waitInitialization();
@@ -602,8 +602,8 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
if (cctx.atomic())
return queue0(name, cap, collocMode, create);
- return CU.outTx(new Callable<GridCacheQueue<T>>() {
- @Override public GridCacheQueue<T> call() throws Exception {
+ return CU.outTx(new Callable<CacheQueue<T>>() {
+ @Override public CacheQueue<T> call() throws Exception {
return queue0(name, cap, collocMode, create);
}
}, cctx);
@@ -620,7 +620,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
* @throws IgniteCheckedException If failed.
*/
@SuppressWarnings({"unchecked", "NonPrivateFieldAccessedInSynchronizedContext"})
- private <T> GridCacheQueue<T> queue0(final String name, final int cap, boolean colloc, final boolean create)
+ private <T> CacheQueue<T> queue0(final String name, final int cap, boolean colloc, final boolean create)
throws IgniteCheckedException {
GridCacheQueueHeaderKey key = new GridCacheQueueHeaderKey(name);
@@ -773,7 +773,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
* {@code create} is false.
* @throws IgniteCheckedException If operation failed.
*/
- public GridCacheCountDownLatch countDownLatch(final String name, final int cnt, final boolean autoDel,
+ public CacheCountDownLatch countDownLatch(final String name, final int cnt, final boolean autoDel,
final boolean create) throws IgniteCheckedException {
A.ensure(cnt >= 0, "count can not be negative");
@@ -785,13 +785,13 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
try {
// Check type of structure received by key from local cache.
- GridCacheCountDownLatch latch = cast(dsMap.get(key), GridCacheCountDownLatch.class);
+ CacheCountDownLatch latch = cast(dsMap.get(key), CacheCountDownLatch.class);
if (latch != null)
return latch;
- return CU.outTx(new Callable<GridCacheCountDownLatch>() {
- @Override public GridCacheCountDownLatch call() throws Exception {
+ return CU.outTx(new Callable<CacheCountDownLatch>() {
+ @Override public CacheCountDownLatch call() throws Exception {
try (IgniteTx tx = CU.txStartInternal(cctx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
GridCacheCountDownLatchValue val = cast(dsView.get(key),
GridCacheCountDownLatchValue.class);
@@ -975,7 +975,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
}
else if (latch != null) {
U.error(log, "Failed to cast object " +
- "[expected=" + GridCacheCountDownLatch.class.getSimpleName() +
+ "[expected=" + CacheCountDownLatch.class.getSimpleName() +
", actual=" + latch.getClass() + ", value=" + latch + ']');
}
}
@@ -1017,18 +1017,18 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
/**
- * @return {@code True} if {@link GridCacheQueue} can be used with current cache configuration.
+ * @return {@code True} if {@link org.apache.ignite.cache.datastructures.CacheQueue} can be used with current cache configuration.
*/
private boolean supportsQueue() {
return !(cctx.atomic() && !cctx.isLocal() && cctx.config().getAtomicWriteOrderMode() == CLOCK);
}
/**
- * @throws IgniteCheckedException If {@link GridCacheQueue} can not be used with current cache configuration.
+ * @throws IgniteCheckedException If {@link org.apache.ignite.cache.datastructures.CacheQueue} can not be used with current cache configuration.
*/
private void checkSupportsQueue() throws IgniteCheckedException {
if (cctx.atomic() && !cctx.isLocal() && cctx.config().getAtomicWriteOrderMode() == CLOCK)
- throw new IgniteCheckedException("GridCacheQueue can not be used with ATOMIC cache with CLOCK write order mode" +
+ throw new IgniteCheckedException("CacheQueue can not be used with ATOMIC cache with CLOCK write order mode" +
" (change write order mode to PRIMARY in configuration)");
}
@@ -1054,7 +1054,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
* @return Set instance.
* @throws IgniteCheckedException If failed.
*/
- @Nullable public <T> GridCacheSet<T> set(final String name, boolean collocated, final boolean create)
+ @Nullable public <T> CacheSet<T> set(final String name, boolean collocated, final boolean create)
throws IgniteCheckedException {
waitInitialization();
@@ -1064,8 +1064,8 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
if (cctx.atomic())
return set0(name, collocMode, create);
- return CU.outTx(new Callable<GridCacheSet<T>>() {
- @Nullable @Override public GridCacheSet<T> call() throws Exception {
+ return CU.outTx(new Callable<CacheSet<T>>() {
+ @Nullable @Override public CacheSet<T> call() throws Exception {
return set0(name, collocMode, create);
}
}, cctx);
@@ -1163,7 +1163,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
* @throws IgniteCheckedException If failed.
*/
@SuppressWarnings("unchecked")
- @Nullable private <T> GridCacheSet<T> set0(String name, boolean collocated, boolean create) throws IgniteCheckedException {
+ @Nullable private <T> CacheSet<T> set0(String name, boolean collocated, boolean create) throws IgniteCheckedException {
GridCacheSetHeaderKey key = new GridCacheSetHeaderKey(name);
GridCacheSetHeader hdr;
@@ -1206,7 +1206,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
private boolean removeSet0(String name) throws IgniteCheckedException {
GridCacheSetHeaderKey key = new GridCacheSetHeaderKey(name);
- GridCache cache = cctx.cache();
+ Cache cache = cctx.cache();
GridCacheSetHeader hdr = retryRemove(cache, key);
@@ -1291,7 +1291,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
if (set == null)
return;
- GridCache cache = cctx.cache();
+ Cache cache = cctx.cache();
final int BATCH_SIZE = 100;
@@ -1358,7 +1358,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
* @return Removed value.
*/
@SuppressWarnings("unchecked")
- @Nullable private <T> T retryRemove(final GridCache cache, final Object key) throws IgniteCheckedException {
+ @Nullable private <T> T retryRemove(final Cache cache, final Object key) throws IgniteCheckedException {
return retry(new Callable<T>() {
@Nullable @Override public T call() throws Exception {
return (T)cache.remove(key);
@@ -1372,7 +1372,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
* @throws IgniteCheckedException If failed.
*/
@SuppressWarnings("unchecked")
- private void retryRemoveAll(final GridCache cache, final Collection<GridCacheSetItemKey> keys)
+ private void retryRemoveAll(final Cache cache, final Collection<GridCacheSetItemKey> keys)
throws IgniteCheckedException {
retry(new Callable<Void>() {
@Override public Void call() throws Exception {
@@ -1391,7 +1391,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
* @return Previous value.
*/
@SuppressWarnings("unchecked")
- @Nullable private <T> T retryPutIfAbsent(final GridCache cache, final Object key, final T val)
+ @Nullable private <T> T retryPutIfAbsent(final Cache cache, final Object key, final T val)
throws IgniteCheckedException {
return retry(new Callable<T>() {
@Nullable @Override public T call() throws Exception {
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresProxy.java
index 9004164..2727c52 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresProxy.java
@@ -27,12 +27,12 @@ import java.io.*;
/**
* Data structures proxy object.
*/
-public class GridCacheDataStructuresProxy<K, V> implements GridCacheDataStructures, Externalizable {
+public class GridCacheDataStructuresProxy<K, V> implements CacheDataStructures, Externalizable {
/** */
private static final long serialVersionUID = 0L;
/** Delegate object. */
- private GridCacheDataStructures delegate;
+ private CacheDataStructures delegate;
/** Cache gateway. */
private GridCacheGateway<K, V> gate;
@@ -51,7 +51,7 @@ public class GridCacheDataStructuresProxy<K, V> implements GridCacheDataStructur
* @param cctx Cache context.
* @param delegate Delegate object.
*/
- public GridCacheDataStructuresProxy(GridCacheContext<K, V> cctx, GridCacheDataStructures delegate) {
+ public GridCacheDataStructuresProxy(GridCacheContext<K, V> cctx, CacheDataStructures delegate) {
this.delegate = delegate;
this.cctx = cctx;
@@ -59,7 +59,7 @@ public class GridCacheDataStructuresProxy<K, V> implements GridCacheDataStructur
}
/** {@inheritDoc} */
- @Override public GridCacheAtomicSequence atomicSequence(String name, long initVal, boolean create)
+ @Override public CacheAtomicSequence atomicSequence(String name, long initVal, boolean create)
throws IgniteCheckedException {
GridCacheProjectionImpl<K, V> old = gate.enter(null);
@@ -84,7 +84,7 @@ public class GridCacheDataStructuresProxy<K, V> implements GridCacheDataStructur
}
/** {@inheritDoc} */
- @Override public GridCacheAtomicLong atomicLong(String name, long initVal, boolean create) throws IgniteCheckedException {
+ @Override public CacheAtomicLong atomicLong(String name, long initVal, boolean create) throws IgniteCheckedException {
GridCacheProjectionImpl<K, V> old = gate.enter(null);
try {
@@ -108,7 +108,7 @@ public class GridCacheDataStructuresProxy<K, V> implements GridCacheDataStructur
}
/** {@inheritDoc} */
- @Override public <T> GridCacheAtomicReference<T> atomicReference(String name, T initVal, boolean create)
+ @Override public <T> CacheAtomicReference<T> atomicReference(String name, T initVal, boolean create)
throws IgniteCheckedException {
GridCacheProjectionImpl<K, V> old = gate.enter(null);
@@ -133,7 +133,7 @@ public class GridCacheDataStructuresProxy<K, V> implements GridCacheDataStructur
}
/** {@inheritDoc} */
- @Override public <T, S> GridCacheAtomicStamped<T, S> atomicStamped(String name, T initVal, S initStamp,
+ @Override public <T, S> CacheAtomicStamped<T, S> atomicStamped(String name, T initVal, S initStamp,
boolean create) throws IgniteCheckedException {
GridCacheProjectionImpl<K, V> old = gate.enter(null);
@@ -158,7 +158,7 @@ public class GridCacheDataStructuresProxy<K, V> implements GridCacheDataStructur
}
/** {@inheritDoc} */
- @Override public <T> GridCacheQueue<T> queue(String name, int cap, boolean collocated, boolean create)
+ @Override public <T> CacheQueue<T> queue(String name, int cap, boolean collocated, boolean create)
throws IgniteCheckedException {
GridCacheProjectionImpl<K, V> old = gate.enter(null);
@@ -195,7 +195,7 @@ public class GridCacheDataStructuresProxy<K, V> implements GridCacheDataStructur
}
/** {@inheritDoc} */
- @Nullable @Override public <T> GridCacheSet<T> set(String name, boolean collocated, boolean create)
+ @Nullable @Override public <T> CacheSet<T> set(String name, boolean collocated, boolean create)
throws IgniteCheckedException {
GridCacheProjectionImpl<K, V> old = gate.enter(null);
@@ -220,7 +220,7 @@ public class GridCacheDataStructuresProxy<K, V> implements GridCacheDataStructur
}
/** {@inheritDoc} */
- @Nullable @Override public GridCacheCountDownLatch countDownLatch(String name, int cnt, boolean autoDel,
+ @Nullable @Override public CacheCountDownLatch countDownLatch(String name, int cnt, boolean autoDel,
boolean create) throws IgniteCheckedException {
GridCacheProjectionImpl<K, V> old = gate.enter(null);
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheInternalKeyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheInternalKeyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheInternalKeyImpl.java
index caaa6a6..9cfa8ad 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheInternalKeyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheInternalKeyImpl.java
@@ -17,7 +17,7 @@
package org.apache.ignite.internal.processors.cache.datastructures;
-import org.apache.ignite.cache.affinity.GridCacheAffinityKeyMapped;
+import org.apache.ignite.cache.affinity.CacheAffinityKeyMapped;
import org.apache.ignite.internal.util.typedef.*;
import org.apache.ignite.internal.util.typedef.internal.*;
@@ -52,7 +52,7 @@ public class GridCacheInternalKeyImpl implements GridCacheInternalKey, Externali
}
/** {@inheritDoc} */
- @GridCacheAffinityKeyMapped
+ @CacheAffinityKeyMapped
@Override public String name() {
return name;
}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueAdapter.java
index 21a092e..344a9c9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueAdapter.java
@@ -35,9 +35,9 @@ import java.util.concurrent.*;
import static java.util.concurrent.TimeUnit.*;
/**
- * Common code for {@link GridCacheQueue} implementation.
+ * Common code for {@link org.apache.ignite.cache.datastructures.CacheQueue} implementation.
*/
-public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> implements GridCacheQueue<T> {
+public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> implements CacheQueue<T> {
/** Value returned by closure updating queue header indicating that queue was removed. */
protected static final long QUEUE_REMOVED_IDX = Long.MIN_VALUE;
@@ -407,7 +407,7 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
}
/**
- * Checks result of closure modifying queue header, throws {@link org.apache.ignite.cache.datastructures.GridCacheDataStructureRemovedRuntimeException}
+ * Checks result of closure modifying queue header, throws {@link org.apache.ignite.cache.datastructures.CacheDataStructureRemovedRuntimeException}
* if queue was removed.
*
* @param idx Result of closure execution.
@@ -418,7 +418,7 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
}
/**
- * Checks queue state, throws {@link org.apache.ignite.cache.datastructures.GridCacheDataStructureRemovedRuntimeException} if queue was removed.
+ * Checks queue state, throws {@link org.apache.ignite.cache.datastructures.CacheDataStructureRemovedRuntimeException} if queue was removed.
*
* @param hdr Queue hdr.
*/
@@ -430,7 +430,7 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
/**
* Marks queue as removed.
*
- * @param throw0 If {@code true} then throws {@link org.apache.ignite.cache.datastructures.GridCacheDataStructureRemovedRuntimeException}.
+ * @param throw0 If {@code true} then throws {@link org.apache.ignite.cache.datastructures.CacheDataStructureRemovedRuntimeException}.
*/
void onRemoved(boolean throw0) {
rmvd = true;
@@ -438,7 +438,7 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
releaseSemaphores();
if (throw0)
- throw new GridCacheDataStructureRemovedRuntimeException("Queue has been removed from cache: " + this);
+ throw new CacheDataStructureRemovedRuntimeException("Queue has been removed from cache: " + this);
}
/**
@@ -636,7 +636,7 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
/**
* @return Item affinity key.
*/
- @GridCacheAffinityKeyMapped
+ @CacheAffinityKeyMapped
public Object affinityKey() {
return queueName();
}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueProxy.java
index eb51410..561afe8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueProxy.java
@@ -32,7 +32,7 @@ import java.util.concurrent.*;
/**
* Cache queue proxy.
*/
-public class GridCacheQueueProxy<T> implements GridCacheQueue<T>, Externalizable {
+public class GridCacheQueueProxy<T> implements CacheQueue<T>, Externalizable {
/** */
private static final long serialVersionUID = 0L;
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetImpl.java
index 78f3d6e..b6f5723 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetImpl.java
@@ -43,7 +43,7 @@ import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryTy
/**
* Cache set implementation.
*/
-public class GridCacheSetImpl<T> extends AbstractCollection<T> implements GridCacheSet<T> {
+public class GridCacheSetImpl<T> extends AbstractCollection<T> implements CacheSet<T> {
/** */
private static final int BATCH_SIZE = 100;
@@ -51,7 +51,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements GridCa
private final GridCacheContext ctx;
/** Cache. */
- private final GridCache<GridCacheSetItemKey, Boolean> cache;
+ private final Cache<GridCacheSetItemKey, Boolean> cache;
/** Logger. */
private final IgniteLogger log;
@@ -75,7 +75,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements GridCa
private final ReferenceQueue<SetIterator<?>> itRefQueue = new ReferenceQueue<>();
/** Iterators futures. */
- private final Map<WeakReference<SetIterator<?>>, GridCacheQueryFuture<?>> itFuts = new ConcurrentHashMap8<>();
+ private final Map<WeakReference<SetIterator<?>>, CacheQueryFuture<?>> itFuts = new ConcurrentHashMap8<>();
/**
* @param ctx Cache context.
@@ -123,7 +123,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements GridCa
return set != null ? set.size() : 0;
}
- GridCacheQuery qry = new GridCacheQueryAdapter<>(ctx, SET, null, null, null,
+ CacheQuery qry = new GridCacheQueryAdapter<>(ctx, SET, null, null, null,
new GridSetQueryPredicate<>(id, collocated), false, false);
Collection<ClusterNode> nodes = dataNodes(ctx.affinity().affinityTopologyVersion());
@@ -341,14 +341,14 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements GridCa
@SuppressWarnings("unchecked")
private GridCloseableIterator<T> iterator0() {
try {
- GridCacheQuery qry = new GridCacheQueryAdapter<>(ctx, SET, null, null, null,
+ CacheQuery qry = new GridCacheQueryAdapter<>(ctx, SET, null, null, null,
new GridSetQueryPredicate<>(id, collocated), false, false);
Collection<ClusterNode> nodes = dataNodes(ctx.affinity().affinityTopologyVersion());
qry.projection(ctx.grid().forNodes(nodes));
- GridCacheQueryFuture<T> fut = qry.execute();
+ CacheQueryFuture<T> fut = qry.execute();
SetIterator<T> it = new SetIterator<>(fut);
@@ -444,7 +444,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements GridCa
this.rmvd = rmvd;
if (rmvd) {
- for (GridCacheQueryFuture<?> fut : itFuts.values()) {
+ for (CacheQueryFuture<?> fut : itFuts.values()) {
try {
fut.cancel();
}
@@ -458,11 +458,11 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements GridCa
}
/**
- * Throws {@link GridCacheDataStructureRemovedRuntimeException} if set was removed.
+ * Throws {@link org.apache.ignite.cache.datastructures.CacheDataStructureRemovedRuntimeException} if set was removed.
*/
private void checkRemoved() {
if (rmvd)
- throw new GridCacheDataStructureRemovedRuntimeException("Set has been removed from cache: " + this);
+ throw new CacheDataStructureRemovedRuntimeException("Set has been removed from cache: " + this);
}
/**
@@ -473,7 +473,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements GridCa
try {
WeakReference<SetIterator<?>> weakRef = (WeakReference<SetIterator<?>>)itRef;
- GridCacheQueryFuture<?> fut = itFuts.remove(weakRef);
+ CacheQueryFuture<?> fut = itFuts.remove(weakRef);
if (fut != null)
fut.cancel();
@@ -528,7 +528,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements GridCa
private static final long serialVersionUID = -1460570789166994846L;
/** Query future. */
- private final GridCacheQueryFuture<T> fut;
+ private final CacheQueryFuture<T> fut;
/** Init flag. */
private boolean init;
@@ -545,7 +545,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements GridCa
/**
* @param fut Query future.
*/
- private SetIterator(GridCacheQueryFuture<T> fut) {
+ private SetIterator(CacheQueryFuture<T> fut) {
this.fut = fut;
weakRef = new WeakReference<SetIterator<?>>(this, itRefQueue);
@@ -701,7 +701,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements GridCa
/**
* @return Item affinity key.
*/
- @GridCacheAffinityKeyMapped
+ @CacheAffinityKeyMapped
public Object affinityKey() {
return setName;
}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetProxy.java
index d2140db..5557a18 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetProxy.java
@@ -33,7 +33,7 @@ import java.util.concurrent.*;
/**
* Cache set proxy.
*/
-public class GridCacheSetProxy<T> implements GridCacheSet<T>, Externalizable {
+public class GridCacheSetProxy<T> implements CacheSet<T>, Externalizable {
/** */
private static final long serialVersionUID = 0L;
@@ -486,7 +486,7 @@ public class GridCacheSetProxy<T> implements GridCacheSet<T>, Externalizable {
*/
private void enterBusy() {
if (!busyLock.enterBusy())
- throw new GridCacheDataStructureRemovedRuntimeException("Set has been removed from cache: " + delegate);
+ throw new CacheDataStructureRemovedRuntimeException("Set has been removed from cache: " + delegate);
}
/**
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridSetQueryPredicate.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridSetQueryPredicate.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridSetQueryPredicate.java
index 7cd6328..c9fad85 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridSetQueryPredicate.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridSetQueryPredicate.java
@@ -24,7 +24,7 @@ import org.apache.ignite.internal.util.typedef.internal.*;
import java.io.*;
/**
- * Predicate for query over {@link org.apache.ignite.cache.datastructures.GridCacheSet} items.
+ * Predicate for query over {@link org.apache.ignite.cache.datastructures.CacheSet} items.
*/
public class GridSetQueryPredicate<K, V> implements IgniteBiPredicate<K, V>, Externalizable {
/** */
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridTransactionalCacheQueueImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridTransactionalCacheQueueImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridTransactionalCacheQueueImpl.java
index 96f6f06..501c09a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridTransactionalCacheQueueImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridTransactionalCacheQueueImpl.java
@@ -31,7 +31,7 @@ import static org.apache.ignite.transactions.IgniteTxConcurrency.*;
import static org.apache.ignite.transactions.IgniteTxIsolation.*;
/**
- * {@link org.apache.ignite.cache.datastructures.GridCacheQueue} implementation using transactional cache.
+ * {@link org.apache.ignite.cache.datastructures.CacheQueue} implementation using transactional cache.
*/
public class GridTransactionalCacheQueueImpl<T> extends GridCacheQueueAdapter<T> {
/** */
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
index abbcf9d..7877771 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
@@ -68,7 +68,7 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
IgniteTxIsolation isolation,
boolean isInvalidate,
long accessTtl,
- IgnitePredicate<GridCacheEntry<K, V>>[] filter
+ IgnitePredicate<CacheEntry<K, V>>[] filter
) {
assert tx != null;
@@ -77,7 +77,7 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
/** {@inheritDoc} */
@Override public IgniteFuture<Boolean> lockAllAsync(Collection<? extends K> keys, long timeout,
- IgnitePredicate<GridCacheEntry<K, V>>... filter) {
+ IgnitePredicate<CacheEntry<K, V>>... filter) {
IgniteTxLocalEx<K, V> tx = ctx.tm().userTxx();
// Return value flag is true because we choose to bring values for explicit locks.
@@ -104,7 +104,7 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
boolean retval,
@Nullable IgniteTxIsolation isolation,
long accessTtl,
- IgnitePredicate<GridCacheEntry<K, V>>[] filter);
+ IgnitePredicate<CacheEntry<K, V>>[] filter);
/**
* @param key Key to remove.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
index bb7b180..9d07c52 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
@@ -200,7 +200,7 @@ public class GridDistributedTxRemoteAdapter<K, V> extends IgniteTxAdapter<K, V>
/** {@inheritDoc} */
@Override public GridTuple<V> peek(GridCacheContext<K, V> cacheCtx, boolean failFast, K key,
- IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridCacheFilterFailedException {
+ IgnitePredicate<CacheEntry<K, V>>[] filter) throws GridCacheFilterFailedException {
assert false : "Method peek can only be called on user transaction: " + this;
throw new IllegalStateException("Method peek can only be called on user transaction: " + this);
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridPartitionedCacheEntryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridPartitionedCacheEntryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridPartitionedCacheEntryImpl.java
index 76f01c39c..7c0dd24 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridPartitionedCacheEntryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridPartitionedCacheEntryImpl.java
@@ -97,7 +97,7 @@ public class GridPartitionedCacheEntryImpl<K, V> extends GridCacheEntryImpl<K, V
* @param filter Filter.
* @return Peeked value.
*/
- @Nullable public V peekDht(@Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
+ @Nullable public V peekDht(@Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
try {
return peekDht0(SMART, filter);
}
@@ -114,7 +114,7 @@ public class GridPartitionedCacheEntryImpl<K, V> extends GridCacheEntryImpl<K, V
* @throws IgniteCheckedException If failed.
*/
@Nullable private V peekNear0(@Nullable Collection<GridCachePeekMode> modes,
- @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
+ @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) throws IgniteCheckedException {
if (F.isEmpty(modes))
return peekNear0(SMART, filter);
@@ -138,7 +138,7 @@ public class GridPartitionedCacheEntryImpl<K, V> extends GridCacheEntryImpl<K, V
*/
@SuppressWarnings({"unchecked"})
@Nullable private V peekNear0(@Nullable GridCachePeekMode mode,
- @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
+ @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) throws IgniteCheckedException {
if (mode == null)
mode = SMART;
@@ -171,7 +171,7 @@ public class GridPartitionedCacheEntryImpl<K, V> extends GridCacheEntryImpl<K, V
* @throws IgniteCheckedException If failed.
*/
@Nullable private V peekDht0(@Nullable Collection<GridCachePeekMode> modes,
- @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
+ @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) throws IgniteCheckedException {
if (F.isEmpty(modes))
return peekDht0(SMART, filter);
@@ -195,7 +195,7 @@ public class GridPartitionedCacheEntryImpl<K, V> extends GridCacheEntryImpl<K, V
*/
@SuppressWarnings({"unchecked"})
@Nullable private V peekDht0(@Nullable GridCachePeekMode mode,
- @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
+ @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) throws IgniteCheckedException {
if (mode == null)
mode = SMART;
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index a55d447..6418e68 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -38,7 +38,7 @@ import java.io.*;
import java.util.*;
import java.util.concurrent.*;
-import static org.apache.ignite.cache.GridCacheDistributionMode.*;
+import static org.apache.ignite.cache.CacheDistributionMode.*;
import static org.apache.ignite.internal.processors.cache.GridCacheUtils.*;
import static org.apache.ignite.internal.processors.dr.GridDrType.*;
@@ -288,7 +288,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
*
* @throws GridDhtInvalidPartitionException If partition for the key is no longer valid.
*/
- @Override public GridCacheEntry<K, V> entry(K key) throws GridDhtInvalidPartitionException {
+ @Override public CacheEntry<K, V> entry(K key) throws GridDhtInvalidPartitionException {
return super.entry(key);
}
@@ -494,7 +494,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
@Nullable UUID subjId,
String taskName,
boolean deserializePortable,
- @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter
+ @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter
) {
return getAllAsync(keys,
true,
@@ -509,7 +509,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
}
/** {@inheritDoc} */
- @Override public V reload(K key, @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter)
+ @Override public V reload(K key, @Nullable IgnitePredicate<CacheEntry<K, V>>... filter)
throws IgniteCheckedException {
try {
return super.reload(key, filter);
@@ -534,7 +534,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
@Nullable UUID subjId,
String taskName,
boolean deserializePortable,
- @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter,
+ @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter,
@Nullable IgniteCacheExpiryPolicy expiry
) {
return getAllAsync(keys,
@@ -572,7 +572,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
@Nullable UUID subjId,
int taskNameHash,
boolean deserializePortable,
- IgnitePredicate<GridCacheEntry<K, V>>[] filter,
+ IgnitePredicate<CacheEntry<K, V>>[] filter,
@Nullable IgniteCacheExpiryPolicy expiry) {
GridDhtGetFuture<K, V> fut = new GridDhtGetFuture<>(ctx,
msgId,
@@ -789,12 +789,12 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
/** {@inheritDoc} */
@Override public void unlockAll(Collection<? extends K> keys,
- IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
+ IgnitePredicate<CacheEntry<K, V>>[] filter) {
assert false;
}
/** {@inheritDoc} */
- @Override public Set<GridCacheEntry<K, V>> entrySet(int part) {
+ @Override public Set<CacheEntry<K, V>> entrySet(int part) {
return new PartitionEntrySet(part);
}
@@ -806,7 +806,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
/**
*
*/
- private class PartitionEntrySet extends AbstractSet<GridCacheEntry<K, V>> {
+ private class PartitionEntrySet extends AbstractSet<CacheEntry<K, V>> {
/** */
private int partId;
@@ -818,7 +818,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
}
/** {@inheritDoc} */
- @NotNull @Override public Iterator<GridCacheEntry<K, V>> iterator() {
+ @NotNull @Override public Iterator<CacheEntry<K, V>> iterator() {
final GridDhtLocalPartition<K, V> part = ctx.topology().localPartition(partId,
ctx.discovery().topologyVersion(), false);
@@ -829,10 +829,10 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
/** {@inheritDoc} */
@Override public boolean remove(Object o) {
- if (!(o instanceof GridCacheEntry))
+ if (!(o instanceof CacheEntry))
return false;
- GridCacheEntry<K, V> entry = (GridCacheEntry<K, V>)o;
+ CacheEntry<K, V> entry = (CacheEntry<K, V>)o;
K key = entry.getKey();
V val = entry.peek();
@@ -861,10 +861,10 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
/** {@inheritDoc} */
@Override public boolean contains(Object o) {
- if (!(o instanceof GridCacheEntry))
+ if (!(o instanceof CacheEntry))
return false;
- GridCacheEntry<K, V> entry = (GridCacheEntry<K, V>)o;
+ CacheEntry<K, V> entry = (CacheEntry<K, V>)o;
return partId == entry.partition() && F.eq(entry.peek(), peek(entry.getKey()));
}
@@ -885,7 +885,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
/** {@inheritDoc} */
@Override public List<GridCacheClearAllRunnable<K, V>> splitClearAll() {
- GridCacheDistributionMode mode = configuration().getDistributionMode();
+ CacheDistributionMode mode = configuration().getDistributionMode();
return (mode == PARTITIONED_ONLY || mode == NEAR_PARTITIONED) ? super.splitClearAll() :
Collections.<GridCacheClearAllRunnable<K, V>>emptyList();
@@ -912,15 +912,15 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
/**
* Complex partition iterator for both partition and swap iteration.
*/
- private static class PartitionEntryIterator<K, V> extends GridIteratorAdapter<GridCacheEntry<K, V>> {
+ private static class PartitionEntryIterator<K, V> extends GridIteratorAdapter<CacheEntry<K, V>> {
/** */
private static final long serialVersionUID = 0L;
/** Next entry. */
- private GridCacheEntry<K, V> entry;
+ private CacheEntry<K, V> entry;
/** Last seen entry to support remove. */
- private GridCacheEntry<K, V> last;
+ private CacheEntry<K, V> last;
/** Partition iterator. */
private final Iterator<GridDhtCacheEntry<K, V>> partIt;
@@ -940,7 +940,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
}
/** {@inheritDoc} */
- @Override public GridCacheEntry<K, V> nextX() throws IgniteCheckedException {
+ @Override public CacheEntry<K, V> nextX() throws IgniteCheckedException {
if (!hasNext())
throw new NoSuchElementException();
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
index 5aa755b..1b0e849 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
@@ -629,7 +629,7 @@ public class GridDhtCacheEntry<K, V> extends GridDistributedCacheEntry<K, V> {
}
/** {@inheritDoc} */
- @Override public GridCacheEntry<K, V> wrap(boolean prjAware) {
+ @Override public CacheEntry<K, V> wrap(boolean prjAware) {
GridCacheContext<K, V> nearCtx = cctx.dht().near().context();
GridCacheProjectionImpl<K, V> prjPerCall = nearCtx.projectionPerCall();
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntryImpl.java
index 14d9387..70b7057 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntryImpl.java
@@ -86,7 +86,7 @@ public class GridDhtCacheEntryImpl<K, V> extends GridCacheEntryImpl<K, V> {
* @param filter Filter.
* @return Peeked value.
*/
- @Nullable private V peekDht(@Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
+ @Nullable private V peekDht(@Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
try {
return peekDht0(SMART, filter);
}
@@ -103,7 +103,7 @@ public class GridDhtCacheEntryImpl<K, V> extends GridCacheEntryImpl<K, V> {
* @throws IgniteCheckedException If failed.
*/
@Nullable private V peekDht0(@Nullable Collection<GridCachePeekMode> modes,
- @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
+ @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) throws IgniteCheckedException {
if (F.isEmpty(modes))
return peekDht0(SMART, filter);
@@ -127,7 +127,7 @@ public class GridDhtCacheEntryImpl<K, V> extends GridCacheEntryImpl<K, V> {
*/
@SuppressWarnings({"unchecked"})
@Nullable private V peekDht0(@Nullable GridCachePeekMode mode,
- @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
+ @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) throws IgniteCheckedException {
if (mode == null)
mode = SMART;
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
index 0bb3a25..170f85d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
@@ -78,7 +78,7 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
private IgniteTxLocalEx<K, V> tx;
/** Filters. */
- private IgnitePredicate<GridCacheEntry<K, V>>[] filters;
+ private IgnitePredicate<CacheEntry<K, V>>[] filters;
/** Logger. */
private IgniteLogger log;
@@ -129,7 +129,7 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
boolean reload,
@Nullable IgniteTxLocalEx<K, V> tx,
long topVer,
- @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filters,
+ @Nullable IgnitePredicate<CacheEntry<K, V>>[] filters,
@Nullable UUID subjId,
int taskNameHash,
boolean deserializePortable,
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
index 56687eb..74d9163 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
@@ -109,7 +109,7 @@ public final class GridDhtLockFuture<K, V> extends GridCompoundIdentityFuture<Bo
private IgniteLogger log;
/** Filter. */
- private IgnitePredicate<GridCacheEntry<K, V>>[] filter;
+ private IgnitePredicate<CacheEntry<K, V>>[] filter;
/** Transaction. */
private GridDhtTxLocalAdapter<K, V> tx;
@@ -163,7 +163,7 @@ public final class GridDhtLockFuture<K, V> extends GridCompoundIdentityFuture<Bo
GridDhtTxLocalAdapter<K, V> tx,
long threadId,
long accessTtl,
- IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
+ IgnitePredicate<CacheEntry<K, V>>[] filter) {
super(cctx.kernalContext(), CU.boolReducer());
assert nearNodeId != null;
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
index 84d9ff0..f6e69eb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
@@ -557,7 +557,7 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
boolean retval,
IgniteTxIsolation isolation,
long accessTtl,
- IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
+ IgnitePredicate<CacheEntry<K, V>>[] filter) {
return lockAllAsyncInternal(keys,
timeout,
txx,
@@ -591,7 +591,7 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
boolean retval,
IgniteTxIsolation isolation,
long accessTtl,
- IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
+ IgnitePredicate<CacheEntry<K, V>>[] filter) {
if (keys == null || keys.isEmpty())
return new GridDhtFinishedFuture<>(ctx.kernalContext(), true);
@@ -668,7 +668,7 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
final GridCacheContext<K, V> cacheCtx,
final ClusterNode nearNode,
final GridNearLockRequest<K, V> req,
- @Nullable final IgnitePredicate<GridCacheEntry<K, V>>[] filter0) {
+ @Nullable final IgnitePredicate<CacheEntry<K, V>>[] filter0) {
final List<K> keys = req.keys();
IgniteFuture<Object> keyFut = null;
@@ -694,7 +694,7 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
if (exx != null)
return new GridDhtFinishedFuture<>(ctx.kernalContext(), exx);
- IgnitePredicate<GridCacheEntry<K, V>>[] filter = filter0;
+ IgnitePredicate<CacheEntry<K, V>>[] filter = filter0;
// Set message into thread context.
GridDhtTxLocal<K, V> tx = null;
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
index 7b168af..564100b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
@@ -615,7 +615,7 @@ public abstract class GridDhtTxLocalAdapter<K, V> extends IgniteTxLocalAdapter<K
final boolean read,
final Set<K> skipped,
final long accessTtl,
- @Nullable final IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
+ @Nullable final IgnitePredicate<CacheEntry<K, V>>[] filter) {
if (log.isDebugEnabled())
log.debug("Before acquiring transaction lock on keys [passedKeys=" + passedKeys + ", skipped=" +
skipped + ']');
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
index ff90852..b16d913 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
@@ -79,7 +79,7 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
private GridCacheVersion ver;
/** Filters. */
- private IgnitePredicate<GridCacheEntry<K, V>>[] filters;
+ private IgnitePredicate<CacheEntry<K, V>>[] filters;
/** Logger. */
private IgniteLogger log;
@@ -130,7 +130,7 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
boolean readThrough,
boolean reload,
boolean forcePrimary,
- @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filters,
+ @Nullable IgnitePredicate<CacheEntry<K, V>>[] filters,
@Nullable UUID subjId,
String taskName,
boolean deserializePortable,
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/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 23be1cd..683b7b9 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
@@ -52,7 +52,7 @@ import java.util.concurrent.locks.*;
import static org.apache.ignite.IgniteSystemProperties.*;
import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.*;
import static org.apache.ignite.cache.GridCachePeekMode.*;
-import static org.apache.ignite.cache.GridCacheWriteSynchronizationMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
import static org.apache.ignite.internal.processors.cache.GridCacheOperation.*;
import static org.apache.ignite.internal.processors.cache.GridCacheUtils.*;
import static org.apache.ignite.internal.processors.dr.GridDrType.*;
@@ -237,7 +237,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
}
/** {@inheritDoc} */
- @Override public GridCacheEntry<K, V> entry(K key) {
+ @Override public CacheEntry<K, V> entry(K key) {
return new GridDhtCacheEntryImpl<>(ctx.projectionPerCall(), ctx, key, null);
}
@@ -269,7 +269,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
@Nullable UUID subjId,
final String taskName,
final boolean deserializePortable,
- @Nullable final IgnitePredicate<GridCacheEntry<K, V>>[] filter
+ @Nullable final IgnitePredicate<CacheEntry<K, V>>[] filter
) {
GridCacheProjectionImpl<K, V> prj = ctx.projectionPerCall();
@@ -295,26 +295,26 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
/** {@inheritDoc} */
@Override public V put(K key, V val, @Nullable GridCacheEntryEx<K, V> cached, long ttl,
- @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
+ @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) throws IgniteCheckedException {
return putAsync(key, val, cached, ttl, filter).get();
}
/** {@inheritDoc} */
@Override public boolean putx(K key, V val, @Nullable GridCacheEntryEx<K, V> cached,
- long ttl, @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) throws IgniteCheckedException {
+ long ttl, @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) throws IgniteCheckedException {
return putxAsync(key, val, cached, ttl, filter).get();
}
/** {@inheritDoc} */
@Override public boolean putx(K key, V val,
- IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
+ IgnitePredicate<CacheEntry<K, V>>[] filter) throws IgniteCheckedException {
return putxAsync(key, val, filter).get();
}
/** {@inheritDoc} */
@SuppressWarnings("unchecked")
@Override public IgniteFuture<V> putAsync(K key, V val, @Nullable GridCacheEntryEx<K, V> entry,
- long ttl, @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) {
+ long ttl, @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
return updateAllAsync0(F0.asMap(key, val),
null,
null,
@@ -329,7 +329,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
/** {@inheritDoc} */
@SuppressWarnings("unchecked")
@Override public IgniteFuture<Boolean> putxAsync(K key, V val, @Nullable GridCacheEntryEx<K, V> entry, long ttl,
- @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) {
+ @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
return updateAllAsync0(F0.asMap(key, val),
null,
null,
@@ -423,13 +423,13 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
/** {@inheritDoc} */
@Override public void putAll(Map<? extends K, ? extends V> m,
- IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
+ IgnitePredicate<CacheEntry<K, V>>[] filter) throws IgniteCheckedException {
putAllAsync(m, filter).get();
}
/** {@inheritDoc} */
@Override public IgniteFuture<?> putAllAsync(Map<? extends K, ? extends V> m,
- @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
+ @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
return updateAllAsync0(m,
null,
null,
@@ -463,26 +463,26 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
/** {@inheritDoc} */
@Override public V remove(K key, @Nullable GridCacheEntryEx<K, V> entry,
- @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) throws IgniteCheckedException {
+ @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) throws IgniteCheckedException {
return removeAsync(key, entry, filter).get();
}
/** {@inheritDoc} */
@SuppressWarnings("unchecked")
@Override public IgniteFuture<V> removeAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
- @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) {
+ @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
return removeAllAsync0(Collections.singletonList(key), null, entry, true, false, filter);
}
/** {@inheritDoc} */
@Override public void removeAll(Collection<? extends K> keys,
- IgnitePredicate<GridCacheEntry<K, V>>... filter) throws IgniteCheckedException {
+ IgnitePredicate<CacheEntry<K, V>>... filter) throws IgniteCheckedException {
removeAllAsync(keys, filter).get();
}
/** {@inheritDoc} */
@Override public IgniteFuture<?> removeAllAsync(Collection<? extends K> keys,
- IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
+ IgnitePredicate<CacheEntry<K, V>>[] filter) {
A.notNull(keys, "keys");
return removeAllAsync0(keys, null, null, false, false, filter);
@@ -490,14 +490,14 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
/** {@inheritDoc} */
@Override public boolean removex(K key, @Nullable GridCacheEntryEx<K, V> entry,
- @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) throws IgniteCheckedException {
+ @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) throws IgniteCheckedException {
return removexAsync(key, entry, filter).get();
}
/** {@inheritDoc} */
@SuppressWarnings("unchecked")
@Override public IgniteFuture<Boolean> removexAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
- @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) {
+ @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
return removeAllAsync0(Collections.singletonList(key), null, entry, false, false, filter);
}
@@ -512,12 +512,12 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
}
/** {@inheritDoc} */
- @Override public void removeAll(IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
+ @Override public void removeAll(IgnitePredicate<CacheEntry<K, V>>[] filter) throws IgniteCheckedException {
removeAllAsync(filter).get();
}
/** {@inheritDoc} */
- @Override public IgniteFuture<?> removeAllAsync(IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
+ @Override public IgniteFuture<?> removeAllAsync(IgnitePredicate<CacheEntry<K, V>>[] filter) {
return removeAllAsync(keySet(filter), filter);
}
@@ -591,7 +591,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
boolean retval,
@Nullable IgniteTxIsolation isolation,
long accessTtl,
- IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
+ IgnitePredicate<CacheEntry<K, V>>[] filter) {
return new FinishedLockFuture(new UnsupportedOperationException("Locks are not supported for " +
"CacheAtomicityMode.ATOMIC mode (use CacheAtomicityMode.TRANSACTIONAL instead)"));
}
@@ -732,7 +732,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
final boolean retval,
final boolean rawRetval,
@Nullable GridCacheEntryEx<K, V> cached,
- @Nullable final IgnitePredicate<GridCacheEntry<K, V>>[] filter
+ @Nullable final IgnitePredicate<CacheEntry<K, V>>[] filter
) {
if (map != null && keyCheck)
validateCacheKeys(map.keySet());
@@ -790,7 +790,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
@Nullable GridCacheEntryEx<K, V> cached,
final boolean retval,
boolean rawRetval,
- @Nullable final IgnitePredicate<GridCacheEntry<K, V>>[] filter
+ @Nullable final IgnitePredicate<CacheEntry<K, V>>[] filter
) {
assert keys != null || drMap != null;
@@ -848,7 +848,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
private IgniteFuture<Map<K, V>> getAllAsync0(@Nullable Collection<? extends K> keys,
boolean reload,
boolean forcePrimary,
- @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter,
+ @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter,
UUID subjId,
String taskName,
boolean deserializePortable,
@@ -1956,7 +1956,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
else {
assert op == DELETE : op;
- // Old value should be already loaded for 'GridCacheInterceptor.onBeforeRemove'.
+ // Old value should be already loaded for 'CacheInterceptor.onBeforeRemove'.
ctx.config().<K, V>getInterceptor().onAfterRemove(entry.key(), updRes.oldValue());
}
}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java
index 8d0fdaa..6078425 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java
@@ -47,7 +47,7 @@ public class GridDhtAtomicCacheEntry<K, V> extends GridDhtCacheEntry<K, V> {
}
/** {@inheritDoc} */
- @Override public GridCacheEntry<K, V> wrap(boolean prjAware) {
+ @Override public CacheEntry<K, V> wrap(boolean prjAware) {
GridCacheProjectionImpl<K, V> prjPerCall = cctx.projectionPerCall();
if (prjPerCall != null && prjAware)
@@ -62,7 +62,7 @@ public class GridDhtAtomicCacheEntry<K, V> extends GridDhtCacheEntry<K, V> {
}
/** {@inheritDoc} */
- @Override public GridCacheEntry<K, V> wrapFilterLocked() throws IgniteCheckedException {
+ @Override public CacheEntry<K, V> wrapFilterLocked() throws IgniteCheckedException {
assert Thread.holdsLock(this);
return new GridCacheFilterEvaluationEntry<>(key, rawGetOrUnmarshal(true), this);
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
index 61e278c..317010f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
@@ -36,7 +36,7 @@ import java.util.*;
import java.util.concurrent.*;
import java.util.concurrent.atomic.*;
-import static org.apache.ignite.cache.GridCacheWriteSynchronizationMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
/**
* DHT atomic cache backup update future.
@@ -225,7 +225,7 @@ public class GridDhtAtomicUpdateFuture<K, V> extends GridFutureAdapter<Void>
if (log.isDebugEnabled())
log.debug("Mapping entry to DHT nodes [nodes=" + U.nodeIds(dhtNodes) + ", entry=" + entry + ']');
- GridCacheWriteSynchronizationMode syncMode = updateReq.writeSynchronizationMode();
+ CacheWriteSynchronizationMode syncMode = updateReq.writeSynchronizationMode();
keys.add(entry.key());
@@ -279,7 +279,7 @@ public class GridDhtAtomicUpdateFuture<K, V> extends GridFutureAdapter<Void>
EntryProcessor<K, V, ?> entryProcessor,
long ttl,
long expireTime) {
- GridCacheWriteSynchronizationMode syncMode = updateReq.writeSynchronizationMode();
+ CacheWriteSynchronizationMode syncMode = updateReq.writeSynchronizationMode();
keys.add(entry.key());
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cafee25f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
index 5f45af7..d1eee48 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
@@ -91,7 +91,7 @@ public class GridDhtAtomicUpdateRequest<K, V> extends GridCacheMessage<K, V> imp
private GridLongList nearExpireTimes;
/** Write synchronization mode. */
- private GridCacheWriteSynchronizationMode syncMode;
+ private CacheWriteSynchronizationMode syncMode;
/** Keys to update. */
@GridToStringInclude
@@ -178,7 +178,7 @@ public class GridDhtAtomicUpdateRequest<K, V> extends GridCacheMessage<K, V> imp
UUID nodeId,
GridCacheVersion futVer,
GridCacheVersion writeVer,
- GridCacheWriteSynchronizationMode syncMode,
+ CacheWriteSynchronizationMode syncMode,
long topVer,
boolean forceTransformBackups,
UUID subjId,
@@ -414,7 +414,7 @@ public class GridDhtAtomicUpdateRequest<K, V> extends GridCacheMessage<K, V> imp
/**
* @return Cache write synchronization mode.
*/
- public GridCacheWriteSynchronizationMode writeSynchronizationMode() {
+ public CacheWriteSynchronizationMode writeSynchronizationMode() {
return syncMode;
}
@@ -1192,7 +1192,7 @@ public class GridDhtAtomicUpdateRequest<K, V> extends GridCacheMessage<K, V> imp
byte syncMode0 = commState.getByte();
- syncMode = GridCacheWriteSynchronizationMode.fromOrdinal(syncMode0);
+ syncMode = CacheWriteSynchronizationMode.fromOrdinal(syncMode0);
commState.idx++;
|