refactoring

pull/5732/head
Nikita Koksharov 10 months ago
parent 99bd0ea6d5
commit 41a492dac2

@ -176,7 +176,7 @@ public final class Redisson implements RedissonClient {
public <V, L> RTimeSeries<V, L> getTimeSeries(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonTimeSeries<>(params.getCodec(), evictionScheduler,
new CommandAsyncService(commandExecutor, params),
commandExecutor.copy(params),
params.getName());
}
@ -193,7 +193,7 @@ public final class Redisson implements RedissonClient {
@Override
public <K, V> RStream<K, V> getStream(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonStream<>(params.getCodec(), new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonStream<>(params.getCodec(), commandExecutor.copy(params), params.getName());
}
@Override
@ -209,7 +209,7 @@ public final class Redisson implements RedissonClient {
@Override
public RSearch getSearch(OptionalOptions options) {
OptionalParams params = (OptionalParams) options;
return new RedissonSearch(params.getCodec(), new CommandAsyncService(commandExecutor, params));
return new RedissonSearch(params.getCodec(), commandExecutor.copy(params));
}
@Override
@ -220,7 +220,7 @@ public final class Redisson implements RedissonClient {
@Override
public RBinaryStream getBinaryStream(CommonOptions options) {
CommonParams params = (CommonParams) options;
return new RedissonBinaryStream(new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonBinaryStream(commandExecutor.copy(params), params.getName());
}
@Override
@ -236,7 +236,7 @@ public final class Redisson implements RedissonClient {
@Override
public <V> RGeo<V> getGeo(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonGeo<>(params.getCodec(), new CommandAsyncService(commandExecutor, params),
return new RedissonGeo<>(params.getCodec(), commandExecutor.copy(params),
params.getName(), this);
}
@ -253,7 +253,7 @@ public final class Redisson implements RedissonClient {
@Override
public RRateLimiter getRateLimiter(CommonOptions options) {
CommonParams params = (CommonParams) options;
return new RedissonRateLimiter(new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonRateLimiter(commandExecutor.copy(params), params.getName());
}
@Override
@ -264,7 +264,7 @@ public final class Redisson implements RedissonClient {
@Override
public <V> RBucket<V> getBucket(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonBucket<>(params.getCodec(), new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonBucket<>(params.getCodec(), commandExecutor.copy(params), params.getName());
}
@Override
@ -280,7 +280,7 @@ public final class Redisson implements RedissonClient {
@Override
public RBuckets getBuckets(OptionalOptions options) {
OptionalParams params = (OptionalParams) options;
return new RedissonBuckets(params.getCodec(), new CommandAsyncService(commandExecutor, params));
return new RedissonBuckets(params.getCodec(), commandExecutor.copy(params));
}
@Override
@ -307,7 +307,7 @@ public final class Redisson implements RedissonClient {
@Override
public <V> RHyperLogLog<V> getHyperLogLog(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonHyperLogLog<V>(params.getCodec(), new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonHyperLogLog<V>(params.getCodec(), commandExecutor.copy(params), params.getName());
}
@Override
@ -323,7 +323,7 @@ public final class Redisson implements RedissonClient {
@Override
public <V> RList<V> getList(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonList<V>(params.getCodec(), new CommandAsyncService(commandExecutor, params), params.getName(), this);
return new RedissonList<V>(params.getCodec(), commandExecutor.copy(params), params.getName(), this);
}
@Override
@ -339,7 +339,7 @@ public final class Redisson implements RedissonClient {
@Override
public <K, V> RListMultimap<K, V> getListMultimap(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonListMultimap<>(params.getCodec(), new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonListMultimap<>(params.getCodec(), commandExecutor.copy(params), params.getName());
}
@Override
@ -384,7 +384,7 @@ public final class Redisson implements RedissonClient {
ops.writerRetryAttempts(params.getWriteRetryAttempts());
}
return new RedissonLocalCachedMap<>(params.getCodec(), new CommandAsyncService(commandExecutor, params), params.getName(),
return new RedissonLocalCachedMap<>(params.getCodec(), commandExecutor.copy(params), params.getName(),
ops, evictionScheduler, this, writeBehindService);
}
@ -417,7 +417,7 @@ public final class Redisson implements RedissonClient {
ops.writerRetryAttempts(params.getWriteRetryAttempts());
}
return new RedissonMap<>(new CommandAsyncService(commandExecutor, params), params.getName(),
return new RedissonMap<>(commandExecutor.copy(params), params.getName(),
this, ops, writeBehindService);
}
@ -440,7 +440,7 @@ public final class Redisson implements RedissonClient {
public <K, V> RSetMultimapCache<K, V> getSetMultimapCache(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonSetMultimapCache<K, V>(evictionScheduler, params.getCodec(),
new CommandAsyncService(commandExecutor, params), params.getName());
commandExecutor.copy(params), params.getName());
}
@Override
@ -457,7 +457,7 @@ public final class Redisson implements RedissonClient {
public <K, V> RListMultimapCache<K, V> getListMultimapCache(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonListMultimapCache<K, V>(evictionScheduler, params.getCodec(),
new CommandAsyncService(commandExecutor, params), params.getName());
commandExecutor.copy(params), params.getName());
}
@Override
@ -468,7 +468,7 @@ public final class Redisson implements RedissonClient {
@Override
public <K, V> RSetMultimap<K, V> getSetMultimap(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonSetMultimap<>(params.getCodec(), new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonSetMultimap<>(params.getCodec(), commandExecutor.copy(params), params.getName());
}
@Override
@ -485,7 +485,7 @@ public final class Redisson implements RedissonClient {
public <V> RSetCache<V> getSetCache(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonSetCache<V>(params.getCodec(), evictionScheduler,
new CommandAsyncService(commandExecutor, params), params.getName(), this);
commandExecutor.copy(params), params.getName(), this);
}
@Override
@ -552,7 +552,7 @@ public final class Redisson implements RedissonClient {
@Override
public RLock getLock(CommonOptions options) {
CommonParams params = (CommonParams) options;
return new RedissonLock(new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonLock(commandExecutor.copy(params), params.getName());
}
@Override
@ -573,7 +573,7 @@ public final class Redisson implements RedissonClient {
@Override
public RFencedLock getFencedLock(CommonOptions options) {
CommonParams params = (CommonParams) options;
return new RedissonFencedLock(new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonFencedLock(commandExecutor.copy(params), params.getName());
}
@Override
@ -594,7 +594,7 @@ public final class Redisson implements RedissonClient {
@Override
public RLock getFairLock(CommonOptions options) {
CommonParams params = (CommonParams) options;
return new RedissonFairLock(new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonFairLock(commandExecutor.copy(params), params.getName());
}
@Override
@ -605,7 +605,7 @@ public final class Redisson implements RedissonClient {
@Override
public RReadWriteLock getReadWriteLock(CommonOptions options) {
CommonParams params = (CommonParams) options;
return new RedissonReadWriteLock(new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonReadWriteLock(commandExecutor.copy(params), params.getName());
}
@Override
@ -621,7 +621,7 @@ public final class Redisson implements RedissonClient {
@Override
public <V> RSet<V> getSet(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonSet<V>(params.getCodec(), new CommandAsyncService(commandExecutor, params), params.getName(), this);
return new RedissonSet<V>(params.getCodec(), commandExecutor.copy(params), params.getName(), this);
}
@Override
@ -637,7 +637,7 @@ public final class Redisson implements RedissonClient {
@Override
public RFunction getFunction(OptionalOptions options) {
OptionalParams params = (OptionalParams) options;
return new RedissonFuction(new CommandAsyncService(commandExecutor, params), params.getCodec());
return new RedissonFuction(commandExecutor.copy(params), params.getCodec());
}
@Override
@ -653,7 +653,7 @@ public final class Redisson implements RedissonClient {
@Override
public RScript getScript(OptionalOptions options) {
OptionalParams params = (OptionalParams) options;
return new RedissonScript(new CommandAsyncService(commandExecutor, params), params.getCodec());
return new RedissonScript(commandExecutor.copy(params), params.getCodec());
}
@Override
@ -683,7 +683,7 @@ public final class Redisson implements RedissonClient {
.idGenerator(params.getIdGenerator())
.taskRetryInterval(params.getTaskRetryInterval(), TimeUnit.MILLISECONDS);
return new RedissonExecutorService(params.getCodec(),
new CommandAsyncService(commandExecutor, params), this, params.getName(), ops);
commandExecutor.copy(params), this, params.getName(), ops);
}
@Override
@ -718,7 +718,7 @@ public final class Redisson implements RedissonClient {
&& params.getCodec() != connectionManager.getServiceManager().getCfg().getCodec()) {
executorId = executorId + ":" + params.getName();
}
return new RedissonRemoteService(params.getCodec(), params.getName(), new CommandAsyncService(commandExecutor, params), executorId);
return new RedissonRemoteService(params.getCodec(), params.getName(), commandExecutor.copy(params), executorId);
}
@Override
@ -735,7 +735,7 @@ public final class Redisson implements RedissonClient {
public <V> RSortedSet<V> getSortedSet(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonSortedSet<V>(params.getCodec(),
new CommandAsyncService(commandExecutor, params), params.getName(), this);
commandExecutor.copy(params), params.getName(), this);
}
@Override
@ -752,7 +752,7 @@ public final class Redisson implements RedissonClient {
public <V> RScoredSortedSet<V> getScoredSortedSet(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonScoredSortedSet<V>(params.getCodec(),
new CommandAsyncService(commandExecutor, params), params.getName(), this);
commandExecutor.copy(params), params.getName(), this);
}
@Override
@ -763,7 +763,7 @@ public final class Redisson implements RedissonClient {
@Override
public RLexSortedSet getLexSortedSet(CommonOptions options) {
CommonParams params = (CommonParams) options;
return new RedissonLexSortedSet(new CommandAsyncService(commandExecutor, params), params.getName(), this);
return new RedissonLexSortedSet(commandExecutor.copy(params), params.getName(), this);
}
@Override
@ -779,7 +779,7 @@ public final class Redisson implements RedissonClient {
@Override
public RShardedTopic getShardedTopic(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonShardedTopic(params.getCodec(), new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonShardedTopic(params.getCodec(), commandExecutor.copy(params), params.getName());
}
@Override
@ -795,7 +795,7 @@ public final class Redisson implements RedissonClient {
@Override
public RTopic getTopic(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonTopic(params.getCodec(), new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonTopic(params.getCodec(), commandExecutor.copy(params), params.getName());
}
@Override
@ -812,7 +812,7 @@ public final class Redisson implements RedissonClient {
public RReliableTopic getReliableTopic(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonReliableTopic(params.getCodec(),
new CommandAsyncService(commandExecutor, params), params.getName(), null);
commandExecutor.copy(params), params.getName(), null);
}
@Override
@ -828,7 +828,7 @@ public final class Redisson implements RedissonClient {
@Override
public RPatternTopic getPatternTopic(PatternTopicOptions options) {
PatternTopicParams params = (PatternTopicParams) options;
return new RedissonPatternTopic(params.getCodec(), new CommandAsyncService(commandExecutor, params), params.getPattern());
return new RedissonPatternTopic(params.getCodec(), commandExecutor.copy(params), params.getPattern());
}
@Override
@ -853,7 +853,7 @@ public final class Redisson implements RedissonClient {
public <V> RQueue<V> getQueue(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonQueue<V>(params.getCodec(),
new CommandAsyncService(commandExecutor, params), params.getName(), this);
commandExecutor.copy(params), params.getName(), this);
}
@Override
@ -876,7 +876,7 @@ public final class Redisson implements RedissonClient {
String remoteName = RedissonObject.suffixName(params.getName(), "remoteService");
RRemoteService service = getRemoteService(remoteName);
return new RedissonTransferQueue<V>(params.getCodec(),
new CommandAsyncService(commandExecutor, params), params.getName(), service);
commandExecutor.copy(params), params.getName(), service);
}
@Override
@ -893,7 +893,7 @@ public final class Redisson implements RedissonClient {
public <V> RRingBuffer<V> getRingBuffer(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonRingBuffer<V>(params.getCodec(),
new CommandAsyncService(commandExecutor, params), params.getName(), this);
commandExecutor.copy(params), params.getName(), this);
}
@Override
@ -910,7 +910,7 @@ public final class Redisson implements RedissonClient {
public <V> RBlockingQueue<V> getBlockingQueue(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonBlockingQueue<V>(params.getCodec(),
new CommandAsyncService(commandExecutor, params), params.getName(), this);
commandExecutor.copy(params), params.getName(), this);
}
@Override
@ -927,7 +927,7 @@ public final class Redisson implements RedissonClient {
public <V> RBoundedBlockingQueue<V> getBoundedBlockingQueue(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonBoundedBlockingQueue<V>(params.getCodec(),
new CommandAsyncService(commandExecutor, params), params.getName(), this);
commandExecutor.copy(params), params.getName(), this);
}
@Override
@ -944,7 +944,7 @@ public final class Redisson implements RedissonClient {
public <V> RDeque<V> getDeque(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonDeque<V>(params.getCodec(),
new CommandAsyncService(commandExecutor, params), params.getName(), this);
commandExecutor.copy(params), params.getName(), this);
}
@Override
@ -961,7 +961,7 @@ public final class Redisson implements RedissonClient {
public <V> RBlockingDeque<V> getBlockingDeque(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonBlockingDeque<V>(params.getCodec(),
new CommandAsyncService(commandExecutor, params), params.getName(), this);
commandExecutor.copy(params), params.getName(), this);
}
@Override
@ -972,7 +972,7 @@ public final class Redisson implements RedissonClient {
@Override
public RAtomicLong getAtomicLong(CommonOptions options) {
CommonParams params = (CommonParams) options;
return new RedissonAtomicLong(new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonAtomicLong(commandExecutor.copy(params), params.getName());
}
@Override
@ -983,7 +983,7 @@ public final class Redisson implements RedissonClient {
@Override
public RLongAdder getLongAdder(CommonOptions options) {
CommonParams params = (CommonParams) options;
return new RedissonLongAdder(new CommandAsyncService(commandExecutor, params), params.getName(), this);
return new RedissonLongAdder(commandExecutor.copy(params), params.getName(), this);
}
@Override
@ -994,7 +994,7 @@ public final class Redisson implements RedissonClient {
@Override
public RDoubleAdder getDoubleAdder(CommonOptions options) {
CommonParams params = (CommonParams) options;
return new RedissonDoubleAdder(new CommandAsyncService(commandExecutor, params), params.getName(), this);
return new RedissonDoubleAdder(commandExecutor.copy(params), params.getName(), this);
}
@Override
@ -1005,7 +1005,7 @@ public final class Redisson implements RedissonClient {
@Override
public RAtomicDouble getAtomicDouble(CommonOptions options) {
CommonParams params = (CommonParams) options;
return new RedissonAtomicDouble(new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonAtomicDouble(commandExecutor.copy(params), params.getName());
}
@Override
@ -1016,7 +1016,7 @@ public final class Redisson implements RedissonClient {
@Override
public RCountDownLatch getCountDownLatch(CommonOptions options) {
CommonParams params = (CommonParams) options;
return new RedissonCountDownLatch(new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonCountDownLatch(commandExecutor.copy(params), params.getName());
}
@Override
@ -1027,7 +1027,7 @@ public final class Redisson implements RedissonClient {
@Override
public RBitSet getBitSet(CommonOptions options) {
CommonParams params = (CommonParams) options;
return new RedissonBitSet(new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonBitSet(commandExecutor.copy(params), params.getName());
}
@Override
@ -1038,7 +1038,7 @@ public final class Redisson implements RedissonClient {
@Override
public RSemaphore getSemaphore(CommonOptions options) {
CommonParams params = (CommonParams) options;
return new RedissonSemaphore(new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonSemaphore(commandExecutor.copy(params), params.getName());
}
@Override
@ -1049,7 +1049,7 @@ public final class Redisson implements RedissonClient {
@Override
public RPermitExpirableSemaphore getPermitExpirableSemaphore(CommonOptions options) {
CommonParams params = (CommonParams) options;
return new RedissonPermitExpirableSemaphore(new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonPermitExpirableSemaphore(commandExecutor.copy(params), params.getName());
}
@Override
@ -1065,7 +1065,7 @@ public final class Redisson implements RedissonClient {
@Override
public <V> RBloomFilter<V> getBloomFilter(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonBloomFilter<V>(params.getCodec(), new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonBloomFilter<V>(params.getCodec(), commandExecutor.copy(params), params.getName());
}
@Override
@ -1076,7 +1076,7 @@ public final class Redisson implements RedissonClient {
@Override
public RIdGenerator getIdGenerator(CommonOptions options) {
CommonParams params = (CommonParams) options;
return new RedissonIdGenerator(new CommandAsyncService(commandExecutor, params), params.getName());
return new RedissonIdGenerator(commandExecutor.copy(params), params.getName());
}
@Override
@ -1087,7 +1087,7 @@ public final class Redisson implements RedissonClient {
@Override
public RKeys getKeys(KeysOptions options) {
KeysParams params = (KeysParams) options;
return new RedissonKeys(new CommandAsyncService(commandExecutor, params));
return new RedissonKeys(commandExecutor.copy(params));
}
@Override
@ -1113,7 +1113,7 @@ public final class Redisson implements RedissonClient {
@Override
public RLiveObjectService getLiveObjectService(LiveObjectOptions options) {
LiveObjectParams params = (LiveObjectParams) options;
return new RedissonLiveObjectService(liveObjectClassCache, new CommandAsyncService(commandExecutor, params));
return new RedissonLiveObjectService(liveObjectClassCache, commandExecutor.copy(params));
}
@Override
@ -1200,7 +1200,7 @@ public final class Redisson implements RedissonClient {
public <V> RPriorityQueue<V> getPriorityQueue(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonPriorityQueue<V>(params.getCodec(),
new CommandAsyncService(commandExecutor, params), params.getName(), this);
commandExecutor.copy(params), params.getName(), this);
}
@Override
@ -1217,7 +1217,7 @@ public final class Redisson implements RedissonClient {
public <V> RPriorityBlockingQueue<V> getPriorityBlockingQueue(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonPriorityBlockingQueue<V>(params.getCodec(),
new CommandAsyncService(commandExecutor, params), params.getName(), this);
commandExecutor.copy(params), params.getName(), this);
}
@Override
@ -1234,7 +1234,7 @@ public final class Redisson implements RedissonClient {
public <V> RPriorityBlockingDeque<V> getPriorityBlockingDeque(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonPriorityBlockingDeque<V>(params.getCodec(),
new CommandAsyncService(commandExecutor, params), params.getName(), this);
commandExecutor.copy(params), params.getName(), this);
}
@Override
@ -1251,7 +1251,7 @@ public final class Redisson implements RedissonClient {
public <V> RPriorityDeque<V> getPriorityDeque(PlainOptions options) {
PlainParams params = (PlainParams) options;
return new RedissonPriorityDeque<V>(params.getCodec(),
new CommandAsyncService(commandExecutor, params), params.getName(), this);
commandExecutor.copy(params), params.getName(), this);
}
@Override

@ -26,7 +26,6 @@ import org.redisson.client.codec.StringCodec;
import org.redisson.client.protocol.RedisCommands;
import org.redisson.client.protocol.pubsub.PubSubType;
import org.redisson.command.CommandAsyncExecutor;
import org.redisson.command.CommandAsyncService;
import org.redisson.config.Protocol;
import org.redisson.connection.ServiceManager;
import org.redisson.misc.CompletableFutureWrapper;
@ -455,7 +454,7 @@ public abstract class RedissonObject implements RObject {
throw new IllegalStateException("`protocol` config setting should be set to RESP3 value");
}
commandExecutor = new CommandAsyncService(commandExecutor, true);
commandExecutor = commandExecutor.copy(true);
PublishSubscribeService subscribeService = commandExecutor.getConnectionManager().getSubscribeService();
CompletableFuture<Integer> r = subscribeService.subscribe(getRawName(), StringCodec.INSTANCE,
commandExecutor, listener);
@ -532,7 +531,7 @@ public abstract class RedissonObject implements RObject {
CompletableFuture<Void> f = subscribeService.removeListenerAsync(PubSubType.UNSUBSCRIBE, new ChannelName("__redis__:invalidate"), listenerId);
f = f.whenComplete((r, e) -> {
if (!commandExecutor.isTrackChanges()) {
commandExecutor = new CommandAsyncService(commandExecutor, false);
commandExecutor = commandExecutor.copy(false);
}
});
return new CompletableFutureWrapper<>(f);

@ -15,13 +15,11 @@
*/
package org.redisson.cache;
import com.github.benmanes.caffeine.cache.Caffeine;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufUtil;
import io.netty.buffer.Unpooled;
import org.redisson.*;
import org.redisson.api.*;
import org.redisson.api.LocalCachedMapOptions.EvictionPolicy;
import org.redisson.api.LocalCachedMapOptions.ReconnectionStrategy;
import org.redisson.api.LocalCachedMapOptions.SyncStrategy;
import org.redisson.api.listener.BaseStatusListener;

@ -18,6 +18,7 @@ package org.redisson.command;
import io.netty.buffer.ByteBuf;
import org.redisson.SlotCallback;
import org.redisson.api.RFuture;
import org.redisson.api.options.ObjectParams;
import org.redisson.client.RedisClient;
import org.redisson.client.RedisException;
import org.redisson.client.codec.Codec;
@ -41,6 +42,10 @@ import java.util.function.Supplier;
*/
public interface CommandAsyncExecutor {
CommandAsyncExecutor copy(ObjectParams objectParams);
CommandAsyncExecutor copy(boolean trackChanges);
RedissonObjectBuilder getObjectBuilder();
ConnectionManager getConnectionManager();

@ -45,7 +45,10 @@ import java.io.IOException;
import java.time.Duration;
import java.util.*;
import java.util.Map.Entry;
import java.util.concurrent.*;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
@ -71,7 +74,12 @@ public class CommandAsyncService implements CommandAsyncExecutor {
private final int responseTimeout;
private final boolean trackChanges;
public CommandAsyncService(CommandAsyncExecutor executor, boolean trackChanges) {
@Override
public CommandAsyncExecutor copy(boolean trackChanges) {
return new CommandAsyncService(this, trackChanges);
}
protected CommandAsyncService(CommandAsyncExecutor executor, boolean trackChanges) {
CommandAsyncService service = (CommandAsyncService) executor;
this.codec = service.codec;
this.connectionManager = service.connectionManager;
@ -83,8 +91,12 @@ public class CommandAsyncService implements CommandAsyncExecutor {
this.trackChanges = trackChanges;
}
@Override
public CommandAsyncExecutor copy(ObjectParams objectParams) {
return new CommandAsyncService(this, objectParams);
}
public CommandAsyncService(CommandAsyncExecutor executor,
protected CommandAsyncService(CommandAsyncExecutor executor,
ObjectParams objectParams) {
CommandAsyncService service = (CommandAsyncService) executor;
this.codec = service.codec;

Loading…
Cancel
Save