Skip to content

Commit

Permalink
IGNITE-23110 Disallow IgniteCache#clear() within transaction
Browse files Browse the repository at this point in the history
  • Loading branch information
J-Bakuli committed Nov 19, 2024
1 parent 26b1a4f commit 23566ce
Show file tree
Hide file tree
Showing 3 changed files with 11 additions and 26 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1064,25 +1064,16 @@ public List<GridCacheClearAllRunnable<K, V>> splitClearLocally(boolean srv, bool

/** {@inheritDoc} */
@Override public void clear() throws IgniteCheckedException {
if (ctx.transactional() && ctx.grid().transactions().tx() != null)
throw new CacheException(NON_TRANSACTIONAL_IGNITE_CACHE_CLEAR_IN_TX_ERROR_MESSAGE);

clear((Set<? extends K>)null);
}

/** {@inheritDoc} */
@Override public void clear(K key) throws IgniteCheckedException {
if (ctx.transactional() && ctx.grid().transactions().tx() != null)
throw new CacheException(NON_TRANSACTIONAL_IGNITE_CACHE_CLEAR_IN_TX_ERROR_MESSAGE);

clear(Collections.singleton(key));
}

/** {@inheritDoc} */
@Override public void clearAll(Set<? extends K> keys) throws IgniteCheckedException {
if (ctx.transactional() && ctx.grid().transactions().tx() != null)
throw new CacheException(NON_TRANSACTIONAL_IGNITE_CACHE_CLEAR_IN_TX_ERROR_MESSAGE);

clear(keys);
}

Expand All @@ -1106,6 +1097,9 @@ public List<GridCacheClearAllRunnable<K, V>> splitClearLocally(boolean srv, bool
* @throws IgniteCheckedException In case of error.
*/
private void clear(@Nullable Set<? extends K> keys) throws IgniteCheckedException {
if (ctx.transactional() && ctx.grid().transactions().tx() != null)
throw new CacheException(NON_TRANSACTIONAL_IGNITE_CACHE_CLEAR_IN_TX_ERROR_MESSAGE);

ctx.shared().cache().checkReadOnlyState("clear", ctx.config());

executeClearTask(keys, false).get();
Expand All @@ -1117,6 +1111,9 @@ private void clear(@Nullable Set<? extends K> keys) throws IgniteCheckedExceptio
* @return Future.
*/
private IgniteInternalFuture<?> clearAsync(@Nullable final Set<? extends K> keys) {
if (ctx.transactional() && ctx.grid().transactions().tx() != null)
throw new CacheException(NON_TRANSACTIONAL_IGNITE_CACHE_CLEAR_IN_TX_ERROR_MESSAGE);

ctx.shared().cache().checkReadOnlyState("clear", ctx.config());

return executeClearTask(keys, false).chainCompose(fut -> executeClearTask(keys, true));
Expand Down Expand Up @@ -3978,6 +3975,9 @@ protected Object readResolve() throws ObjectStreamException {
* @param readers Whether to clear readers.
*/
private boolean clearLocally0(K key, boolean readers) {
if (ctx.transactional() && ctx.grid().transactions().tx() != null)
throw new CacheException(NON_TRANSACTIONAL_IGNITE_CACHE_CLEAR_IN_TX_ERROR_MESSAGE);

ctx.shared().cache().checkReadOnlyState("clear", ctx.config());

ctx.checkSecurity(SecurityPermission.CACHE_REMOVE);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1645,9 +1645,6 @@ private IgniteInternalFuture<Void> putAsync0(K key, V val) {

/** {@inheritDoc} */
@Override public IgniteFuture<Void> clearAsync(K key) {
if (ctx.transactional() && ctx.grid().transactions().tx() != null)
throw new CacheException(GridCacheAdapter.NON_TRANSACTIONAL_IGNITE_CACHE_CLEAR_IN_TX_ERROR_MESSAGE);

IgniteInternalCache<K, V> delegate = getDelegateSafe();

return (IgniteFuture<Void>)createFuture(delegate.clearAsync(key));
Expand All @@ -1670,9 +1667,6 @@ private IgniteInternalFuture<Void> putAsync0(K key, V val) {

/** {@inheritDoc} */
@Override public IgniteFuture<Void> clearAllAsync(Set<? extends K> keys) {
if (ctx.transactional() && ctx.grid().transactions().tx() != null)
throw new CacheException(GridCacheAdapter.NON_TRANSACTIONAL_IGNITE_CACHE_CLEAR_IN_TX_ERROR_MESSAGE);

IgniteInternalCache<K, V> delegate = getDelegateSafe();

return (IgniteFuture<Void>)createFuture(delegate.clearAllAsync(keys));
Expand All @@ -1695,9 +1689,6 @@ private IgniteInternalFuture<Void> putAsync0(K key, V val) {

/** {@inheritDoc} */
@Override public IgniteFuture<Void> clearAsync() {
if (ctx.transactional() && ctx.grid().transactions().tx() != null)
throw new CacheException(GridCacheAdapter.NON_TRANSACTIONAL_IGNITE_CACHE_CLEAR_IN_TX_ERROR_MESSAGE);

IgniteInternalCache<K, V> delegate = getDelegateSafe();

return (IgniteFuture<Void>)createFuture(delegate.clearAsync());
Expand All @@ -1712,9 +1703,8 @@ private IgniteInternalFuture<Void> putAsync0(K key, V val) {

/** {@inheritDoc} */
@Override public void localClearAll(Set<? extends K> keys) {
if (ctx.transactional() && ctx.grid().transactions().tx() != null)
throw new CacheException(GridCacheAdapter.NON_TRANSACTIONAL_IGNITE_CACHE_CLEAR_IN_TX_ERROR_MESSAGE);

/* if (ctx.transactional() && ctx.grid().transactions().tx() != null)
throw new CacheException(GridCacheAdapter.NON_TRANSACTIONAL_IGNITE_CACHE_CLEAR_IN_TX_ERROR_MESSAGE);*/
IgniteInternalCache<K, V> delegate = getDelegateSafe();

for (K key : keys)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@
import java.util.Set;
import java.util.UUID;
import javax.cache.Cache;
import javax.cache.CacheException;
import javax.cache.expiry.ExpiryPolicy;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteException;
Expand All @@ -38,7 +37,6 @@
import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
import org.apache.ignite.internal.processors.cache.CacheEntryPredicate;
import org.apache.ignite.internal.processors.cache.CacheEntryPredicateAdapter;
import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
import org.apache.ignite.internal.processors.cache.GridCacheClearAllRunnable;
import org.apache.ignite.internal.processors.cache.GridCacheContext;
import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
Expand Down Expand Up @@ -352,9 +350,6 @@ public Set<Cache.Entry<K, V>> nearEntries() {

/** {@inheritDoc} */
@Override public boolean clearLocally(K key) {
if (ctx.transactional() && ctx.grid().transactions().tx() != null)
throw new CacheException(GridCacheAdapter.NON_TRANSACTIONAL_IGNITE_CACHE_CLEAR_IN_TX_ERROR_MESSAGE);

return super.clearLocally(key) | dht().clearLocally(key);
}

Expand Down

0 comments on commit 23566ce

Please sign in to comment.