From d00a1a7ae853dada9c4a5d165e38c63780020819 Mon Sep 17 00:00:00 2001 From: Nikita Date: Wed, 24 Aug 2016 14:33:56 +0300 Subject: [PATCH 01/34] netty Future interface methods moved to RFuture interface --- .../java/org/redisson/BaseRemoteService.java | 28 ++-- .../java/org/redisson/EvictionScheduler.java | 7 +- .../main/java/org/redisson/PubSubEntry.java | 4 +- .../main/java/org/redisson/RedisNodes.java | 17 +- .../main/java/org/redisson/RedissonBatch.java | 2 - .../org/redisson/RedissonBlockingQueue.java | 8 +- .../org/redisson/RedissonBloomFilter.java | 6 +- .../java/org/redisson/RedissonBuckets.java | 5 +- .../org/redisson/RedissonCountDownLatch.java | 10 +- .../redisson/RedissonCountDownLatchEntry.java | 9 +- .../org/redisson/RedissonExecutorService.java | 21 ++- .../java/org/redisson/RedissonFairLock.java | 6 +- .../main/java/org/redisson/RedissonKeys.java | 5 +- .../main/java/org/redisson/RedissonList.java | 35 +++- .../redisson/RedissonListMultimapValues.java | 17 +- .../main/java/org/redisson/RedissonLock.java | 45 +++-- .../java/org/redisson/RedissonLockEntry.java | 12 +- .../java/org/redisson/RedissonMapCache.java | 2 +- .../java/org/redisson/RedissonMultiLock.java | 31 ++-- .../java/org/redisson/RedissonMultimap.java | 4 +- .../main/java/org/redisson/RedissonNode.java | 6 +- .../java/org/redisson/RedissonObject.java | 7 +- .../org/redisson/RedissonPatternTopic.java | 5 +- .../java/org/redisson/RedissonReactive.java | 5 +- .../java/org/redisson/RedissonRedLock.java | 5 +- .../org/redisson/RedissonRemoteService.java | 13 +- .../org/redisson/RedissonScoredSortedSet.java | 6 +- .../java/org/redisson/RedissonSemaphore.java | 25 ++- .../java/org/redisson/RedissonSubList.java | 4 +- .../main/java/org/redisson/RedissonTopic.java | 4 +- .../main/java/org/redisson/api/RFuture.java | 77 ++++++++- .../java/org/redisson/api/RListAsync.java | 6 +- .../redisson/client/ReconnectListener.java | 4 +- .../java/org/redisson/client/RedisClient.java | 22 +-- .../org/redisson/client/RedisConnection.java | 24 +-- .../client/handler/CommandDecoder.java | 4 +- .../client/handler/ConnectionWatchdog.java | 5 +- .../client/protocol/BatchCommandData.java | 5 +- .../redisson/client/protocol/CommandData.java | 11 +- .../client/protocol/CommandsData.java | 8 +- .../cluster/ClusterConnectionManager.java | 59 +++---- .../org/redisson/command/AsyncDetails.java | 22 +-- .../command/CommandAsyncExecutor.java | 8 +- .../redisson/command/CommandAsyncService.java | 35 ++-- .../redisson/command/CommandBatchService.java | 20 +-- .../command/CommandReactiveExecutor.java | 5 +- .../command/CommandReactiveService.java | 29 ++-- .../redisson/command/CommandSyncExecutor.java | 5 +- .../redisson/command/CommandSyncService.java | 19 +-- .../connection/ClientConnectionsEntry.java | 21 +-- .../connection/ConnectionInitializer.java | 5 +- .../connection/ConnectionManager.java | 15 +- .../connection/DefaultConnectionListener.java | 5 +- .../ElasticacheConnectionManager.java | 4 +- .../connection/FutureConnectionListener.java | 9 +- .../MasterSlaveConnectionManager.java | 45 +++-- .../redisson/connection/MasterSlaveEntry.java | 28 ++-- .../redisson/connection/RedisClientEntry.java | 9 +- .../connection/SentinelConnectionManager.java | 19 ++- .../org/redisson/connection/SingleEntry.java | 17 +- .../balancer/LoadBalancerManager.java | 11 +- .../balancer/LoadBalancerManagerImpl.java | 17 +- .../connection/pool/ConnectionPool.java | 49 +++--- .../connection/pool/PubSubConnectionPool.java | 7 +- .../executor/ExecutorRemoteService.java | 8 +- .../executor/RedissonScheduledFuture.java | 4 +- .../org/redisson/executor/RemotePromise.java | 13 +- .../org/redisson/misc/PromiseDelegator.java | 27 ++- .../main/java/org/redisson/misc/RPromise.java | 71 +++++++- .../org/redisson/misc/RedissonFuture.java | 42 +++-- .../org/redisson/misc/RedissonPromise.java | 154 +++++++++++++++++- .../redisson/pubsub/CountDownLatchPubSub.java | 5 +- .../java/org/redisson/pubsub/LockPubSub.java | 5 +- .../org/redisson/pubsub/PublishSubscribe.java | 10 +- .../org/redisson/pubsub/SemaphorePubSub.java | 5 +- .../org/redisson/pubsub/TransferListener.java | 7 +- .../reactive/NettyFuturePublisher.java | 5 +- .../reactive/RedissonListReactive.java | 43 +---- .../reactive/RedissonObjectReactive.java | 4 +- .../RedissonPatternTopicReactive.java | 11 +- .../reactive/RedissonTopicReactive.java | 7 +- .../src/test/java/org/redisson/BaseTest.java | 3 +- .../java/org/redisson/RedisClientTest.java | 12 +- .../java/org/redisson/RedissonBatchTest.java | 13 +- .../redisson/RedissonBlockingQueueTest.java | 19 +-- .../RedissonBoundedBlockingQueueTest.java | 11 +- .../org/redisson/RedissonMapCacheTest.java | 5 +- .../java/org/redisson/RedissonMapTest.java | 7 +- .../redisson/RedissonRemoteServiceTest.java | 10 +- .../redisson/RedissonScoredSortedSetTest.java | 7 +- .../java/org/redisson/RedissonScriptTest.java | 10 +- .../java/org/redisson/RedissonSetTest.java | 6 +- .../org/redisson/RedissonSortedSetTest.java | 5 +- .../test/java/org/redisson/TimeoutTest.java | 18 +- 94 files changed, 899 insertions(+), 631 deletions(-) diff --git a/redisson/src/main/java/org/redisson/BaseRemoteService.java b/redisson/src/main/java/org/redisson/BaseRemoteService.java index a0fcf2d0a..60eae501d 100644 --- a/redisson/src/main/java/org/redisson/BaseRemoteService.java +++ b/redisson/src/main/java/org/redisson/BaseRemoteService.java @@ -33,6 +33,7 @@ import org.redisson.client.codec.LongCodec; import org.redisson.client.protocol.RedisCommands; import org.redisson.command.CommandExecutor; import org.redisson.executor.RemotePromise; +import org.redisson.misc.RPromise; import org.redisson.remote.RRemoteServiceResponse; import org.redisson.remote.RemoteServiceAck; import org.redisson.remote.RemoteServiceAckTimeoutException; @@ -47,7 +48,6 @@ import org.slf4j.LoggerFactory; import io.netty.buffer.ByteBufUtil; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; -import io.netty.util.concurrent.Promise; import io.netty.util.internal.ThreadLocalRandom; /** @@ -200,7 +200,7 @@ public abstract class BaseRemoteService { } if (optionsCopy.isAckExpected()) { - Future future = commandExecutor.evalWriteAsync(responseName, LongCodec.INSTANCE, + RFuture future = commandExecutor.evalWriteAsync(responseName, LongCodec.INSTANCE, RedisCommands.EVAL_BOOLEAN, "if redis.call('setnx', KEYS[1], 1) == 1 then " + "redis.call('pexpire', KEYS[1], ARGV[2]);" @@ -250,7 +250,7 @@ public abstract class BaseRemoteService { result.setRequestId(requestId); - Future addFuture = addAsync(requestQueue, request, result); + RFuture addFuture = addAsync(requestQueue, request, result); addFuture.addListener(new FutureListener() { @Override @@ -262,7 +262,7 @@ public abstract class BaseRemoteService { if (optionsCopy.isAckExpected()) { final RBlockingQueue responseQueue = redisson.getBlockingQueue(responseName, getCodec()); - Future ackFuture = responseQueue.pollAsync(optionsCopy.getAckTimeoutInMillis(), TimeUnit.MILLISECONDS); + RFuture ackFuture = responseQueue.pollAsync(optionsCopy.getAckTimeoutInMillis(), TimeUnit.MILLISECONDS); ackFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -273,7 +273,7 @@ public abstract class BaseRemoteService { RemoteServiceAck ack = future.getNow(); if (ack == null) { - Future ackFutureAttempt = + RFuture ackFutureAttempt = tryPollAckAgainAsync(optionsCopy, responseQueue, ackName); ackFutureAttempt.addListener(new FutureListener() { @@ -318,7 +318,7 @@ public abstract class BaseRemoteService { private void awaitResultAsync(final RemoteInvocationOptions optionsCopy, final RemotePromise result, final RemoteServiceRequest request, final String responseName, final String ackName) { - Future deleteFuture = redisson.getBucket(ackName).deleteAsync(); + RFuture deleteFuture = redisson.getBucket(ackName).deleteAsync(); deleteFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -340,7 +340,7 @@ public abstract class BaseRemoteService { } RBlockingQueue responseQueue = redisson.getBlockingQueue(responseName, getCodec()); - Future responseFuture = responseQueue + RFuture responseFuture = responseQueue .pollAsync(optionsCopy.getExecutionTimeoutInMillis(), TimeUnit.MILLISECONDS); responseFuture.addListener(new FutureListener() { @@ -448,7 +448,7 @@ public abstract class BaseRemoteService { private RemoteServiceAck tryPollAckAgain(RemoteInvocationOptions optionsCopy, RBlockingQueue responseQueue, String ackName) throws InterruptedException { - Future ackClientsFuture = commandExecutor.evalWriteAsync(ackName, LongCodec.INSTANCE, RedisCommands.EVAL_BOOLEAN, + RFuture ackClientsFuture = commandExecutor.evalWriteAsync(ackName, LongCodec.INSTANCE, RedisCommands.EVAL_BOOLEAN, "if redis.call('setnx', KEYS[1], 1) == 1 then " + "redis.call('pexpire', KEYS[1], ARGV[1]);" + "return 0;" @@ -464,11 +464,11 @@ public abstract class BaseRemoteService { return null; } - private Future tryPollAckAgainAsync(RemoteInvocationOptions optionsCopy, + private RFuture tryPollAckAgainAsync(RemoteInvocationOptions optionsCopy, final RBlockingQueue responseQueue, String ackName) throws InterruptedException { - final Promise promise = commandExecutor.getConnectionManager().newPromise(); - Future ackClientsFuture = commandExecutor.evalWriteAsync(ackName, LongCodec.INSTANCE, RedisCommands.EVAL_BOOLEAN, + final RPromise promise = commandExecutor.getConnectionManager().newPromise(); + RFuture ackClientsFuture = commandExecutor.evalWriteAsync(ackName, LongCodec.INSTANCE, RedisCommands.EVAL_BOOLEAN, "if redis.call('setnx', KEYS[1], 1) == 1 then " + "redis.call('pexpire', KEYS[1], ARGV[1]);" + "return 0;" @@ -485,7 +485,7 @@ public abstract class BaseRemoteService { } if (future.getNow()) { - Future pollFuture = responseQueue.pollAsync(); + RFuture pollFuture = responseQueue.pollAsync(); pollFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -512,9 +512,9 @@ public abstract class BaseRemoteService { return ByteBufUtil.hexDump(id); } - protected Future addAsync(RBlockingQueue requestQueue, RemoteServiceRequest request, + protected RFuture addAsync(RBlockingQueue requestQueue, RemoteServiceRequest request, RemotePromise result) { - Future future = requestQueue.addAsync(request); + RFuture future = requestQueue.addAsync(request); result.setAddFuture(future); return future; } diff --git a/redisson/src/main/java/org/redisson/EvictionScheduler.java b/redisson/src/main/java/org/redisson/EvictionScheduler.java index f72d72cb3..dee26b056 100644 --- a/redisson/src/main/java/org/redisson/EvictionScheduler.java +++ b/redisson/src/main/java/org/redisson/EvictionScheduler.java @@ -21,6 +21,7 @@ import java.util.LinkedList; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; +import org.redisson.api.RFuture; import org.redisson.client.codec.LongCodec; import org.redisson.client.protocol.RedisCommands; import org.redisson.command.CommandAsyncExecutor; @@ -70,7 +71,7 @@ public class EvictionScheduler { @Override public void run() { - Future future = cleanupExpiredEntires(name, timeoutSetName, maxIdleSetName, keysLimit, multimap); + RFuture future = cleanupExpiredEntires(name, timeoutSetName, maxIdleSetName, keysLimit, multimap); future.addListener(new FutureListener() { @Override @@ -169,7 +170,7 @@ public class EvictionScheduler { return; } - Future future = cleanupExpiredEntires(name, timeoutSetName, null, valuesAmountToClean, false); + RFuture future = cleanupExpiredEntires(name, timeoutSetName, null, valuesAmountToClean, false); future.addListener(new FutureListener() { @Override @@ -189,7 +190,7 @@ public class EvictionScheduler { }); } - private Future cleanupExpiredEntires(String name, String timeoutSetName, String maxIdleSetName, int keysLimit, boolean multimap) { + private RFuture cleanupExpiredEntires(String name, String timeoutSetName, String maxIdleSetName, int keysLimit, boolean multimap) { if (multimap) { return executor.evalWriteAsync(name, LongCodec.INSTANCE, RedisCommands.EVAL_INTEGER, "local expiredKeys = redis.call('zrangebyscore', KEYS[2], 0, ARGV[1], 'limit', 0, ARGV[2]); " diff --git a/redisson/src/main/java/org/redisson/PubSubEntry.java b/redisson/src/main/java/org/redisson/PubSubEntry.java index 5685d2a3d..a11915eb1 100644 --- a/redisson/src/main/java/org/redisson/PubSubEntry.java +++ b/redisson/src/main/java/org/redisson/PubSubEntry.java @@ -15,7 +15,7 @@ */ package org.redisson; -import io.netty.util.concurrent.Promise; +import org.redisson.misc.RPromise; public interface PubSubEntry { @@ -23,6 +23,6 @@ public interface PubSubEntry { int release(); - Promise getPromise(); + RPromise getPromise(); } diff --git a/redisson/src/main/java/org/redisson/RedisNodes.java b/redisson/src/main/java/org/redisson/RedisNodes.java index 057790008..c953b66c6 100644 --- a/redisson/src/main/java/org/redisson/RedisNodes.java +++ b/redisson/src/main/java/org/redisson/RedisNodes.java @@ -26,15 +26,16 @@ import java.util.concurrent.CountDownLatch; import org.redisson.api.Node; import org.redisson.api.NodeType; import org.redisson.api.NodesGroup; +import org.redisson.api.RFuture; import org.redisson.client.RedisConnection; import org.redisson.client.protocol.RedisCommands; import org.redisson.connection.ConnectionListener; import org.redisson.connection.ConnectionManager; import org.redisson.connection.RedisClientEntry; +import org.redisson.misc.RPromise; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; -import io.netty.util.concurrent.Promise; public class RedisNodes implements NodesGroup { @@ -65,21 +66,21 @@ public class RedisNodes implements NodesGroup { @Override public boolean pingAll() { List clients = new ArrayList(connectionManager.getClients()); - final Map> result = new ConcurrentHashMap>(clients.size()); + final Map> result = new ConcurrentHashMap>(clients.size()); final CountDownLatch latch = new CountDownLatch(clients.size()); for (RedisClientEntry entry : clients) { - Future f = entry.getClient().connectAsync(); + RFuture f = entry.getClient().connectAsync(); f.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { if (future.isSuccess()) { final RedisConnection c = future.getNow(); - Promise connectionFuture = connectionManager.newPromise(); + RPromise connectionFuture = connectionManager.newPromise(); connectionManager.getConnectListener().onConnect(connectionFuture, c, null, connectionManager.getConfig()); connectionFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { - Future r = c.async(connectionManager.getConfig().getPingTimeout(), RedisCommands.PING); + RFuture r = c.async(connectionManager.getConfig().getPingTimeout(), RedisCommands.PING); result.put(c, r); latch.countDown(); } @@ -99,7 +100,7 @@ public class RedisNodes implements NodesGroup { } if (System.currentTimeMillis() - time >= connectionManager.getConfig().getConnectTimeout()) { - for (Entry> entry : result.entrySet()) { + for (Entry> entry : result.entrySet()) { entry.getKey().closeAsync(); } return false; @@ -107,8 +108,8 @@ public class RedisNodes implements NodesGroup { time = System.currentTimeMillis(); boolean res = true; - for (Entry> entry : result.entrySet()) { - Future f = entry.getValue(); + for (Entry> entry : result.entrySet()) { + RFuture f = entry.getValue(); f.awaitUninterruptibly(); if (!"PONG".equals(f.getNow())) { res = false; diff --git a/redisson/src/main/java/org/redisson/RedissonBatch.java b/redisson/src/main/java/org/redisson/RedissonBatch.java index 554e9366b..dbc6698bf 100644 --- a/redisson/src/main/java/org/redisson/RedissonBatch.java +++ b/redisson/src/main/java/org/redisson/RedissonBatch.java @@ -45,8 +45,6 @@ import org.redisson.client.codec.Codec; import org.redisson.command.CommandBatchService; import org.redisson.connection.ConnectionManager; -import io.netty.util.concurrent.Future; - /** * * diff --git a/redisson/src/main/java/org/redisson/RedissonBlockingQueue.java b/redisson/src/main/java/org/redisson/RedissonBlockingQueue.java index 997ea8bf4..a11919f58 100644 --- a/redisson/src/main/java/org/redisson/RedissonBlockingQueue.java +++ b/redisson/src/main/java/org/redisson/RedissonBlockingQueue.java @@ -93,7 +93,7 @@ public class RedissonBlockingQueue extends RedissonQueue implements RBlock */ @Override public V take() throws InterruptedException { - Future res = takeAsync(); + RFuture res = takeAsync(); return res.await().getNow(); } @@ -108,7 +108,7 @@ public class RedissonBlockingQueue extends RedissonQueue implements RBlock */ @Override public V poll(long timeout, TimeUnit unit) throws InterruptedException { - Future res = pollAsync(timeout, unit); + RFuture res = pollAsync(timeout, unit); return res.await().getNow(); } @@ -118,7 +118,7 @@ public class RedissonBlockingQueue extends RedissonQueue implements RBlock */ @Override public V pollFromAny(long timeout, TimeUnit unit, String ... queueNames) throws InterruptedException { - Future res = pollFromAnyAsync(timeout, unit, queueNames); + RFuture res = pollFromAnyAsync(timeout, unit, queueNames); return res.await().getNow(); } @@ -144,7 +144,7 @@ public class RedissonBlockingQueue extends RedissonQueue implements RBlock @Override public V pollLastAndOfferFirstTo(String queueName, long timeout, TimeUnit unit) throws InterruptedException { - Future res = pollLastAndOfferFirstToAsync(queueName, timeout, unit); + RFuture res = pollLastAndOfferFirstToAsync(queueName, timeout, unit); return res.await().getNow(); } diff --git a/redisson/src/main/java/org/redisson/RedissonBloomFilter.java b/redisson/src/main/java/org/redisson/RedissonBloomFilter.java index 430889c7c..7609707e2 100644 --- a/redisson/src/main/java/org/redisson/RedissonBloomFilter.java +++ b/redisson/src/main/java/org/redisson/RedissonBloomFilter.java @@ -178,9 +178,9 @@ public class RedissonBloomFilter extends RedissonExpirable implements RBloomF @Override public int count() { CommandBatchService executorService = new CommandBatchService(commandExecutor.getConnectionManager()); - Future> configFuture = executorService.readAsync(getConfigName(), StringCodec.INSTANCE, + RFuture> configFuture = executorService.readAsync(getConfigName(), StringCodec.INSTANCE, new RedisCommand>("HGETALL", new ObjectMapReplayDecoder()), getConfigName()); - Future cardinalityFuture = executorService.readAsync(getName(), codec, RedisCommands.BITCOUNT, getName()); + RFuture cardinalityFuture = executorService.readAsync(getName(), codec, RedisCommands.BITCOUNT, getName()); executorService.execute(); readConfig(configFuture.getNow()); @@ -194,7 +194,7 @@ public class RedissonBloomFilter extends RedissonExpirable implements RBloomF } private void readConfig() { - Future> future = commandExecutor.readAsync(getConfigName(), StringCodec.INSTANCE, + RFuture> future = commandExecutor.readAsync(getConfigName(), StringCodec.INSTANCE, new RedisCommand>("HGETALL", new ObjectMapReplayDecoder()), getConfigName()); Map config = commandExecutor.get(future); diff --git a/redisson/src/main/java/org/redisson/RedissonBuckets.java b/redisson/src/main/java/org/redisson/RedissonBuckets.java index 492dc0c31..0abe91b5b 100644 --- a/redisson/src/main/java/org/redisson/RedissonBuckets.java +++ b/redisson/src/main/java/org/redisson/RedissonBuckets.java @@ -26,6 +26,7 @@ import java.util.Map.Entry; import org.redisson.api.RBucket; import org.redisson.api.RBuckets; +import org.redisson.api.RFuture; import org.redisson.client.codec.Codec; import org.redisson.client.codec.DelegateDecoderCodec; import org.redisson.client.protocol.RedisCommand; @@ -34,8 +35,6 @@ import org.redisson.client.protocol.RedisCommands; import org.redisson.command.CommandExecutor; import org.redisson.connection.decoder.MapGetAllDecoder; -import io.netty.util.concurrent.Future; - public class RedissonBuckets implements RBuckets { private final Codec codec; @@ -73,7 +72,7 @@ public class RedissonBuckets implements RBuckets { } RedisCommand> command = new RedisCommand>("MGET", new MapGetAllDecoder(Arrays.asList(keys), 0), ValueType.OBJECTS); - Future> future = commandExecutor.readAsync(keys[0], new DelegateDecoderCodec(codec), command, keys); + RFuture> future = commandExecutor.readAsync(keys[0], new DelegateDecoderCodec(codec), command, keys); return commandExecutor.get(future); } diff --git a/redisson/src/main/java/org/redisson/RedissonCountDownLatch.java b/redisson/src/main/java/org/redisson/RedissonCountDownLatch.java index 6eddbf960..b27c2dfb6 100644 --- a/redisson/src/main/java/org/redisson/RedissonCountDownLatch.java +++ b/redisson/src/main/java/org/redisson/RedissonCountDownLatch.java @@ -26,8 +26,6 @@ import org.redisson.client.protocol.RedisCommands; import org.redisson.command.CommandAsyncExecutor; import org.redisson.pubsub.CountDownLatchPubSub; -import io.netty.util.concurrent.Future; - /** * Distributed alternative to the {@link java.util.concurrent.CountDownLatch} * @@ -52,7 +50,7 @@ public class RedissonCountDownLatch extends RedissonObject implements RCountDown } public void await() throws InterruptedException { - Future promise = subscribe(); + RFuture promise = subscribe(); try { get(promise); @@ -70,7 +68,7 @@ public class RedissonCountDownLatch extends RedissonObject implements RCountDown @Override public boolean await(long time, TimeUnit unit) throws InterruptedException { - Future promise = subscribe(); + RFuture promise = subscribe(); try { if (!await(promise, time, unit)) { return false; @@ -102,11 +100,11 @@ public class RedissonCountDownLatch extends RedissonObject implements RCountDown return PUBSUB.getEntry(getEntryName()); } - private Future subscribe() { + private RFuture subscribe() { return PUBSUB.subscribe(getEntryName(), getChannelName(), commandExecutor.getConnectionManager()); } - private void unsubscribe(Future future) { + private void unsubscribe(RFuture future) { PUBSUB.unsubscribe(future.getNow(), getEntryName(), getChannelName(), commandExecutor.getConnectionManager()); } diff --git a/redisson/src/main/java/org/redisson/RedissonCountDownLatchEntry.java b/redisson/src/main/java/org/redisson/RedissonCountDownLatchEntry.java index 2e16d115f..e0f5be1f8 100644 --- a/redisson/src/main/java/org/redisson/RedissonCountDownLatchEntry.java +++ b/redisson/src/main/java/org/redisson/RedissonCountDownLatchEntry.java @@ -15,18 +15,17 @@ */ package org.redisson; +import org.redisson.misc.RPromise; import org.redisson.misc.ReclosableLatch; -import io.netty.util.concurrent.Promise; - public class RedissonCountDownLatchEntry implements PubSubEntry { private int counter; private final ReclosableLatch latch; - private final Promise promise; + private final RPromise promise; - public RedissonCountDownLatchEntry(Promise promise) { + public RedissonCountDownLatchEntry(RPromise promise) { super(); this.latch = new ReclosableLatch(); this.promise = promise; @@ -40,7 +39,7 @@ public class RedissonCountDownLatchEntry implements PubSubEntry getPromise() { + public RPromise getPromise() { return promise; } diff --git a/redisson/src/main/java/org/redisson/RedissonExecutorService.java b/redisson/src/main/java/org/redisson/RedissonExecutorService.java index c978c5695..92bda305c 100644 --- a/redisson/src/main/java/org/redisson/RedissonExecutorService.java +++ b/redisson/src/main/java/org/redisson/RedissonExecutorService.java @@ -67,7 +67,6 @@ import org.slf4j.LoggerFactory; import io.netty.util.Timeout; import io.netty.util.TimerTask; import io.netty.util.concurrent.FutureListener; -import io.netty.util.concurrent.Promise; import io.netty.util.internal.PlatformDependent; /** @@ -442,7 +441,7 @@ public class RedissonExecutorService implements RScheduledExecutorService { } private void execute(RemotePromise promise) { - io.netty.util.concurrent.Future addFuture = promise.getAddFuture(); + RFuture addFuture = promise.getAddFuture(); addFuture.syncUninterruptibly(); Boolean res = addFuture.getNow(); if (!res) { @@ -451,9 +450,9 @@ public class RedissonExecutorService implements RScheduledExecutorService { } @Override - public Future submit(Runnable task, final T result) { - final Promise resultFuture = connectionManager.newPromise(); - io.netty.util.concurrent.Future future = (io.netty.util.concurrent.Future) submit(task); + public RFuture submit(Runnable task, final T result) { + final RPromise resultFuture = connectionManager.newPromise(); + RFuture future = (RFuture) submit(task); future.addListener(new FutureListener() { @Override public void operationComplete(io.netty.util.concurrent.Future future) throws Exception { @@ -487,7 +486,7 @@ public class RedissonExecutorService implements RScheduledExecutorService { @Override public ScheduledFuture schedule(Runnable task, long delay, TimeUnit unit) { RedissonScheduledFuture future = (RedissonScheduledFuture) scheduleAsync(task, delay, unit); - execute((RemotePromise)future.getInnerFuture()); + execute((RemotePromise)future.getInnerPromise()); return future; } @@ -505,7 +504,7 @@ public class RedissonExecutorService implements RScheduledExecutorService { @Override public ScheduledFuture schedule(Callable task, long delay, TimeUnit unit) { RedissonScheduledFuture future = (RedissonScheduledFuture) scheduleAsync(task, delay, unit); - execute((RemotePromise)future.getInnerFuture()); + execute((RemotePromise)future.getInnerPromise()); return future; } @@ -523,7 +522,7 @@ public class RedissonExecutorService implements RScheduledExecutorService { @Override public ScheduledFuture scheduleAtFixedRate(Runnable task, long initialDelay, long period, TimeUnit unit) { RedissonScheduledFuture future = (RedissonScheduledFuture) scheduleAtFixedRateAsync(task, initialDelay, period, unit); - execute((RemotePromise)future.getInnerFuture()); + execute((RemotePromise)future.getInnerPromise()); return future; } @@ -541,7 +540,7 @@ public class RedissonExecutorService implements RScheduledExecutorService { @Override public RScheduledFuture schedule(Runnable task, CronSchedule cronSchedule) { RedissonScheduledFuture future = (RedissonScheduledFuture) scheduleAsync(task, cronSchedule); - execute((RemotePromise)future.getInnerFuture()); + execute((RemotePromise)future.getInnerPromise()); return future; } @@ -564,7 +563,7 @@ public class RedissonExecutorService implements RScheduledExecutorService { @Override public ScheduledFuture scheduleWithFixedDelay(Runnable task, long initialDelay, long delay, TimeUnit unit) { RedissonScheduledFuture future = (RedissonScheduledFuture) scheduleWithFixedDelayAsync(task, initialDelay, delay, unit); - execute((RemotePromise)future.getInnerFuture()); + execute((RemotePromise)future.getInnerPromise()); return future; } @@ -760,7 +759,7 @@ public class RedissonExecutorService implements RScheduledExecutorService { if (millis <= 0) { int remainFutures = tasks.size() - futures.size(); for (int i = 0; i < remainFutures; i++) { - Promise cancelledFuture = connectionManager.newPromise(); + RPromise cancelledFuture = connectionManager.newPromise(); cancelledFuture.cancel(true); futures.add(cancelledFuture); diff --git a/redisson/src/main/java/org/redisson/RedissonFairLock.java b/redisson/src/main/java/org/redisson/RedissonFairLock.java index 824deb5a7..14e23e2b7 100644 --- a/redisson/src/main/java/org/redisson/RedissonFairLock.java +++ b/redisson/src/main/java/org/redisson/RedissonFairLock.java @@ -28,8 +28,6 @@ import org.redisson.client.protocol.RedisStrictCommand; import org.redisson.command.CommandExecutor; import org.redisson.pubsub.LockPubSub; -import io.netty.util.concurrent.Future; - /** * Distributed implementation of {@link java.util.concurrent.locks.Lock} * Implements reentrant lock.
@@ -63,13 +61,13 @@ public class RedissonFairLock extends RedissonLock implements RLock { } @Override - protected Future subscribe(long threadId) { + protected RFuture subscribe(long threadId) { return PUBSUB.subscribe(getEntryName() + ":" + threadId, getChannelName() + ":" + getLockName(threadId), commandExecutor.getConnectionManager()); } @Override - protected void unsubscribe(Future future, long threadId) { + protected void unsubscribe(RFuture future, long threadId) { PUBSUB.unsubscribe(future.getNow(), getEntryName() + ":" + threadId, getChannelName() + ":" + getLockName(threadId), commandExecutor.getConnectionManager()); } diff --git a/redisson/src/main/java/org/redisson/RedissonKeys.java b/redisson/src/main/java/org/redisson/RedissonKeys.java index e726deffc..f19108910 100644 --- a/redisson/src/main/java/org/redisson/RedissonKeys.java +++ b/redisson/src/main/java/org/redisson/RedissonKeys.java @@ -43,7 +43,6 @@ import org.redisson.misc.RPromise; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; -import io.netty.util.concurrent.Promise; public class RedissonKeys implements RKeys { @@ -255,7 +254,7 @@ public class RedissonKeys implements RKeys { executorService.writeAsync(entry.getKey(), null, RedisCommands.DEL, key); } - Future> future = executorService.executeAsync(); + RFuture> future = executorService.executeAsync(); future.addListener(listener); } @@ -303,7 +302,7 @@ public class RedissonKeys implements RKeys { return commandExecutor.writeAllAsync(RedisCommands.FLUSHALL); } - private void checkExecution(final Promise result, final AtomicReference failed, + private void checkExecution(final RPromise result, final AtomicReference failed, final AtomicLong count, final AtomicLong executed) { if (executed.decrementAndGet() == 0) { if (failed.get() != null) { diff --git a/redisson/src/main/java/org/redisson/RedissonList.java b/redisson/src/main/java/org/redisson/RedissonList.java index af1f3a911..6020d77bf 100644 --- a/redisson/src/main/java/org/redisson/RedissonList.java +++ b/redisson/src/main/java/org/redisson/RedissonList.java @@ -339,19 +339,26 @@ public class RedissonList extends RedissonExpirable implements RList { @Override public V remove(int index) { + return remove((long) index); + } + + public V remove(long index) { + return get(removeAsync(index)); + } + + public RFuture removeAsync(long index) { if (index == 0) { - RFuture f = commandExecutor.writeAsync(getName(), codec, LPOP, getName()); - return get(f); + return commandExecutor.writeAsync(getName(), codec, LPOP, getName()); } - RFuture f = commandExecutor.evalWriteAsync(getName(), codec, EVAL_OBJECT, + return commandExecutor.evalWriteAsync(getName(), codec, EVAL_OBJECT, "local v = redis.call('lindex', KEYS[1], ARGV[1]); " + "redis.call('lset', KEYS[1], ARGV[1], 'DELETED_BY_REDISSON');" + "redis.call('lrem', KEYS[1], 1, 'DELETED_BY_REDISSON');" + "return v", Collections.singletonList(getName()), index); - return get(f); } + @Override public void fastRemove(int index) { @@ -359,7 +366,7 @@ public class RedissonList extends RedissonExpirable implements RList { } @Override - public RFuture fastRemoveAsync(int index) { + public RFuture fastRemoveAsync(long index) { return commandExecutor.evalWriteAsync(getName(), codec, RedisCommands.EVAL_VOID, "redis.call('lset', KEYS[1], ARGV[1], 'DELETED_BY_REDISSON');" + "redis.call('lrem', KEYS[1], 1, 'DELETED_BY_REDISSON');", @@ -376,7 +383,7 @@ public class RedissonList extends RedissonExpirable implements RList { return indexOfAsync(o, new BooleanNumberReplayConvertor(-1L)); } - private RFuture indexOfAsync(Object o, Convertor convertor) { + public RFuture indexOfAsync(Object o, Convertor convertor) { return commandExecutor.evalReadAsync(getName(), codec, new RedisCommand("EVAL", convertor, 4), "local key = KEYS[1] " + "local obj = ARGV[1] " + @@ -414,6 +421,20 @@ public class RedissonList extends RedissonExpirable implements RList { "return -1", Collections.singletonList(getName()), o); } + + public RFuture lastIndexOfAsync(Object o, Convertor convertor) { + return commandExecutor.evalReadAsync(getName(), codec, new RedisCommand("EVAL", convertor, 4), + "local key = KEYS[1] " + + "local obj = ARGV[1] " + + "local items = redis.call('lrange', key, 0, -1) " + + "for i = #items, 1, -1 do " + + "if items[i] == obj then " + + "return i - 1 " + + "end " + + "end " + + "return -1", + Collections.singletonList(getName()), o); + } @Override public void trim(int fromIndex, int toIndex) { @@ -421,7 +442,7 @@ public class RedissonList extends RedissonExpirable implements RList { } @Override - public RFuture trimAsync(int fromIndex, int toIndex) { + public RFuture trimAsync(long fromIndex, long toIndex) { return commandExecutor.writeAsync(getName(), codec, RedisCommands.LTRIM, getName(), fromIndex, toIndex); } diff --git a/redisson/src/main/java/org/redisson/RedissonListMultimapValues.java b/redisson/src/main/java/org/redisson/RedissonListMultimapValues.java index a84e1a197..1a43bba4d 100644 --- a/redisson/src/main/java/org/redisson/RedissonListMultimapValues.java +++ b/redisson/src/main/java/org/redisson/RedissonListMultimapValues.java @@ -432,27 +432,30 @@ public class RedissonListMultimapValues extends RedissonExpirable implements @Override public V remove(int index) { + return get(removeAsync(index)); + } + + @Override + public RFuture removeAsync(long index) { if (index == 0) { - RFuture f = commandExecutor.writeAsync(getName(), codec, LPOP, getName()); - return get(f); + return commandExecutor.writeAsync(getName(), codec, LPOP, getName()); } - RFuture f = commandExecutor.evalWriteAsync(getName(), codec, EVAL_OBJECT, + return commandExecutor.evalWriteAsync(getName(), codec, EVAL_OBJECT, "local v = redis.call('lindex', KEYS[1], ARGV[1]); " + "redis.call('lset', KEYS[1], ARGV[1], 'DELETED_BY_REDISSON');" + "redis.call('lrem', KEYS[1], 1, 'DELETED_BY_REDISSON');" + "return v", Collections.singletonList(getName()), index); - return get(f); } @Override public void fastRemove(int index) { - get(fastRemoveAsync(index)); + get(fastRemoveAsync((long)index)); } @Override - public RFuture fastRemoveAsync(int index) { + public RFuture fastRemoveAsync(long index) { return commandExecutor.evalWriteAsync(getName(), codec, RedisCommands.EVAL_VOID, "redis.call('lset', KEYS[1], ARGV[1], 'DELETED_BY_REDISSON');" + "redis.call('lrem', KEYS[1], 1, 'DELETED_BY_REDISSON');", @@ -528,7 +531,7 @@ public class RedissonListMultimapValues extends RedissonExpirable implements } @Override - public RFuture trimAsync(int fromIndex, int toIndex) { + public RFuture trimAsync(long fromIndex, long toIndex) { return commandExecutor.writeAsync(getName(), codec, RedisCommands.LTRIM, getName(), fromIndex, toIndex); } diff --git a/redisson/src/main/java/org/redisson/RedissonLock.java b/redisson/src/main/java/org/redisson/RedissonLock.java index 35b9badbd..48178e5b1 100644 --- a/redisson/src/main/java/org/redisson/RedissonLock.java +++ b/redisson/src/main/java/org/redisson/RedissonLock.java @@ -40,7 +40,6 @@ import io.netty.util.Timeout; import io.netty.util.TimerTask; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; -import io.netty.util.concurrent.Promise; import io.netty.util.internal.PlatformDependent; /** @@ -121,7 +120,7 @@ public class RedissonLock extends RedissonExpirable implements RLock { } long threadId = Thread.currentThread().getId(); - Future future = subscribe(threadId); + RFuture future = subscribe(threadId); get(future); try { @@ -171,11 +170,11 @@ public class RedissonLock extends RedissonExpirable implements RLock { return ttlRemainingFuture; } - private Future tryAcquireAsync(long leaseTime, TimeUnit unit, long threadId) { + private RFuture tryAcquireAsync(long leaseTime, TimeUnit unit, long threadId) { if (leaseTime != -1) { return tryLockInnerAsync(leaseTime, unit, threadId, RedisCommands.EVAL_LONG); } - Future ttlRemainingFuture = tryLockInnerAsync(LOCK_EXPIRATION_INTERVAL_SECONDS, TimeUnit.SECONDS, threadId, RedisCommands.EVAL_LONG); + RFuture ttlRemainingFuture = tryLockInnerAsync(LOCK_EXPIRATION_INTERVAL_SECONDS, TimeUnit.SECONDS, threadId, RedisCommands.EVAL_LONG); ttlRemainingFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -206,7 +205,7 @@ public class RedissonLock extends RedissonExpirable implements RLock { Timeout task = commandExecutor.getConnectionManager().newTimeout(new TimerTask() { @Override public void run(Timeout timeout) throws Exception { - Future future = expireAsync(internalLockLeaseTime, TimeUnit.MILLISECONDS); + RFuture future = expireAsync(internalLockLeaseTime, TimeUnit.MILLISECONDS); future.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -265,14 +264,14 @@ public class RedissonLock extends RedissonExpirable implements RLock { } final long threadId = Thread.currentThread().getId(); - Future future = subscribe(threadId); - if (!await(future, time, TimeUnit.MILLISECONDS)) { - if (!future.cancel(false)) { - future.addListener(new FutureListener() { + final RFuture subscribeFuture = subscribe(threadId); + if (!await(subscribeFuture, time, TimeUnit.MILLISECONDS)) { + if (!subscribeFuture.cancel(false)) { + subscribeFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { - if (future.isSuccess()) { - unsubscribe(future, threadId); + if (subscribeFuture.isSuccess()) { + unsubscribe(subscribeFuture, threadId); } } }); @@ -304,7 +303,7 @@ public class RedissonLock extends RedissonExpirable implements RLock { time -= elapsed; } } finally { - unsubscribe(future, threadId); + unsubscribe(subscribeFuture, threadId); } // return get(tryLockAsync(waitTime, leaseTime, unit)); } @@ -313,11 +312,11 @@ public class RedissonLock extends RedissonExpirable implements RLock { return PUBSUB.getEntry(getEntryName()); } - protected Future subscribe(long threadId) { + protected RFuture subscribe(long threadId) { return PUBSUB.subscribe(getEntryName(), getChannelName(), commandExecutor.getConnectionManager()); } - protected void unsubscribe(Future future, long threadId) { + protected void unsubscribe(RFuture future, long threadId) { PUBSUB.unsubscribe(future.getNow(), getEntryName(), getChannelName(), commandExecutor.getConnectionManager()); } @@ -421,7 +420,7 @@ public class RedissonLock extends RedissonExpirable implements RLock { public RFuture unlockAsync(final long threadId) { final RPromise result = newPromise(); - Future future = commandExecutor.evalWriteAsync(getName(), LongCodec.INSTANCE, RedisCommands.EVAL_BOOLEAN, + RFuture future = commandExecutor.evalWriteAsync(getName(), LongCodec.INSTANCE, RedisCommands.EVAL_BOOLEAN, "if (redis.call('exists', KEYS[1]) == 0) then " + "redis.call('publish', KEYS[2], ARGV[1]); " + "return 1; " + @@ -477,7 +476,7 @@ public class RedissonLock extends RedissonExpirable implements RLock { public RFuture lockAsync(final long leaseTime, final TimeUnit unit, final long currentThreadId) { final RPromise result = newPromise(); - Future ttlFuture = tryAcquireAsync(leaseTime, unit, currentThreadId); + RFuture ttlFuture = tryAcquireAsync(leaseTime, unit, currentThreadId); ttlFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -494,7 +493,7 @@ public class RedissonLock extends RedissonExpirable implements RLock { return; } - final Future subscribeFuture = subscribe(currentThreadId); + final RFuture subscribeFuture = subscribe(currentThreadId); subscribeFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -514,8 +513,8 @@ public class RedissonLock extends RedissonExpirable implements RLock { } private void lockAsync(final long leaseTime, final TimeUnit unit, - final Future subscribeFuture, final Promise result, final long currentThreadId) { - Future ttlFuture = tryAcquireAsync(leaseTime, unit, currentThreadId); + final RFuture subscribeFuture, final RPromise result, final long currentThreadId) { + RFuture ttlFuture = tryAcquireAsync(leaseTime, unit, currentThreadId); ttlFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -596,7 +595,7 @@ public class RedissonLock extends RedissonExpirable implements RLock { final RPromise result = newPromise(); final AtomicLong time = new AtomicLong(unit.toMillis(waitTime)); - Future ttlFuture = tryAcquireAsync(leaseTime, unit, currentThreadId); + RFuture ttlFuture = tryAcquireAsync(leaseTime, unit, currentThreadId); ttlFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -615,7 +614,7 @@ public class RedissonLock extends RedissonExpirable implements RLock { final long current = System.currentTimeMillis(); final AtomicReference futureRef = new AtomicReference(); - final Future subscribeFuture = subscribe(currentThreadId); + final RFuture subscribeFuture = subscribe(currentThreadId); subscribeFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -660,8 +659,8 @@ public class RedissonLock extends RedissonExpirable implements RLock { } private void tryLockAsync(final AtomicLong time, final long leaseTime, final TimeUnit unit, - final Future subscribeFuture, final Promise result, final long currentThreadId) { - Future ttlFuture = tryAcquireAsync(leaseTime, unit, currentThreadId); + final RFuture subscribeFuture, final RPromise result, final long currentThreadId) { + RFuture ttlFuture = tryAcquireAsync(leaseTime, unit, currentThreadId); ttlFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { diff --git a/redisson/src/main/java/org/redisson/RedissonLockEntry.java b/redisson/src/main/java/org/redisson/RedissonLockEntry.java index 0535dd2d5..ce563d714 100644 --- a/redisson/src/main/java/org/redisson/RedissonLockEntry.java +++ b/redisson/src/main/java/org/redisson/RedissonLockEntry.java @@ -15,20 +15,22 @@ */ package org.redisson; -import io.netty.util.concurrent.Promise; - import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.Semaphore; +import org.redisson.misc.RPromise; + +import io.netty.util.concurrent.Promise; + public class RedissonLockEntry implements PubSubEntry { private int counter; private final Semaphore latch; - private final Promise promise; + private final RPromise promise; private final ConcurrentLinkedQueue listeners = new ConcurrentLinkedQueue(); - public RedissonLockEntry(Promise promise) { + public RedissonLockEntry(RPromise promise) { super(); this.latch = new Semaphore(0); this.promise = promise; @@ -42,7 +44,7 @@ public class RedissonLockEntry implements PubSubEntry { return --counter; } - public Promise getPromise() { + public RPromise getPromise() { return promise; } diff --git a/redisson/src/main/java/org/redisson/RedissonMapCache.java b/redisson/src/main/java/org/redisson/RedissonMapCache.java index 9148fe3ff..9d644b789 100644 --- a/redisson/src/main/java/org/redisson/RedissonMapCache.java +++ b/redisson/src/main/java/org/redisson/RedissonMapCache.java @@ -534,7 +534,7 @@ public class RedissonMapCache extends RedissonMap implements RMapCac MapScanResult scanIterator(String name, InetSocketAddress client, long startPos) { RedisCommand> EVAL_HSCAN = new RedisCommand>("EVAL", new ListMultiDecoder(new LongMultiDecoder(), new ObjectMapDecoder(new ScanCodec(codec)), new ObjectListDecoder(codec), new MapCacheScanResultReplayDecoder()), ValueType.MAP); - Future> f = commandExecutor.evalReadAsync(client, getName(), codec, EVAL_HSCAN, + RFuture> f = commandExecutor.evalReadAsync(client, getName(), codec, EVAL_HSCAN, "local result = {}; " + "local idleKeys = {}; " + "local res = redis.call('hscan', KEYS[1], ARGV[2]); " diff --git a/redisson/src/main/java/org/redisson/RedissonMultiLock.java b/redisson/src/main/java/org/redisson/RedissonMultiLock.java index fdd6f9f6d..517d155ab 100644 --- a/redisson/src/main/java/org/redisson/RedissonMultiLock.java +++ b/redisson/src/main/java/org/redisson/RedissonMultiLock.java @@ -21,8 +21,8 @@ import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Queue; import java.util.Map.Entry; +import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -30,14 +30,15 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; +import org.redisson.api.RFuture; import org.redisson.api.RLock; +import org.redisson.misc.RPromise; import org.redisson.misc.RedissonFuture; import org.redisson.misc.RedissonPromise; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; import io.netty.util.concurrent.ImmediateEventExecutor; -import io.netty.util.concurrent.Promise; /** * Groups multiple independent locks and manages them as one lock. @@ -80,7 +81,7 @@ public class RedissonMultiLock implements Lock { } public void lockInterruptibly(long leaseTime, TimeUnit unit) throws InterruptedException { - Promise promise = ImmediateEventExecutor.INSTANCE.newPromise(); + RPromise promise = new RedissonPromise(ImmediateEventExecutor.INSTANCE.newPromise()); long currentThreadId = Thread.currentThread().getId(); Queue lockedLocks = new ConcurrentLinkedQueue(); @@ -94,7 +95,7 @@ public class RedissonMultiLock implements Lock { lockInterruptibly(-1, null); } - private void lock(final Promise promise, final long waitTime, final long leaseTime, final TimeUnit unit, + private void lock(final RPromise promise, final long waitTime, final long leaseTime, final TimeUnit unit, final List locks, final long currentThreadId, final Queue lockedLocks) throws InterruptedException { final AtomicInteger tryLockRequestsAmount = new AtomicInteger(); final Map, RLock> tryLockFutures = new HashMap, RLock>(locks.size()); @@ -145,7 +146,7 @@ public class RedissonMultiLock implements Lock { } } - protected void tryLockAgain(final Promise promise, final long waitTime, final long leaseTime, + protected void tryLockAgain(final RPromise promise, final long waitTime, final long leaseTime, final TimeUnit unit, final long currentThreadId, final Map, RLock> tryLockFutures) throws InterruptedException { lockedLocks.clear(); if (failed.get() != null) { @@ -176,9 +177,9 @@ public class RedissonMultiLock implements Lock { tryLockRequestsAmount.incrementAndGet(); Future future; if (waitTime > 0 || leaseTime > 0) { - future = ((RedissonLock)lock).tryLockAsync(waitTime, leaseTime, unit, currentThreadId); + future = ((RedissonPromise)((RedissonLock)lock).tryLockAsync(waitTime, leaseTime, unit, currentThreadId)).getInnerPromise(); } else { - future = ((RedissonLock)lock).tryLockAsync(currentThreadId); + future = ((RedissonPromise)(((RedissonLock)lock).tryLockAsync(currentThreadId))).getInnerPromise(); } if (future instanceof RedissonPromise) { @@ -197,7 +198,7 @@ public class RedissonMultiLock implements Lock { @Override public boolean tryLock() { - Map> tryLockFutures = new HashMap>(locks.size()); + Map> tryLockFutures = new HashMap>(locks.size()); for (RLock lock : locks) { tryLockFutures.put(lock, lock.tryLockAsync()); } @@ -205,10 +206,10 @@ public class RedissonMultiLock implements Lock { return sync(tryLockFutures); } - protected boolean sync(Map> tryLockFutures) { + protected boolean sync(Map> tryLockFutures) { List lockedLocks = new ArrayList(tryLockFutures.size()); RuntimeException latestException = null; - for (Entry> entry : tryLockFutures.entrySet()) { + for (Entry> entry : tryLockFutures.entrySet()) { try { if (entry.getValue().syncUninterruptibly().getNow()) { lockedLocks.add(entry.getKey()); @@ -230,12 +231,12 @@ public class RedissonMultiLock implements Lock { } protected void unlockInner(Collection locks) { - List> futures = new ArrayList>(locks.size()); + List> futures = new ArrayList>(locks.size()); for (RLock lock : locks) { futures.add(lock.unlockAsync()); } - for (Future unlockFuture : futures) { + for (RFuture unlockFuture : futures) { unlockFuture.awaitUninterruptibly(); } } @@ -246,7 +247,7 @@ public class RedissonMultiLock implements Lock { } public boolean tryLock(long waitTime, long leaseTime, TimeUnit unit) throws InterruptedException { - Map> tryLockFutures = new HashMap>(locks.size()); + Map> tryLockFutures = new HashMap>(locks.size()); for (RLock lock : locks) { tryLockFutures.put(lock, lock.tryLockAsync(waitTime, leaseTime, unit)); } @@ -257,13 +258,13 @@ public class RedissonMultiLock implements Lock { @Override public void unlock() { - List> futures = new ArrayList>(locks.size()); + List> futures = new ArrayList>(locks.size()); for (RLock lock : locks) { futures.add(lock.unlockAsync()); } - for (Future future : futures) { + for (RFuture future : futures) { future.syncUninterruptibly(); } } diff --git a/redisson/src/main/java/org/redisson/RedissonMultimap.java b/redisson/src/main/java/org/redisson/RedissonMultimap.java index 01ae4be7f..430e1c7dd 100644 --- a/redisson/src/main/java/org/redisson/RedissonMultimap.java +++ b/redisson/src/main/java/org/redisson/RedissonMultimap.java @@ -41,8 +41,6 @@ import org.redisson.client.protocol.decoder.ScanObjectEntry; import org.redisson.command.CommandAsyncExecutor; import org.redisson.misc.Hash; -import io.netty.util.concurrent.Future; - /** * @author Nikita Koksharov * @@ -250,7 +248,7 @@ public abstract class RedissonMultimap extends RedissonExpirable implement MapScanResult scanIterator(InetSocketAddress client, long startPos) { - Future> f = commandExecutor.readAsync(client, getName(), new ScanCodec(codec, StringCodec.INSTANCE), RedisCommands.HSCAN, getName(), startPos); + RFuture> f = commandExecutor.readAsync(client, getName(), new ScanCodec(codec, StringCodec.INSTANCE), RedisCommands.HSCAN, getName(), startPos); return get(f); } diff --git a/redisson/src/main/java/org/redisson/RedissonNode.java b/redisson/src/main/java/org/redisson/RedissonNode.java index 3c56062f0..7585c5f34 100644 --- a/redisson/src/main/java/org/redisson/RedissonNode.java +++ b/redisson/src/main/java/org/redisson/RedissonNode.java @@ -23,6 +23,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import org.redisson.api.RFuture; import org.redisson.api.RedissonClient; import org.redisson.client.RedisConnection; import org.redisson.config.RedissonNodeConfig; @@ -33,7 +34,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import io.netty.buffer.ByteBufUtil; -import io.netty.util.concurrent.Future; import io.netty.util.internal.ThreadLocalRandom; /** @@ -157,7 +157,7 @@ public class RedissonNode { private void retrieveAdresses() { ConnectionManager connectionManager = ((Redisson)redisson).getConnectionManager(); for (MasterSlaveEntry entry : connectionManager.getEntrySet()) { - Future readFuture = entry.connectionReadOp(); + RFuture readFuture = entry.connectionReadOp(); if (readFuture.awaitUninterruptibly((long)connectionManager.getConfig().getConnectTimeout()) && readFuture.isSuccess()) { RedisConnection connection = readFuture.getNow(); @@ -166,7 +166,7 @@ public class RedissonNode { localAddress = (InetSocketAddress) connection.getChannel().localAddress(); return; } - Future writeFuture = entry.connectionWriteOp(); + RFuture writeFuture = entry.connectionWriteOp(); if (writeFuture.awaitUninterruptibly((long)connectionManager.getConfig().getConnectTimeout()) && writeFuture.isSuccess()) { RedisConnection connection = writeFuture.getNow(); diff --git a/redisson/src/main/java/org/redisson/RedissonObject.java b/redisson/src/main/java/org/redisson/RedissonObject.java index 22775b396..6f756967d 100644 --- a/redisson/src/main/java/org/redisson/RedissonObject.java +++ b/redisson/src/main/java/org/redisson/RedissonObject.java @@ -25,9 +25,6 @@ import org.redisson.client.protocol.RedisCommands; import org.redisson.command.CommandAsyncExecutor; import org.redisson.misc.RPromise; -import io.netty.util.concurrent.Future; -import io.netty.util.concurrent.Promise; - /** * Base Redisson object * @@ -50,11 +47,11 @@ abstract class RedissonObject implements RObject { this(commandExecutor.getConnectionManager().getCodec(), commandExecutor, name); } - protected boolean await(Future future, long timeout, TimeUnit timeoutUnit) throws InterruptedException { + protected boolean await(RFuture future, long timeout, TimeUnit timeoutUnit) throws InterruptedException { return commandExecutor.await(future, timeout, timeoutUnit); } - protected V get(Future future) { + protected V get(RFuture future) { return commandExecutor.get(future); } diff --git a/redisson/src/main/java/org/redisson/RedissonPatternTopic.java b/redisson/src/main/java/org/redisson/RedissonPatternTopic.java index 52bc043c3..b99b8438d 100644 --- a/redisson/src/main/java/org/redisson/RedissonPatternTopic.java +++ b/redisson/src/main/java/org/redisson/RedissonPatternTopic.java @@ -18,6 +18,7 @@ package org.redisson; import java.util.Collections; import java.util.List; +import org.redisson.api.RFuture; import org.redisson.api.RPatternTopic; import org.redisson.api.listener.PatternMessageListener; import org.redisson.api.listener.PatternStatusListener; @@ -27,8 +28,6 @@ import org.redisson.command.CommandExecutor; import org.redisson.connection.PubSubConnectionEntry; import org.redisson.pubsub.AsyncSemaphore; -import io.netty.util.concurrent.Future; - /** * Distributed topic implementation. Messages are delivered to all message listeners across Redis cluster. * @@ -64,7 +63,7 @@ public class RedissonPatternTopic implements RPatternTopic { } private int addListener(RedisPubSubListener pubSubListener) { - Future future = commandExecutor.getConnectionManager().psubscribe(name, codec, pubSubListener); + RFuture future = commandExecutor.getConnectionManager().psubscribe(name, codec, pubSubListener); future.syncUninterruptibly(); return System.identityHashCode(pubSubListener); } diff --git a/redisson/src/main/java/org/redisson/RedissonReactive.java b/redisson/src/main/java/org/redisson/RedissonReactive.java index 1f8ec5d69..bc9f39c2e 100644 --- a/redisson/src/main/java/org/redisson/RedissonReactive.java +++ b/redisson/src/main/java/org/redisson/RedissonReactive.java @@ -28,6 +28,7 @@ import org.redisson.api.RBitSetReactive; import org.redisson.api.RBlockingQueueReactive; import org.redisson.api.RBucketReactive; import org.redisson.api.RDequeReactive; +import org.redisson.api.RFuture; import org.redisson.api.RHyperLogLogReactive; import org.redisson.api.RKeysReactive; import org.redisson.api.RLexSortedSetReactive; @@ -68,8 +69,6 @@ import org.redisson.reactive.RedissonSetCacheReactive; import org.redisson.reactive.RedissonSetReactive; import org.redisson.reactive.RedissonTopicReactive; -import io.netty.util.concurrent.Future; - /** * Main infrastructure class allows to get access * to all Redisson objects on top of Redis server. @@ -116,7 +115,7 @@ public class RedissonReactive implements RedissonReactiveClient { @Override public List> findBuckets(String pattern) { - Future> r = commandExecutor.readAllAsync(RedisCommands.KEYS, pattern); + RFuture> r = commandExecutor.readAllAsync(RedisCommands.KEYS, pattern); Collection keys = commandExecutor.get(r); List> buckets = new ArrayList>(keys.size()); diff --git a/redisson/src/main/java/org/redisson/RedissonRedLock.java b/redisson/src/main/java/org/redisson/RedissonRedLock.java index dd5a1a148..25b296d01 100644 --- a/redisson/src/main/java/org/redisson/RedissonRedLock.java +++ b/redisson/src/main/java/org/redisson/RedissonRedLock.java @@ -22,6 +22,7 @@ import java.util.Map.Entry; import java.util.Queue; import java.util.concurrent.atomic.AtomicReference; +import org.redisson.api.RFuture; import org.redisson.api.RLock; import io.netty.util.concurrent.Future; @@ -47,10 +48,10 @@ public class RedissonRedLock extends RedissonMultiLock { super(locks); } - protected boolean sync(Map> tryLockFutures) { + protected boolean sync(Map> tryLockFutures) { List lockedLocks = new ArrayList(tryLockFutures.size()); RuntimeException latestException = null; - for (Entry> entry : tryLockFutures.entrySet()) { + for (Entry> entry : tryLockFutures.entrySet()) { try { if (entry.getValue().syncUninterruptibly().getNow()) { lockedLocks.add(entry.getKey()); diff --git a/redisson/src/main/java/org/redisson/RedissonRemoteService.java b/redisson/src/main/java/org/redisson/RedissonRemoteService.java index 6607b5646..925958e24 100644 --- a/redisson/src/main/java/org/redisson/RedissonRemoteService.java +++ b/redisson/src/main/java/org/redisson/RedissonRemoteService.java @@ -26,6 +26,7 @@ import java.util.concurrent.atomic.AtomicReference; import org.redisson.api.RBatch; import org.redisson.api.RBlockingQueue; import org.redisson.api.RBlockingQueueAsync; +import org.redisson.api.RFuture; import org.redisson.api.RRemoteService; import org.redisson.api.RedissonClient; import org.redisson.client.codec.Codec; @@ -107,7 +108,7 @@ public class RedissonRemoteService extends BaseRemoteService implements RRemoteS private void subscribe(final Class remoteInterface, final RBlockingQueue requestQueue, final ExecutorService executor) { - Future take = requestQueue.takeAsync(); + RFuture take = requestQueue.takeAsync(); take.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -139,7 +140,7 @@ public class RedissonRemoteService extends BaseRemoteService implements RRemoteS // send the ack only if expected if (request.getOptions().isAckExpected()) { String ackName = getAckName(remoteInterface, request.getRequestId()); - Future ackClientsFuture = commandExecutor.evalWriteAsync(responseName, + RFuture ackClientsFuture = commandExecutor.evalWriteAsync(responseName, LongCodec.INSTANCE, RedisCommands.EVAL_BOOLEAN, "if redis.call('setnx', KEYS[1], 1) == 1 then " + "redis.call('pexpire', KEYS[1], ARGV[2]);" @@ -188,7 +189,7 @@ public class RedissonRemoteService extends BaseRemoteService implements RRemoteS if (executor != null) { RBlockingQueue cancelRequestQueue = redisson.getBlockingQueue(getCancelRequestQueueName(remoteInterface, request.getRequestId()), getCodec()); - final Future cancelRequestFuture = cancelRequestQueue.takeAsync(); + final RFuture cancelRequestFuture = cancelRequestQueue.takeAsync(); final AtomicReference responseHolder = new AtomicReference(); @@ -230,7 +231,7 @@ public class RedissonRemoteService extends BaseRemoteService implements RRemoteS private void invokeMethod(final Class remoteInterface, final RBlockingQueue requestQueue, final RemoteServiceRequest request, RemoteServiceMethod method, String responseName, final ExecutorService executor, - Future cancelRequestFuture, final AtomicReference responseHolder) { + RFuture cancelRequestFuture, final AtomicReference responseHolder) { try { if (method.getBean() instanceof RemoteParams) { ((RemoteParams)method.getBean()).setRequestId(request.getRequestId()); @@ -258,7 +259,7 @@ public class RedissonRemoteService extends BaseRemoteService implements RRemoteS timeout = request.getOptions().getExecutionTimeoutInMillis(); } - Future> clientsFuture = send(timeout, responseName, + RFuture> clientsFuture = send(timeout, responseName, responseHolder.get()); clientsFuture.addListener(new FutureListener>() { @Override @@ -282,7 +283,7 @@ public class RedissonRemoteService extends BaseRemoteService implements RRemoteS } } - private Future> send(long timeout, String responseName, T response) { + private RFuture> send(long timeout, String responseName, T response) { RBatch batch = redisson.createBatch(); RBlockingQueueAsync queue = batch.getBlockingQueue(responseName, getCodec()); queue.putAsync(response); diff --git a/redisson/src/main/java/org/redisson/RedissonScoredSortedSet.java b/redisson/src/main/java/org/redisson/RedissonScoredSortedSet.java index 8ba39aa9d..227d9bb51 100644 --- a/redisson/src/main/java/org/redisson/RedissonScoredSortedSet.java +++ b/redisson/src/main/java/org/redisson/RedissonScoredSortedSet.java @@ -33,15 +33,13 @@ import org.redisson.client.codec.Codec; import org.redisson.client.codec.ScoredCodec; import org.redisson.client.codec.StringCodec; import org.redisson.client.protocol.RedisCommand; +import org.redisson.client.protocol.RedisCommand.ValueType; import org.redisson.client.protocol.RedisCommands; import org.redisson.client.protocol.ScoredEntry; -import org.redisson.client.protocol.RedisCommand.ValueType; import org.redisson.client.protocol.convertor.BooleanReplayConvertor; import org.redisson.client.protocol.decoder.ListScanResult; import org.redisson.command.CommandAsyncExecutor; -import io.netty.util.concurrent.Future; - public class RedissonScoredSortedSet extends RedissonExpirable implements RScoredSortedSet { public RedissonScoredSortedSet(CommandAsyncExecutor commandExecutor, String name) { @@ -253,7 +251,7 @@ public class RedissonScoredSortedSet extends RedissonExpirable implements RSc } private ListScanResult scanIterator(InetSocketAddress client, long startPos) { - Future> f = commandExecutor.readAsync(client, getName(), codec, RedisCommands.ZSCAN, getName(), startPos); + RFuture> f = commandExecutor.readAsync(client, getName(), codec, RedisCommands.ZSCAN, getName(), startPos); return get(f); } diff --git a/redisson/src/main/java/org/redisson/RedissonSemaphore.java b/redisson/src/main/java/org/redisson/RedissonSemaphore.java index 8ac547166..000eff669 100644 --- a/redisson/src/main/java/org/redisson/RedissonSemaphore.java +++ b/redisson/src/main/java/org/redisson/RedissonSemaphore.java @@ -35,7 +35,6 @@ import io.netty.util.Timeout; import io.netty.util.TimerTask; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; -import io.netty.util.concurrent.Promise; /** * Distributed and concurrent implementation of {@link java.util.concurrent.Semaphore}. @@ -79,7 +78,7 @@ public class RedissonSemaphore extends RedissonExpirable implements RSemaphore { return; } - Future future = subscribe(); + RFuture future = subscribe(); get(future); try { while (true) { @@ -103,7 +102,7 @@ public class RedissonSemaphore extends RedissonExpirable implements RSemaphore { @Override public RFuture acquireAsync(final int permits) { final RPromise result = newPromise(); - Future tryAcquireFuture = tryAcquireAsync(permits); + RFuture tryAcquireFuture = tryAcquireAsync(permits); tryAcquireFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -117,7 +116,7 @@ public class RedissonSemaphore extends RedissonExpirable implements RSemaphore { return; } - final Future subscribeFuture = subscribe(); + final RFuture subscribeFuture = subscribe(); subscribeFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -135,8 +134,8 @@ public class RedissonSemaphore extends RedissonExpirable implements RSemaphore { return result; } - private void tryAcquireAsync(final AtomicLong time, final int permits, final Future subscribeFuture, final Promise result) { - Future tryAcquireFuture = tryAcquireAsync(permits); + private void tryAcquireAsync(final AtomicLong time, final int permits, final RFuture subscribeFuture, final RPromise result) { + RFuture tryAcquireFuture = tryAcquireAsync(permits); tryAcquireFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -206,8 +205,8 @@ public class RedissonSemaphore extends RedissonExpirable implements RSemaphore { } - private void acquireAsync(final int permits, final Future subscribeFuture, final Promise result) { - Future tryAcquireFuture = tryAcquireAsync(permits); + private void acquireAsync(final int permits, final RFuture subscribeFuture, final RPromise result) { + RFuture tryAcquireFuture = tryAcquireAsync(permits); tryAcquireFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -284,7 +283,7 @@ public class RedissonSemaphore extends RedissonExpirable implements RSemaphore { } long time = unit.toMillis(waitTime); - Future future = subscribe(); + RFuture future = subscribe(); if (!await(future, time, TimeUnit.MILLISECONDS)) { return false; } @@ -317,7 +316,7 @@ public class RedissonSemaphore extends RedissonExpirable implements RSemaphore { public RFuture tryAcquireAsync(final int permits, long waitTime, TimeUnit unit) { final RPromise result = newPromise(); final AtomicLong time = new AtomicLong(unit.toMillis(waitTime)); - Future tryAcquireFuture = tryAcquireAsync(permits); + RFuture tryAcquireFuture = tryAcquireAsync(permits); tryAcquireFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -333,7 +332,7 @@ public class RedissonSemaphore extends RedissonExpirable implements RSemaphore { final long current = System.currentTimeMillis(); final AtomicReference futureRef = new AtomicReference(); - final Future subscribeFuture = subscribe(); + final RFuture subscribeFuture = subscribe(); subscribeFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -381,11 +380,11 @@ public class RedissonSemaphore extends RedissonExpirable implements RSemaphore { return semaphorePubSub.getEntry(getName()); } - private Future subscribe() { + private RFuture subscribe() { return semaphorePubSub.subscribe(getName(), getChannelName(), commandExecutor.getConnectionManager()); } - private void unsubscribe(Future future) { + private void unsubscribe(RFuture future) { semaphorePubSub.unsubscribe(future.getNow(), getName(), getChannelName(), commandExecutor.getConnectionManager()); } diff --git a/redisson/src/main/java/org/redisson/RedissonSubList.java b/redisson/src/main/java/org/redisson/RedissonSubList.java index 9b026de17..27ac6b8d2 100644 --- a/redisson/src/main/java/org/redisson/RedissonSubList.java +++ b/redisson/src/main/java/org/redisson/RedissonSubList.java @@ -323,7 +323,7 @@ public class RedissonSubList extends RedissonList implements RList { return get(f); } - private RFuture indexOfAsync(Object o, Convertor convertor) { + public RFuture indexOfAsync(Object o, Convertor convertor) { return commandExecutor.evalReadAsync(getName(), codec, new RedisCommand("EVAL", convertor, 4), "local items = redis.call('lrange', KEYS[1], tonumber(ARGV[2]), tonumber(ARGV[3])) " + "for i=1,#items do " + @@ -470,7 +470,7 @@ public class RedissonSubList extends RedissonList implements RList { } @Override - public RFuture trimAsync(int fromIndex, int toIndex) { + public RFuture trimAsync(long fromIndex, long toIndex) { if (fromIndex < this.fromIndex || toIndex >= this.toIndex.get()) { throw new IndexOutOfBoundsException("fromIndex: " + fromIndex + " toIndex: " + toIndex); } diff --git a/redisson/src/main/java/org/redisson/RedissonTopic.java b/redisson/src/main/java/org/redisson/RedissonTopic.java index 5e290bce4..5bd324cf4 100644 --- a/redisson/src/main/java/org/redisson/RedissonTopic.java +++ b/redisson/src/main/java/org/redisson/RedissonTopic.java @@ -29,8 +29,6 @@ import org.redisson.command.CommandAsyncExecutor; import org.redisson.connection.PubSubConnectionEntry; import org.redisson.pubsub.AsyncSemaphore; -import io.netty.util.concurrent.Future; - /** * Distributed topic implementation. Messages are delivered to all message listeners across Redis cluster. * @@ -80,7 +78,7 @@ public class RedissonTopic implements RTopic { } private int addListener(RedisPubSubListener pubSubListener) { - Future future = commandExecutor.getConnectionManager().subscribe(codec, name, pubSubListener); + RFuture future = commandExecutor.getConnectionManager().subscribe(codec, name, pubSubListener); future.syncUninterruptibly(); return System.identityHashCode(pubSubListener); } diff --git a/redisson/src/main/java/org/redisson/api/RFuture.java b/redisson/src/main/java/org/redisson/api/RFuture.java index d433def85..bbe9c6005 100644 --- a/redisson/src/main/java/org/redisson/api/RFuture.java +++ b/redisson/src/main/java/org/redisson/api/RFuture.java @@ -15,7 +15,9 @@ */ package org.redisson.api; -import io.netty.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +import io.netty.util.concurrent.FutureListener; /** * Represents the result of an asynchronous computation @@ -24,6 +26,77 @@ import io.netty.util.concurrent.Future; * * @param */ -public interface RFuture extends Future { +public interface RFuture extends java.util.concurrent.Future { + + /** + * Returns {@code true} if and only if the I/O operation was completed + * successfully. + */ + boolean isSuccess(); + + /** + * Returns the cause of the failed I/O operation if the I/O operation has + * failed. + * + * @return the cause of the failure. + * {@code null} if succeeded or this future is not + * completed yet. + */ + Throwable cause(); + + /** + * Return the result without blocking. If the future is not done yet this will return {@code null}. + * + * As it is possible that a {@code null} value is used to mark the future as successful you also need to check + * if the future is really done with {@link #isDone()} and not relay on the returned {@code null} value. + */ + V getNow(); + + /** + * Waits for this future to be completed within the + * specified time limit. + * + * @return {@code true} if and only if the future was completed within + * the specified time limit + * + * @throws InterruptedException + * if the current thread was interrupted + */ + boolean await(long timeout, TimeUnit unit) throws InterruptedException; + + /** + * Waits for this future to be completed within the + * specified time limit. + * + * @return {@code true} if and only if the future was completed within + * the specified time limit + * + * @throws InterruptedException + * if the current thread was interrupted + */ + boolean await(long timeoutMillis) throws InterruptedException; + + RFuture addListener(FutureListener listener); + + RFuture addListeners(FutureListener... listeners); + + RFuture removeListener(FutureListener listener); + + RFuture removeListeners(FutureListener... listeners); + + boolean isCancellable(); + + RFuture sync() throws InterruptedException; + + RFuture syncUninterruptibly(); + + RFuture await() throws InterruptedException; + + RFuture awaitUninterruptibly(); + + boolean awaitUninterruptibly(long timeout, TimeUnit unit); + + boolean awaitUninterruptibly(long timeoutMillis); + } diff --git a/redisson/src/main/java/org/redisson/api/RListAsync.java b/redisson/src/main/java/org/redisson/api/RListAsync.java index a93101ac4..df8de1f03 100644 --- a/redisson/src/main/java/org/redisson/api/RListAsync.java +++ b/redisson/src/main/java/org/redisson/api/RListAsync.java @@ -81,8 +81,10 @@ public interface RListAsync extends RCollectionAsync, RandomAccess { * @param toIndex * @return */ - RFuture trimAsync(int fromIndex, int toIndex); + RFuture trimAsync(long fromIndex, long toIndex); - RFuture fastRemoveAsync(int index); + RFuture fastRemoveAsync(long index); + + RFuture removeAsync(long index); } diff --git a/redisson/src/main/java/org/redisson/client/ReconnectListener.java b/redisson/src/main/java/org/redisson/client/ReconnectListener.java index 9248dfe08..3d4e6aebf 100644 --- a/redisson/src/main/java/org/redisson/client/ReconnectListener.java +++ b/redisson/src/main/java/org/redisson/client/ReconnectListener.java @@ -15,10 +15,10 @@ */ package org.redisson.client; -import io.netty.util.concurrent.Promise; +import org.redisson.misc.RPromise; public interface ReconnectListener { - void onReconnect(RedisConnection redisConnection, Promise connectionFuture) throws RedisException; + void onReconnect(RedisConnection redisConnection, RPromise connectionFuture) throws RedisException; } diff --git a/redisson/src/main/java/org/redisson/client/RedisClient.java b/redisson/src/main/java/org/redisson/client/RedisClient.java index 7dd23ae9f..cfd7f7524 100644 --- a/redisson/src/main/java/org/redisson/client/RedisClient.java +++ b/redisson/src/main/java/org/redisson/client/RedisClient.java @@ -19,12 +19,15 @@ import java.net.InetSocketAddress; import java.net.URI; import java.util.Map; +import org.redisson.api.RFuture; import org.redisson.client.handler.CommandBatchEncoder; import org.redisson.client.handler.CommandDecoder; import org.redisson.client.handler.CommandEncoder; import org.redisson.client.handler.CommandsQueue; import org.redisson.client.handler.ConnectionWatchdog; import org.redisson.client.protocol.RedisCommands; +import org.redisson.misc.RPromise; +import org.redisson.misc.RedissonPromise; import org.redisson.misc.URIBuilder; import io.netty.bootstrap.Bootstrap; @@ -43,10 +46,9 @@ import io.netty.channel.socket.nio.NioSocketChannel; import io.netty.util.HashedWheelTimer; import io.netty.util.Timer; import io.netty.util.concurrent.Future; -import io.netty.util.concurrent.GenericFutureListener; +import io.netty.util.concurrent.FutureListener; import io.netty.util.concurrent.GlobalEventExecutor; import io.netty.util.concurrent.ImmediateEventExecutor; -import io.netty.util.concurrent.Promise; /** * Low-level Redis client @@ -128,8 +130,8 @@ public class RedisClient { } } - public Future connectAsync() { - final Promise f = ImmediateEventExecutor.INSTANCE.newPromise(); + public RFuture connectAsync() { + final RPromise f = new RedissonPromise(ImmediateEventExecutor.INSTANCE.newPromise()); ChannelFuture channelFuture = bootstrap.connect(); channelFuture.addListener(new ChannelFutureListener() { @Override @@ -155,8 +157,8 @@ public class RedisClient { } } - public Future connectPubSubAsync() { - final Promise f = ImmediateEventExecutor.INSTANCE.newPromise(); + public RFuture connectPubSubAsync() { + final RPromise f = new RedissonPromise(ImmediateEventExecutor.INSTANCE.newPromise()); ChannelFuture channelFuture = bootstrap.connect(); channelFuture.addListener(new ChannelFutureListener() { @Override @@ -206,12 +208,12 @@ public class RedisClient { * @return A future for a map extracted from each response line splitting by * ':' symbol */ - public Future> serverInfoAsync() { + public RFuture> serverInfoAsync() { final RedisConnection connection = connect(); - Promise> async = (Promise) connection.async(RedisCommands.SERVER_INFO); - async.addListener(new GenericFutureListener>>() { + RFuture> async = connection.async(RedisCommands.SERVER_INFO); + async.addListener(new FutureListener>() { @Override - public void operationComplete(Promise> future) throws Exception { + public void operationComplete(Future> future) throws Exception { connection.closeAsync(); } }); diff --git a/redisson/src/main/java/org/redisson/client/RedisConnection.java b/redisson/src/main/java/org/redisson/client/RedisConnection.java index 07cc2fab8..a4845c4c7 100644 --- a/redisson/src/main/java/org/redisson/client/RedisConnection.java +++ b/redisson/src/main/java/org/redisson/client/RedisConnection.java @@ -18,6 +18,7 @@ package org.redisson.client; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import org.redisson.api.RFuture; import org.redisson.client.codec.Codec; import org.redisson.client.handler.CommandsQueue; import org.redisson.client.protocol.CommandData; @@ -26,6 +27,9 @@ import org.redisson.client.protocol.QueueCommand; import org.redisson.client.protocol.RedisCommand; import org.redisson.client.protocol.RedisCommands; import org.redisson.client.protocol.RedisStrictCommand; +import org.redisson.misc.RPromise; +import org.redisson.misc.RedissonFuture; +import org.redisson.misc.RedissonPromise; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; @@ -49,7 +53,7 @@ public class RedisConnection implements RedisCommands { private ReconnectListener reconnectListener; private long lastUsageTime; - private final Future acquireFuture = ImmediateEventExecutor.INSTANCE.newSucceededFuture(this); + private final RFuture acquireFuture = new RedissonFuture(ImmediateEventExecutor.INSTANCE.newSucceededFuture(this)); public RedisConnection(RedisClient redisClient, Channel channel) { super(); @@ -109,7 +113,7 @@ public class RedisConnection implements RedisCommands { return redisClient; } - public R await(Future future) { + public R await(RFuture future) { final CountDownLatch l = new CountDownLatch(1); future.addListener(new FutureListener() { @Override @@ -151,25 +155,25 @@ public class RedisConnection implements RedisCommands { } public R sync(Codec encoder, RedisCommand command, Object ... params) { - Promise promise = ImmediateEventExecutor.INSTANCE.newPromise(); + RPromise promise = new RedissonPromise(ImmediateEventExecutor.INSTANCE.newPromise()); send(new CommandData(promise, encoder, command, params)); return await(promise); } - public Future async(RedisCommand command, Object ... params) { + public RFuture async(RedisCommand command, Object ... params) { return async(null, command, params); } - public Future async(long timeout, RedisCommand command, Object ... params) { + public RFuture async(long timeout, RedisCommand command, Object ... params) { return async(null, command, params); } - public Future async(Codec encoder, RedisCommand command, Object ... params) { + public RFuture async(Codec encoder, RedisCommand command, Object ... params) { return async(-1, encoder, command, params); } - public Future async(long timeout, Codec encoder, RedisCommand command, Object ... params) { - final Promise promise = ImmediateEventExecutor.INSTANCE.newPromise(); + public RFuture async(long timeout, Codec encoder, RedisCommand command, Object ... params) { + final RPromise promise = new RedissonPromise(ImmediateEventExecutor.INSTANCE.newPromise()); if (timeout == -1) { timeout = redisClient.getCommandTimeout(); } @@ -193,7 +197,7 @@ public class RedisConnection implements RedisCommands { } public CommandData create(Codec encoder, RedisCommand command, Object ... params) { - Promise promise = ImmediateEventExecutor.INSTANCE.newPromise(); + RPromise promise = new RedissonPromise(ImmediateEventExecutor.INSTANCE.newPromise()); return new CommandData(promise, encoder, command, params); } @@ -237,7 +241,7 @@ public class RedisConnection implements RedisCommands { return getClass().getSimpleName() + "@" + System.identityHashCode(this) + " [redisClient=" + redisClient + ", channel=" + channel + "]"; } - public Future getAcquireFuture() { + public RFuture getAcquireFuture() { return acquireFuture; } diff --git a/redisson/src/main/java/org/redisson/client/handler/CommandDecoder.java b/redisson/src/main/java/org/redisson/client/handler/CommandDecoder.java index 6058b912d..30ac25c3f 100644 --- a/redisson/src/main/java/org/redisson/client/handler/CommandDecoder.java +++ b/redisson/src/main/java/org/redisson/client/handler/CommandDecoder.java @@ -43,6 +43,7 @@ import org.redisson.client.protocol.pubsub.Message; import org.redisson.client.protocol.pubsub.PubSubMessage; import org.redisson.client.protocol.pubsub.PubSubPatternMessage; import org.redisson.client.protocol.pubsub.PubSubStatusMessage; +import org.redisson.misc.RPromise; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,7 +52,6 @@ import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; import io.netty.handler.codec.ReplayingDecoder; import io.netty.util.CharsetUtil; -import io.netty.util.concurrent.Promise; import io.netty.util.internal.PlatformDependent; /** @@ -192,7 +192,7 @@ public class CommandDecoder extends ReplayingDecoder { } if (i == commandBatch.getCommands().size()) { - Promise promise = commandBatch.getPromise(); + RPromise promise = commandBatch.getPromise(); if (error != null) { if (!promise.tryFailure(error) && promise.cause() instanceof RedisTimeoutException) { log.warn("response has been skipped due to timeout! channel: {}, command: {}", ctx.channel(), data); diff --git a/redisson/src/main/java/org/redisson/client/handler/ConnectionWatchdog.java b/redisson/src/main/java/org/redisson/client/handler/ConnectionWatchdog.java index af90d62fa..c15bce8aa 100644 --- a/redisson/src/main/java/org/redisson/client/handler/ConnectionWatchdog.java +++ b/redisson/src/main/java/org/redisson/client/handler/ConnectionWatchdog.java @@ -23,6 +23,8 @@ import org.redisson.client.RedisException; import org.redisson.client.RedisPubSubConnection; import org.redisson.client.codec.Codec; import org.redisson.client.protocol.CommandData; +import org.redisson.misc.RPromise; +import org.redisson.misc.RedissonPromise; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,7 +41,6 @@ import io.netty.util.TimerTask; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; import io.netty.util.concurrent.ImmediateEventExecutor; -import io.netty.util.concurrent.Promise; public class ConnectionWatchdog extends ChannelInboundHandlerAdapter { @@ -125,7 +126,7 @@ public class ConnectionWatchdog extends ChannelInboundHandlerAdapter { if (connection.getReconnectListener() != null) { // new connection used only for channel init RedisConnection rc = new RedisConnection(connection.getRedisClient(), channel); - Promise connectionFuture = ImmediateEventExecutor.INSTANCE.newPromise(); + RPromise connectionFuture = new RedissonPromise(ImmediateEventExecutor.INSTANCE.newPromise()); connection.getReconnectListener().onReconnect(rc, connectionFuture); connectionFuture.addListener(new FutureListener() { @Override diff --git a/redisson/src/main/java/org/redisson/client/protocol/BatchCommandData.java b/redisson/src/main/java/org/redisson/client/protocol/BatchCommandData.java index e6b3e058a..3173169f9 100644 --- a/redisson/src/main/java/org/redisson/client/protocol/BatchCommandData.java +++ b/redisson/src/main/java/org/redisson/client/protocol/BatchCommandData.java @@ -19,15 +19,14 @@ import java.util.concurrent.atomic.AtomicReference; import org.redisson.client.RedisRedirectException; import org.redisson.client.codec.Codec; - -import io.netty.util.concurrent.Promise; +import org.redisson.misc.RPromise; public class BatchCommandData extends CommandData implements Comparable> { private final int index; private final AtomicReference redirectError = new AtomicReference(); - public BatchCommandData(Promise promise, Codec codec, RedisCommand command, Object[] params, int index) { + public BatchCommandData(RPromise promise, Codec codec, RedisCommand command, Object[] params, int index) { super(promise, codec, command, params); this.index = index; } diff --git a/redisson/src/main/java/org/redisson/client/protocol/CommandData.java b/redisson/src/main/java/org/redisson/client/protocol/CommandData.java index 92bd1763b..9f470c549 100644 --- a/redisson/src/main/java/org/redisson/client/protocol/CommandData.java +++ b/redisson/src/main/java/org/redisson/client/protocol/CommandData.java @@ -21,22 +21,21 @@ import java.util.List; import org.redisson.client.codec.Codec; import org.redisson.client.protocol.decoder.MultiDecoder; - -import io.netty.util.concurrent.Promise; +import org.redisson.misc.RPromise; public class CommandData implements QueueCommand { - final Promise promise; + final RPromise promise; final RedisCommand command; final Object[] params; final Codec codec; final MultiDecoder messageDecoder; - public CommandData(Promise promise, Codec codec, RedisCommand command, Object[] params) { + public CommandData(RPromise promise, Codec codec, RedisCommand command, Object[] params) { this(promise, null, codec, command, params); } - public CommandData(Promise promise, MultiDecoder messageDecoder, Codec codec, RedisCommand command, Object[] params) { + public CommandData(RPromise promise, MultiDecoder messageDecoder, Codec codec, RedisCommand command, Object[] params) { this.promise = promise; this.command = command; this.params = params; @@ -56,7 +55,7 @@ public class CommandData implements QueueCommand { return messageDecoder; } - public Promise getPromise() { + public RPromise getPromise() { return promise; } diff --git a/redisson/src/main/java/org/redisson/client/protocol/CommandsData.java b/redisson/src/main/java/org/redisson/client/protocol/CommandsData.java index 85c07b116..b30556f09 100644 --- a/redisson/src/main/java/org/redisson/client/protocol/CommandsData.java +++ b/redisson/src/main/java/org/redisson/client/protocol/CommandsData.java @@ -18,20 +18,20 @@ package org.redisson.client.protocol; import java.util.ArrayList; import java.util.List; -import io.netty.util.concurrent.Promise; +import org.redisson.misc.RPromise; public class CommandsData implements QueueCommand { private final List> commands; - private final Promise promise; + private final RPromise promise; - public CommandsData(Promise promise, List> commands) { + public CommandsData(RPromise promise, List> commands) { super(); this.promise = promise; this.commands = commands; } - public Promise getPromise() { + public RPromise getPromise() { return promise; } diff --git a/redisson/src/main/java/org/redisson/cluster/ClusterConnectionManager.java b/redisson/src/main/java/org/redisson/cluster/ClusterConnectionManager.java index c19bfb80d..e0d6249a5 100644 --- a/redisson/src/main/java/org/redisson/cluster/ClusterConnectionManager.java +++ b/redisson/src/main/java/org/redisson/cluster/ClusterConnectionManager.java @@ -31,6 +31,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import org.redisson.api.RFuture; import org.redisson.client.RedisClient; import org.redisson.client.RedisConnection; import org.redisson.client.RedisConnectionException; @@ -46,13 +47,13 @@ import org.redisson.connection.ClientConnectionsEntry.FreezeReason; import org.redisson.connection.MasterSlaveConnectionManager; import org.redisson.connection.MasterSlaveEntry; import org.redisson.connection.SingleEntry; +import org.redisson.misc.RPromise; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; import io.netty.util.concurrent.GlobalEventExecutor; -import io.netty.util.concurrent.Promise; import io.netty.util.concurrent.ScheduledFuture; import io.netty.util.internal.PlatformDependent; @@ -77,7 +78,7 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { Exception lastException = null; for (URI addr : cfg.getNodeAddresses()) { - Future connectionFuture = connect(cfg, addr); + RFuture connectionFuture = connect(cfg, addr); try { RedisConnection connection = connectionFuture.syncUninterruptibly().getNow(); List nodes = connection.sync(RedisCommands.CLUSTER_NODES); @@ -93,21 +94,21 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { lastClusterNode = addr; Collection partitions = parsePartitions(nodes); - List>>> futures = new ArrayList>>>(); + List>>> futures = new ArrayList>>>(); for (ClusterPartition partition : partitions) { if (partition.isMasterFail()) { continue; } - Future>> masterFuture = addMasterEntry(partition, cfg); + RFuture>> masterFuture = addMasterEntry(partition, cfg); futures.add(masterFuture); } - for (Future>> masterFuture : futures) { + for (RFuture>> masterFuture : futures) { masterFuture.awaitUninterruptibly(); if (!masterFuture.isSuccess()) { continue; } - for (Future future : masterFuture.getNow()) { + for (RFuture future : masterFuture.getNow()) { future.awaitUninterruptibly(); if (!future.isSuccess()) { continue; @@ -140,15 +141,15 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { } } - private Future connect(ClusterServersConfig cfg, final URI addr) { + private RFuture connect(ClusterServersConfig cfg, final URI addr) { RedisConnection connection = nodeConnections.get(addr); if (connection != null) { return newSucceededFuture(connection); } RedisClient client = createClient(addr.getHost(), addr.getPort(), cfg.getConnectTimeout(), cfg.getRetryInterval() * cfg.getRetryAttempts()); - final Promise result = newPromise(); - Future future = client.connectAsync(); + final RPromise result = newPromise(); + RFuture future = client.connectAsync(); future.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -158,7 +159,7 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { } RedisConnection connection = future.getNow(); - Promise promise = newPromise(); + RPromise promise = newPromise(); connectListener.onConnect(promise, connection, null, config); promise.addListener(new FutureListener() { @Override @@ -188,7 +189,7 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { protected void initEntry(MasterSlaveServersConfig config) { } - private Future>> addMasterEntry(final ClusterPartition partition, final ClusterServersConfig cfg) { + private RFuture>> addMasterEntry(final ClusterPartition partition, final ClusterServersConfig cfg) { if (partition.isMasterFail()) { RedisException e = new RedisException("Failed to add master: " + partition.getMasterAddress() + " for slot ranges: " + @@ -201,8 +202,8 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { return newFailedFuture(e); } - final Promise>> result = newPromise(); - Future connectionFuture = connect(cfg, partition.getMasterAddress()); + final RPromise>> result = newPromise(); + RFuture connectionFuture = connect(cfg, partition.getMasterAddress()); connectionFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -213,7 +214,7 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { } final RedisConnection connection = future.getNow(); - Future> clusterFuture = connection.async(RedisCommands.CLUSTER_INFO); + RFuture> clusterFuture = connection.async(RedisCommands.CLUSTER_INFO); clusterFuture.addListener(new FutureListener>() { @Override @@ -238,7 +239,7 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { config.setMasterAddress(partition.getMasterAddress()); final MasterSlaveEntry e; - List> futures = new ArrayList>(); + List> futures = new ArrayList>(); if (config.getReadMode() == ReadMode.MASTER) { e = new SingleEntry(partition.getSlotRanges(), ClusterConnectionManager.this, config); } else { @@ -246,7 +247,7 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { e = new MasterSlaveEntry(partition.getSlotRanges(), ClusterConnectionManager.this, config); - List> fs = e.initSlaveBalancer(partition.getFailedSlaveAddresses()); + List> fs = e.initSlaveBalancer(partition.getFailedSlaveAddresses()); futures.addAll(fs); if (!partition.getSlaveAddresses().isEmpty()) { log.info("slaves: {} added for slot ranges: {}", partition.getSlaveAddresses(), partition.getSlotRanges()); @@ -256,8 +257,8 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { } } - Future f = e.setupMasterEntry(config.getMasterAddress().getHost(), config.getMasterAddress().getPort()); - final Promise initFuture = newPromise(); + RFuture f = e.setupMasterEntry(config.getMasterAddress().getHost(), config.getMasterAddress().getPort()); + final RPromise initFuture = newPromise(); futures.add(initFuture); f.addListener(new FutureListener() { @Override @@ -327,7 +328,7 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { return; } final URI uri = iterator.next(); - Future connectionFuture = connect(cfg, uri); + RFuture connectionFuture = connect(cfg, uri); connectionFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -344,7 +345,7 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { } private void updateClusterState(final ClusterServersConfig cfg, final RedisConnection connection, final Iterator iterator, final URI uri) { - Future> future = connection.async(RedisCommands.CLUSTER_NODES); + RFuture> future = connection.async(RedisCommands.CLUSTER_NODES); future.addListener(new FutureListener>() { @Override public void operationComplete(Future> future) throws Exception { @@ -367,7 +368,7 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { } final Collection newPartitions = parsePartitions(nodes); - Future masterFuture = checkMasterNodesChange(cfg, newPartitions); + RFuture masterFuture = checkMasterNodesChange(cfg, newPartitions); checkSlaveNodesChange(newPartitions); masterFuture.addListener(new FutureListener() { @Override @@ -434,7 +435,7 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { Set addedSlaves = new HashSet(newPart.getSlaveAddresses()); addedSlaves.removeAll(currentPart.getSlaveAddresses()); for (final URI uri : addedSlaves) { - Future future = entry.addSlave(uri.getHost(), uri.getPort()); + RFuture future = entry.addSlave(uri.getHost(), uri.getPort()); future.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -470,7 +471,7 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { return null; } - private Future checkMasterNodesChange(ClusterServersConfig cfg, Collection newPartitions) { + private RFuture checkMasterNodesChange(ClusterServersConfig cfg, Collection newPartitions) { List newMasters = new ArrayList(); for (final ClusterPartition newPart : newPartitions) { boolean masterFound = false; @@ -509,21 +510,21 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { return newSucceededFuture(null); } - final Promise result = newPromise(); + final RPromise result = newPromise(); final AtomicInteger masters = new AtomicInteger(newMasters.size()); - final Queue> futures = new ConcurrentLinkedQueue>(); + final Queue> futures = new ConcurrentLinkedQueue>(); for (ClusterPartition newPart : newMasters) { - Future>> future = addMasterEntry(newPart, cfg); - future.addListener(new FutureListener>>() { + RFuture>> future = addMasterEntry(newPart, cfg); + future.addListener(new FutureListener>>() { @Override - public void operationComplete(Future>> future) throws Exception { + public void operationComplete(Future>> future) throws Exception { if (future.isSuccess()) { futures.addAll(future.getNow()); } if (masters.decrementAndGet() == 0) { final AtomicInteger nodes = new AtomicInteger(futures.size()); - for (Future nodeFuture : futures) { + for (RFuture nodeFuture : futures) { nodeFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { diff --git a/redisson/src/main/java/org/redisson/command/AsyncDetails.java b/redisson/src/main/java/org/redisson/command/AsyncDetails.java index 0ea4c2fcd..f1aaf3f17 100644 --- a/redisson/src/main/java/org/redisson/command/AsyncDetails.java +++ b/redisson/src/main/java/org/redisson/command/AsyncDetails.java @@ -17,32 +17,32 @@ package org.redisson.command; import java.util.concurrent.ConcurrentLinkedQueue; +import org.redisson.api.RFuture; import org.redisson.client.RedisConnection; import org.redisson.client.RedisException; import org.redisson.client.codec.Codec; import org.redisson.client.protocol.RedisCommand; -import org.redisson.client.protocol.decoder.MultiDecoder; import org.redisson.connection.ConnectionManager; import org.redisson.connection.NodeSource; +import org.redisson.misc.RPromise; import io.netty.channel.ChannelFuture; import io.netty.util.Timeout; import io.netty.util.concurrent.Future; -import io.netty.util.concurrent.Promise; public class AsyncDetails { static final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue(); - Future connectionFuture; + RFuture connectionFuture; ConnectionManager connectionManager; - Promise attemptPromise; + RPromise attemptPromise; boolean readOnlyMode; NodeSource source; Codec codec; RedisCommand command; Object[] params; - Promise mainPromise; + RPromise mainPromise; int attempt; @@ -69,10 +69,10 @@ public class AsyncDetails { // queue.add(details); } - public void init(Future connectionFuture, - Promise attemptPromise, boolean readOnlyMode, NodeSource source, + public void init(RFuture connectionFuture, + RPromise attemptPromise, boolean readOnlyMode, NodeSource source, Codec codec, RedisCommand command, Object[] params, - Promise mainPromise, int attempt) { + RPromise mainPromise, int attempt) { this.connectionFuture = connectionFuture; this.attemptPromise = attemptPromise; this.readOnlyMode = readOnlyMode; @@ -108,11 +108,11 @@ public class AsyncDetails { this.timeout = timeout; } - public Future getConnectionFuture() { + public RFuture getConnectionFuture() { return connectionFuture; } - public Promise getAttemptPromise() { + public RPromise getAttemptPromise() { return attemptPromise; } @@ -136,7 +136,7 @@ public class AsyncDetails { return params; } - public Promise getMainPromise() { + public RPromise getMainPromise() { return mainPromise; } diff --git a/redisson/src/main/java/org/redisson/command/CommandAsyncExecutor.java b/redisson/src/main/java/org/redisson/command/CommandAsyncExecutor.java index 1aeaf7271..59bf5e9c1 100644 --- a/redisson/src/main/java/org/redisson/command/CommandAsyncExecutor.java +++ b/redisson/src/main/java/org/redisson/command/CommandAsyncExecutor.java @@ -28,8 +28,6 @@ import org.redisson.client.protocol.RedisCommand; import org.redisson.connection.ConnectionManager; import org.redisson.connection.MasterSlaveEntry; -import io.netty.util.concurrent.Future; - /** * * @author Nikita Koksharov @@ -39,11 +37,11 @@ public interface CommandAsyncExecutor { ConnectionManager getConnectionManager(); - RedisException convertException(Future RFuture); + RedisException convertException(RFuture RFuture); - boolean await(Future RFuture, long timeout, TimeUnit timeoutUnit) throws InterruptedException; + boolean await(RFuture RFuture, long timeout, TimeUnit timeoutUnit) throws InterruptedException; - V get(Future RFuture); + V get(RFuture RFuture); RFuture writeAsync(MasterSlaveEntry entry, Codec codec, RedisCommand command, Object ... params); diff --git a/redisson/src/main/java/org/redisson/command/CommandAsyncService.java b/redisson/src/main/java/org/redisson/command/CommandAsyncService.java index e0da953c3..01dc900ab 100644 --- a/redisson/src/main/java/org/redisson/command/CommandAsyncService.java +++ b/redisson/src/main/java/org/redisson/command/CommandAsyncService.java @@ -59,7 +59,6 @@ import io.netty.util.Timeout; import io.netty.util.TimerTask; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; -import io.netty.util.concurrent.Promise; /** * @@ -82,7 +81,7 @@ public class CommandAsyncService implements CommandAsyncExecutor { } @Override - public V get(Future future) { + public V get(RFuture future) { final CountDownLatch l = new CountDownLatch(1); future.addListener(new FutureListener() { @Override @@ -114,7 +113,7 @@ public class CommandAsyncService implements CommandAsyncExecutor { } @Override - public boolean await(Future future, long timeout, TimeUnit timeoutUnit) throws InterruptedException { + public boolean await(RFuture future, long timeout, TimeUnit timeoutUnit) throws InterruptedException { final CountDownLatch l = new CountDownLatch(1); future.addListener(new FutureListener() { @Override @@ -173,7 +172,7 @@ public class CommandAsyncService implements CommandAsyncExecutor { }; for (MasterSlaveEntry entry : nodes) { - Promise promise = connectionManager.newPromise(); + RPromise promise = connectionManager.newPromise(); promise.addListener(listener); async(true, new NodeSource(entry), connectionManager.getCodec(), command, params, promise, 0); } @@ -190,7 +189,7 @@ public class CommandAsyncService implements CommandAsyncExecutor { return mainPromise; } - private void retryReadRandomAsync(final RedisCommand command, final Promise mainPromise, + private void retryReadRandomAsync(final RedisCommand command, final RPromise mainPromise, final List nodes, final Object... params) { final RPromise attemptPromise = connectionManager.newPromise(); attemptPromise.addListener(new FutureListener() { @@ -257,14 +256,14 @@ public class CommandAsyncService implements CommandAsyncExecutor { }; for (MasterSlaveEntry entry : nodes) { - Promise promise = connectionManager.newPromise(); + RPromise promise = connectionManager.newPromise(); promise.addListener(listener); async(readOnlyMode, new NodeSource(entry), connectionManager.getCodec(), command, params, promise, 0); } return mainPromise; } - public RedisException convertException(Future future) { + public RedisException convertException(RFuture future) { return future.cause() instanceof RedisException ? (RedisException) future.cause() : new RedisException("Unexpected exception while processing command", future.cause()); @@ -385,7 +384,7 @@ public class CommandAsyncService implements CommandAsyncExecutor { args.addAll(keys); args.addAll(Arrays.asList(params)); for (MasterSlaveEntry entry : entries) { - Promise promise = connectionManager.newPromise(); + RPromise promise = connectionManager.newPromise(); promise.addListener(listener); async(readOnlyMode, new NodeSource(entry), connectionManager.getCodec(), command, args.toArray(), promise, 0); } @@ -417,7 +416,7 @@ public class CommandAsyncService implements CommandAsyncExecutor { } protected void async(final boolean readOnlyMode, final NodeSource source, final Codec codec, - final RedisCommand command, final Object[] params, final Promise mainPromise, final int attempt) { + final RedisCommand command, final Object[] params, final RPromise mainPromise, final int attempt) { if (mainPromise.isCancelled()) { return; } @@ -427,9 +426,9 @@ public class CommandAsyncService implements CommandAsyncExecutor { return; } - final Promise attemptPromise = connectionManager.newPromise(); + final RPromise attemptPromise = connectionManager.newPromise(); - final Future connectionFuture; + final RFuture connectionFuture; if (readOnlyMode) { connectionFuture = connectionManager.connectionReadOp(source, command); } else { @@ -499,22 +498,22 @@ public class CommandAsyncService implements CommandAsyncExecutor { if (!connFuture.isSuccess()) { connectionManager.getShutdownLatch().release(); - details.setException(convertException(connFuture)); + details.setException(convertException(connectionFuture)); return; } if (details.getAttemptPromise().isDone() || details.getMainPromise().isDone()) { - releaseConnection(source, connFuture, details.isReadOnlyMode(), details.getAttemptPromise(), details); + releaseConnection(source, connectionFuture, details.isReadOnlyMode(), details.getAttemptPromise(), details); return; } final RedisConnection connection = connFuture.getNow(); if (details.getSource().getRedirect() == Redirect.ASK) { List> list = new ArrayList>(2); - Promise promise = connectionManager.newPromise(); + RPromise promise = connectionManager.newPromise(); list.add(new CommandData(promise, details.getCodec(), RedisCommands.ASKING, new Object[] {})); list.add(new CommandData(details.getAttemptPromise(), details.getCodec(), details.getCommand(), details.getParams())); - Promise main = connectionManager.newPromise(); + RPromise main = connectionManager.newPromise(); ChannelFuture future = connection.send(new CommandsData(main, list)); details.setWriteFuture(future); } else { @@ -533,7 +532,7 @@ public class CommandAsyncService implements CommandAsyncExecutor { } }); - releaseConnection(source, connFuture, details.isReadOnlyMode(), details.getAttemptPromise(), details); + releaseConnection(source, connectionFuture, details.isReadOnlyMode(), details.getAttemptPromise(), details); } }); @@ -647,8 +646,8 @@ public class CommandAsyncService implements CommandAsyncExecutor { } } - protected void releaseConnection(final NodeSource source, final Future connectionFuture, - final boolean isReadOnly, Promise attemptPromise, final AsyncDetails details) { + protected void releaseConnection(final NodeSource source, final RFuture connectionFuture, + final boolean isReadOnly, RPromise attemptPromise, final AsyncDetails details) { attemptPromise.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { diff --git a/redisson/src/main/java/org/redisson/command/CommandBatchService.java b/redisson/src/main/java/org/redisson/command/CommandBatchService.java index 38cbb646a..d693f3670 100644 --- a/redisson/src/main/java/org/redisson/command/CommandBatchService.java +++ b/redisson/src/main/java/org/redisson/command/CommandBatchService.java @@ -93,7 +93,7 @@ public class CommandBatchService extends CommandReactiveService { @Override protected void async(boolean readOnlyMode, NodeSource nodeSource, - Codec codec, RedisCommand command, Object[] params, Promise mainPromise, int attempt) { + Codec codec, RedisCommand command, Object[] params, RPromise mainPromise, int attempt) { if (executed) { throw new IllegalStateException("Batch already has been executed!"); } @@ -153,7 +153,7 @@ public class CommandBatchService extends CommandReactiveService { } executed = true; - Promise voidPromise = connectionManager.newPromise(); + RPromise voidPromise = connectionManager.newPromise(); final RPromise> promise = connectionManager.newPromise(); voidPromise.addListener(new FutureListener() { @Override @@ -185,7 +185,7 @@ public class CommandBatchService extends CommandReactiveService { return promise; } - public void execute(final Entry entry, final NodeSource source, final Promise mainPromise, final AtomicInteger slots, final int attempt) { + public void execute(final Entry entry, final NodeSource source, final RPromise mainPromise, final AtomicInteger slots, final int attempt) { if (mainPromise.isCancelled()) { return; } @@ -195,11 +195,11 @@ public class CommandBatchService extends CommandReactiveService { return; } - final Promise attemptPromise = connectionManager.newPromise(); + final RPromise attemptPromise = connectionManager.newPromise(); final AsyncDetails details = new AsyncDetails(); - final Future connectionFuture; + final RFuture connectionFuture; if (entry.isReadOnlyMode()) { connectionFuture = connectionManager.connectionReadOp(source, null); } else { @@ -254,7 +254,7 @@ public class CommandBatchService extends CommandReactiveService { connectionFuture.addListener(new FutureListener() { @Override public void operationComplete(Future connFuture) throws Exception { - checkConnectionFuture(entry, source, mainPromise, attemptPromise, details, connFuture); + checkConnectionFuture(entry, source, mainPromise, attemptPromise, details, connectionFuture); } }); } @@ -296,7 +296,7 @@ public class CommandBatchService extends CommandReactiveService { }); } - private void checkWriteFuture(final Promise attemptPromise, AsyncDetails details, + private void checkWriteFuture(final RPromise attemptPromise, AsyncDetails details, final RedisConnection connection, ChannelFuture future) { if (attemptPromise.isDone() || future.isCancelled()) { return; @@ -319,8 +319,8 @@ public class CommandBatchService extends CommandReactiveService { } private void checkConnectionFuture(final Entry entry, final NodeSource source, - final Promise mainPromise, final Promise attemptPromise, final AsyncDetails details, - Future connFuture) { + final RPromise mainPromise, final RPromise attemptPromise, final AsyncDetails details, + RFuture connFuture) { if (attemptPromise.isDone() || mainPromise.isCancelled() || connFuture.isCancelled()) { return; } @@ -335,7 +335,7 @@ public class CommandBatchService extends CommandReactiveService { List> list = new ArrayList>(entry.getCommands().size() + 1); if (source.getRedirect() == Redirect.ASK) { - Promise promise = connectionManager.newPromise(); + RPromise promise = connectionManager.newPromise(); list.add(new CommandData(promise, StringCodec.INSTANCE, RedisCommands.ASKING, new Object[] {})); } for (BatchCommandData c : entry.getCommands()) { diff --git a/redisson/src/main/java/org/redisson/command/CommandReactiveExecutor.java b/redisson/src/main/java/org/redisson/command/CommandReactiveExecutor.java index 05e205a4e..32df60fca 100644 --- a/redisson/src/main/java/org/redisson/command/CommandReactiveExecutor.java +++ b/redisson/src/main/java/org/redisson/command/CommandReactiveExecutor.java @@ -21,13 +21,12 @@ import java.util.List; import org.reactivestreams.Publisher; import org.redisson.SlotCallback; +import org.redisson.api.RFuture; import org.redisson.client.codec.Codec; import org.redisson.client.protocol.RedisCommand; import org.redisson.connection.ConnectionManager; import org.redisson.connection.MasterSlaveEntry; -import io.netty.util.concurrent.Future; - /** * * @author Nikita Koksharov @@ -35,7 +34,7 @@ import io.netty.util.concurrent.Future; */ public interface CommandReactiveExecutor extends CommandAsyncExecutor { - Publisher reactive(Future future); + Publisher reactive(RFuture future); ConnectionManager getConnectionManager(); diff --git a/redisson/src/main/java/org/redisson/command/CommandReactiveService.java b/redisson/src/main/java/org/redisson/command/CommandReactiveService.java index 236a4da64..fc33c3f5f 100644 --- a/redisson/src/main/java/org/redisson/command/CommandReactiveService.java +++ b/redisson/src/main/java/org/redisson/command/CommandReactiveService.java @@ -21,14 +21,13 @@ import java.util.List; import org.reactivestreams.Publisher; import org.redisson.SlotCallback; +import org.redisson.api.RFuture; import org.redisson.client.codec.Codec; import org.redisson.client.protocol.RedisCommand; import org.redisson.connection.ConnectionManager; import org.redisson.connection.MasterSlaveEntry; import org.redisson.reactive.NettyFuturePublisher; -import io.netty.util.concurrent.Future; - /** * * @author Nikita Koksharov @@ -42,29 +41,29 @@ public class CommandReactiveService extends CommandAsyncService implements Comma @Override public Publisher evalWriteAllReactive(RedisCommand command, SlotCallback callback, String script, List keys, Object ... params) { - Future f = evalWriteAllAsync(command, callback, script, keys, params); + RFuture f = evalWriteAllAsync(command, callback, script, keys, params); return new NettyFuturePublisher(f); } - public Publisher reactive(Future future) { + public Publisher reactive(RFuture future) { return new NettyFuturePublisher(future); } @Override public Publisher> readAllReactive(RedisCommand command, Object ... params) { - Future> f = readAllAsync(command, params); + RFuture> f = readAllAsync(command, params); return new NettyFuturePublisher>(f); } @Override public Publisher readRandomReactive(RedisCommand command, Object ... params) { - Future f = readRandomAsync(command, params); + RFuture f = readRandomAsync(command, params); return new NettyFuturePublisher(f); } @Override public Publisher readReactive(InetSocketAddress client, String key, Codec codec, RedisCommand command, Object ... params) { - Future f = readAsync(client, key, codec, command, params); + RFuture f = readAsync(client, key, codec, command, params); return new NettyFuturePublisher(f); } @@ -75,13 +74,13 @@ public class CommandReactiveService extends CommandAsyncService implements Comma @Override public Publisher writeReactive(String key, Codec codec, RedisCommand command, Object ... params) { - Future f = writeAsync(key, codec, command, params); + RFuture f = writeAsync(key, codec, command, params); return new NettyFuturePublisher(f); } @Override public Publisher writeReactive(MasterSlaveEntry entry, Codec codec, RedisCommand command, Object ... params) { - Future f = writeAsync(entry, codec, command, params); + RFuture f = writeAsync(entry, codec, command, params); return new NettyFuturePublisher(f); } @@ -92,21 +91,21 @@ public class CommandReactiveService extends CommandAsyncService implements Comma @Override public Publisher readReactive(String key, Codec codec, RedisCommand command, Object ... params) { - Future f = readAsync(key, codec, command, params); + RFuture f = readAsync(key, codec, command, params); return new NettyFuturePublisher(f); } @Override public Publisher evalReadReactive(String key, Codec codec, RedisCommand evalCommandType, String script, List keys, Object... params) { - Future f = evalReadAsync(key, codec, evalCommandType, script, keys, params); + RFuture f = evalReadAsync(key, codec, evalCommandType, script, keys, params); return new NettyFuturePublisher(f); } @Override public Publisher evalReadReactive(InetSocketAddress client, String key, Codec codec, RedisCommand evalCommandType, String script, List keys, Object ... params) { - Future f = evalReadAsync(client, key, codec, evalCommandType, script, keys, params); + RFuture f = evalReadAsync(client, key, codec, evalCommandType, script, keys, params); return new NettyFuturePublisher(f); } @@ -114,19 +113,19 @@ public class CommandReactiveService extends CommandAsyncService implements Comma @Override public Publisher evalWriteReactive(String key, Codec codec, RedisCommand evalCommandType, String script, List keys, Object... params) { - Future f = evalWriteAsync(key, codec, evalCommandType, script, keys, params); + RFuture f = evalWriteAsync(key, codec, evalCommandType, script, keys, params); return new NettyFuturePublisher(f); } @Override public Publisher writeAllReactive(RedisCommand command, Object ... params) { - Future f = writeAllAsync(command, params); + RFuture f = writeAllAsync(command, params); return new NettyFuturePublisher(f); } @Override public Publisher writeAllReactive(RedisCommand command, SlotCallback callback, Object ... params) { - Future f = writeAllAsync(command, callback, params); + RFuture f = writeAllAsync(command, callback, params); return new NettyFuturePublisher(f); } diff --git a/redisson/src/main/java/org/redisson/command/CommandSyncExecutor.java b/redisson/src/main/java/org/redisson/command/CommandSyncExecutor.java index ac6c70ca9..104171b52 100644 --- a/redisson/src/main/java/org/redisson/command/CommandSyncExecutor.java +++ b/redisson/src/main/java/org/redisson/command/CommandSyncExecutor.java @@ -18,12 +18,11 @@ package org.redisson.command; import java.net.InetSocketAddress; import java.util.List; +import org.redisson.api.RFuture; import org.redisson.client.codec.Codec; import org.redisson.client.protocol.RedisCommand; import org.redisson.connection.ConnectionManager; -import io.netty.util.concurrent.Future; - /** * * @author Nikita Koksharov @@ -31,7 +30,7 @@ import io.netty.util.concurrent.Future; */ public interface CommandSyncExecutor { - V get(Future future); + V get(RFuture future); R write(Integer slot, Codec codec, RedisCommand command, Object ... params); diff --git a/redisson/src/main/java/org/redisson/command/CommandSyncService.java b/redisson/src/main/java/org/redisson/command/CommandSyncService.java index f99372496..8a05f4821 100644 --- a/redisson/src/main/java/org/redisson/command/CommandSyncService.java +++ b/redisson/src/main/java/org/redisson/command/CommandSyncService.java @@ -18,14 +18,13 @@ package org.redisson.command; import java.net.InetSocketAddress; import java.util.List; +import org.redisson.api.RFuture; import org.redisson.client.codec.Codec; import org.redisson.client.protocol.RedisCommand; import org.redisson.connection.ConnectionManager; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import io.netty.util.concurrent.Future; - /** * * @author Nikita Koksharov @@ -46,19 +45,19 @@ public class CommandSyncService extends CommandAsyncService implements CommandEx @Override public R read(String key, Codec codec, RedisCommand command, Object ... params) { - Future res = readAsync(key, codec, command, params); + RFuture res = readAsync(key, codec, command, params); return get(res); } @Override public R read(InetSocketAddress client, String key, RedisCommand command, Object ... params) { - Future res = readAsync(client, key, connectionManager.getCodec(), command, params); + RFuture res = readAsync(client, key, connectionManager.getCodec(), command, params); return get(res); } @Override public R read(InetSocketAddress client, String key, Codec codec, RedisCommand command, Object ... params) { - Future res = readAsync(client, key, codec, command, params); + RFuture res = readAsync(client, key, codec, command, params); return get(res); } @@ -69,7 +68,7 @@ public class CommandSyncService extends CommandAsyncService implements CommandEx @Override public R evalRead(String key, Codec codec, RedisCommand evalCommandType, String script, List keys, Object ... params) { - Future res = evalReadAsync(key, codec, evalCommandType, script, keys, params); + RFuture res = evalReadAsync(key, codec, evalCommandType, script, keys, params); return get(res); } @@ -80,25 +79,25 @@ public class CommandSyncService extends CommandAsyncService implements CommandEx @Override public R evalWrite(String key, Codec codec, RedisCommand evalCommandType, String script, List keys, Object ... params) { - Future res = evalWriteAsync(key, codec, evalCommandType, script, keys, params); + RFuture res = evalWriteAsync(key, codec, evalCommandType, script, keys, params); return get(res); } @Override public R write(Integer slot, Codec codec, RedisCommand command, Object ... params) { - Future res = writeAsync(slot, codec, command, params); + RFuture res = writeAsync(slot, codec, command, params); return get(res); } @Override public R write(String key, Codec codec, RedisCommand command, Object ... params) { - Future res = writeAsync(key, codec, command, params); + RFuture res = writeAsync(key, codec, command, params); return get(res); } @Override public R write(String key, RedisCommand command, Object ... params) { - Future res = writeAsync(key, command, params); + RFuture res = writeAsync(key, command, params); return get(res); } diff --git a/redisson/src/main/java/org/redisson/connection/ClientConnectionsEntry.java b/redisson/src/main/java/org/redisson/connection/ClientConnectionsEntry.java index a4c2613cd..e0d869a23 100644 --- a/redisson/src/main/java/org/redisson/connection/ClientConnectionsEntry.java +++ b/redisson/src/main/java/org/redisson/connection/ClientConnectionsEntry.java @@ -20,17 +20,18 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicInteger; import org.redisson.api.NodeType; +import org.redisson.api.RFuture; import org.redisson.client.ReconnectListener; import org.redisson.client.RedisClient; import org.redisson.client.RedisConnection; import org.redisson.client.RedisPubSubConnection; import org.redisson.config.MasterSlaveServersConfig; +import org.redisson.misc.RPromise; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; -import io.netty.util.concurrent.ImmediateEventExecutor; import io.netty.util.concurrent.Promise; public class ClientConnectionsEntry { @@ -138,9 +139,9 @@ public class ClientConnectionsEntry { freeConnections.add(connection); } - public Future connect() { - final Promise connectionFuture = connectionManager.newPromise(); - Future future = client.connectAsync(); + public RFuture connect() { + final RPromise connectionFuture = connectionManager.newPromise(); + RFuture future = client.connectAsync(); future.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -158,18 +159,18 @@ public class ClientConnectionsEntry { return connectionFuture; } - private void addReconnectListener(Promise connectionFuture, T conn) { + private void addReconnectListener(RPromise connectionFuture, T conn) { addFireEventListener(conn, connectionFuture); conn.setReconnectListener(new ReconnectListener() { @Override - public void onReconnect(RedisConnection conn, Promise connectionFuture) { + public void onReconnect(RedisConnection conn, RPromise connectionFuture) { addFireEventListener(conn, connectionFuture); } }); } - private void addFireEventListener(T conn, Promise connectionFuture) { + private void addFireEventListener(T conn, RPromise connectionFuture) { connectionManager.getConnectListener().onConnect(connectionFuture, conn, nodeType, connectionManager.getConfig()); if (connectionFuture.isSuccess()) { @@ -191,9 +192,9 @@ public class ClientConnectionsEntry { return connectionManager.getConfig(); } - public Future connectPubSub() { - final Promise connectionFuture = connectionManager.newPromise(); - Future future = client.connectPubSubAsync(); + public RFuture connectPubSub() { + final RPromise connectionFuture = connectionManager.newPromise(); + RFuture future = client.connectPubSubAsync(); future.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { diff --git a/redisson/src/main/java/org/redisson/connection/ConnectionInitializer.java b/redisson/src/main/java/org/redisson/connection/ConnectionInitializer.java index 124ace6b5..f9b98c35b 100644 --- a/redisson/src/main/java/org/redisson/connection/ConnectionInitializer.java +++ b/redisson/src/main/java/org/redisson/connection/ConnectionInitializer.java @@ -18,11 +18,10 @@ package org.redisson.connection; import org.redisson.api.NodeType; import org.redisson.client.RedisConnection; import org.redisson.config.MasterSlaveServersConfig; - -import io.netty.util.concurrent.Promise; +import org.redisson.misc.RPromise; public interface ConnectionInitializer { - void onConnect(Promise connectionFuture, T conn, NodeType nodeType, MasterSlaveServersConfig config); + void onConnect(RPromise connectionFuture, T conn, NodeType nodeType, MasterSlaveServersConfig config); } diff --git a/redisson/src/main/java/org/redisson/connection/ConnectionManager.java b/redisson/src/main/java/org/redisson/connection/ConnectionManager.java index 4c9f69063..4f3022fd8 100644 --- a/redisson/src/main/java/org/redisson/connection/ConnectionManager.java +++ b/redisson/src/main/java/org/redisson/connection/ConnectionManager.java @@ -36,7 +36,6 @@ import org.redisson.pubsub.AsyncSemaphore; import io.netty.channel.EventLoopGroup; import io.netty.util.Timeout; import io.netty.util.TimerTask; -import io.netty.util.concurrent.Future; /** * @@ -59,9 +58,9 @@ public interface ConnectionManager { boolean isShuttingDown(); - Future subscribe(Codec codec, String channelName, RedisPubSubListener listener); + RFuture subscribe(Codec codec, String channelName, RedisPubSubListener listener); - Future subscribe(Codec codec, String channelName, RedisPubSubListener listener, AsyncSemaphore semaphore); + RFuture subscribe(Codec codec, String channelName, RedisPubSubListener listener, AsyncSemaphore semaphore); ConnectionInitializer getConnectListener(); @@ -89,9 +88,9 @@ public interface ConnectionManager { void releaseWrite(NodeSource source, RedisConnection connection); - Future connectionReadOp(NodeSource source, RedisCommand command); + RFuture connectionReadOp(NodeSource source, RedisCommand command); - Future connectionWriteOp(NodeSource source, RedisCommand command); + RFuture connectionWriteOp(NodeSource source, RedisCommand command); RedisClient createClient(String host, int port, int timeout, int commandTimeout); @@ -101,9 +100,9 @@ public interface ConnectionManager { PubSubConnectionEntry getPubSubEntry(String channelName); - Future psubscribe(String pattern, Codec codec, RedisPubSubListener listener); + RFuture psubscribe(String pattern, Codec codec, RedisPubSubListener listener); - Future psubscribe(String pattern, Codec codec, RedisPubSubListener listener, AsyncSemaphore semaphore); + RFuture psubscribe(String pattern, Codec codec, RedisPubSubListener listener, AsyncSemaphore semaphore); Codec unsubscribe(String channelName, AsyncSemaphore lock); @@ -123,6 +122,6 @@ public interface ConnectionManager { InfinitySemaphoreLatch getShutdownLatch(); - Future getShutdownPromise(); + RFuture getShutdownPromise(); } diff --git a/redisson/src/main/java/org/redisson/connection/DefaultConnectionListener.java b/redisson/src/main/java/org/redisson/connection/DefaultConnectionListener.java index 534723474..8dc8f3a3c 100644 --- a/redisson/src/main/java/org/redisson/connection/DefaultConnectionListener.java +++ b/redisson/src/main/java/org/redisson/connection/DefaultConnectionListener.java @@ -20,12 +20,11 @@ import org.redisson.client.RedisConnection; import org.redisson.client.RedisException; import org.redisson.client.protocol.RedisCommands; import org.redisson.config.MasterSlaveServersConfig; - -import io.netty.util.concurrent.Promise; +import org.redisson.misc.RPromise; public class DefaultConnectionListener implements ConnectionInitializer { - public final void onConnect(Promise connectionFuture, T conn, NodeType nodeType, MasterSlaveServersConfig config) { + public final void onConnect(RPromise connectionFuture, T conn, NodeType nodeType, MasterSlaveServersConfig config) { FutureConnectionListener listener = new FutureConnectionListener(connectionFuture, conn); doConnect(config, nodeType, listener); listener.executeCommands(); diff --git a/redisson/src/main/java/org/redisson/connection/ElasticacheConnectionManager.java b/redisson/src/main/java/org/redisson/connection/ElasticacheConnectionManager.java index 539134d7f..615b6ffee 100644 --- a/redisson/src/main/java/org/redisson/connection/ElasticacheConnectionManager.java +++ b/redisson/src/main/java/org/redisson/connection/ElasticacheConnectionManager.java @@ -30,11 +30,11 @@ import org.redisson.config.BaseMasterSlaveServersConfig; import org.redisson.config.Config; import org.redisson.config.ElasticacheServersConfig; import org.redisson.config.MasterSlaveServersConfig; +import org.redisson.misc.RPromise; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import io.netty.util.concurrent.GlobalEventExecutor; -import io.netty.util.concurrent.Promise; import io.netty.util.concurrent.ScheduledFuture; /** @@ -110,7 +110,7 @@ public class ElasticacheConnectionManager extends MasterSlaveConnectionManager { RedisClient client = createClient(addr.getHost(), addr.getPort(), cfg.getConnectTimeout(), cfg.getRetryInterval() * cfg.getRetryAttempts()); try { connection = client.connect(); - Promise future = newPromise(); + RPromise future = newPromise(); connectListener.onConnect(future, connection, null, config); future.syncUninterruptibly(); nodeConnections.put(addr, connection); diff --git a/redisson/src/main/java/org/redisson/connection/FutureConnectionListener.java b/redisson/src/main/java/org/redisson/connection/FutureConnectionListener.java index 23af19ebb..2d907b441 100644 --- a/redisson/src/main/java/org/redisson/connection/FutureConnectionListener.java +++ b/redisson/src/main/java/org/redisson/connection/FutureConnectionListener.java @@ -19,22 +19,23 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; +import org.redisson.api.RFuture; import org.redisson.client.RedisConnection; import org.redisson.client.protocol.RedisCommand; +import org.redisson.misc.RPromise; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; -import io.netty.util.concurrent.Promise; public class FutureConnectionListener implements FutureListener { private final AtomicInteger commandsCounter = new AtomicInteger(); - private final Promise connectionPromise; + private final RPromise connectionPromise; private final T connection; private final List commands = new ArrayList(4); - public FutureConnectionListener(Promise connectionFuture, T connection) { + public FutureConnectionListener(RPromise connectionFuture, T connection) { super(); this.connectionPromise = connectionFuture; this.connection = connection; @@ -45,7 +46,7 @@ public class FutureConnectionListener implements Futu commands.add(new Runnable() { @Override public void run() { - Future future = connection.async(command, params); + RFuture future = connection.async(command, params); future.addListener(FutureConnectionListener.this); } }); diff --git a/redisson/src/main/java/org/redisson/connection/MasterSlaveConnectionManager.java b/redisson/src/main/java/org/redisson/connection/MasterSlaveConnectionManager.java index 0570b0c76..eb496253e 100644 --- a/redisson/src/main/java/org/redisson/connection/MasterSlaveConnectionManager.java +++ b/redisson/src/main/java/org/redisson/connection/MasterSlaveConnectionManager.java @@ -70,7 +70,6 @@ import io.netty.util.TimerTask; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; import io.netty.util.concurrent.ImmediateEventExecutor; -import io.netty.util.concurrent.Promise; import io.netty.util.internal.PlatformDependent; /** @@ -131,7 +130,7 @@ public class MasterSlaveConnectionManager implements ConnectionManager { private final Map entries = PlatformDependent.newConcurrentHashMap(); - private final Promise shutdownPromise; + private final RPromise shutdownPromise; private final InfinitySemaphoreLatch shutdownLatch = new InfinitySemaphoreLatch(); @@ -245,7 +244,7 @@ public class MasterSlaveConnectionManager implements ConnectionManager { MasterSlaveEntry entry; if (config.getReadMode() == ReadMode.MASTER) { entry = new SingleEntry(slots, this, config); - Future f = entry.setupMasterEntry(config.getMasterAddress().getHost(), config.getMasterAddress().getPort()); + RFuture f = entry.setupMasterEntry(config.getMasterAddress().getHost(), config.getMasterAddress().getPort()); f.syncUninterruptibly(); } else { entry = createMasterSlaveEntry(config, slots); @@ -259,11 +258,11 @@ public class MasterSlaveConnectionManager implements ConnectionManager { protected MasterSlaveEntry createMasterSlaveEntry(MasterSlaveServersConfig config, HashSet slots) { MasterSlaveEntry entry = new MasterSlaveEntry(slots, this, config); - List> fs = entry.initSlaveBalancer(java.util.Collections.emptySet()); - for (Future future : fs) { + List> fs = entry.initSlaveBalancer(java.util.Collections.emptySet()); + for (RFuture future : fs) { future.syncUninterruptibly(); } - Future f = entry.setupMasterEntry(config.getMasterAddress().getHost(), config.getMasterAddress().getPort()); + RFuture f = entry.setupMasterEntry(config.getMasterAddress().getHost(), config.getMasterAddress().getPort()); f.syncUninterruptibly(); return entry; } @@ -322,40 +321,40 @@ public class MasterSlaveConnectionManager implements ConnectionManager { } @Override - public Future psubscribe(final String channelName, final Codec codec, final RedisPubSubListener listener) { + public RFuture psubscribe(final String channelName, final Codec codec, final RedisPubSubListener listener) { final AsyncSemaphore lock = getSemaphore(channelName); - final Promise result = newPromise(); + final RPromise result = newPromise(); lock.acquire(new Runnable() { @Override public void run() { - Future future = psubscribe(channelName, codec, listener, lock); + RFuture future = psubscribe(channelName, codec, listener, lock); future.addListener(new TransferListener(result)); } }); return result; } - public Future psubscribe(String channelName, Codec codec, RedisPubSubListener listener, AsyncSemaphore semaphore) { - Promise promise = newPromise(); + public RFuture psubscribe(String channelName, Codec codec, RedisPubSubListener listener, AsyncSemaphore semaphore) { + RPromise promise = newPromise(); subscribe(codec, channelName, listener, promise, PubSubType.PSUBSCRIBE, semaphore); return promise; } - public Future subscribe(final Codec codec, final String channelName, final RedisPubSubListener listener) { + public RFuture subscribe(final Codec codec, final String channelName, final RedisPubSubListener listener) { final AsyncSemaphore lock = getSemaphore(channelName); - final Promise result = newPromise(); + final RPromise result = newPromise(); lock.acquire(new Runnable() { @Override public void run() { - Future future = subscribe(codec, channelName, listener, lock); + RFuture future = subscribe(codec, channelName, listener, lock); future.addListener(new TransferListener(result)); } }); return result; } - public Future subscribe(Codec codec, String channelName, RedisPubSubListener listener, AsyncSemaphore semaphore) { - Promise promise = newPromise(); + public RFuture subscribe(Codec codec, String channelName, RedisPubSubListener listener, AsyncSemaphore semaphore) { + RPromise promise = newPromise(); subscribe(codec, channelName, listener, promise, PubSubType.SUBSCRIBE, semaphore); return promise; } @@ -365,7 +364,7 @@ public class MasterSlaveConnectionManager implements ConnectionManager { } private void subscribe(final Codec codec, final String channelName, final RedisPubSubListener listener, - final Promise promise, final PubSubType type, final AsyncSemaphore lock) { + final RPromise promise, final PubSubType type, final AsyncSemaphore lock) { final PubSubConnectionEntry сonnEntry = name2PubSubConnection.get(channelName); if (сonnEntry != null) { сonnEntry.addListener(channelName, listener); @@ -439,9 +438,9 @@ public class MasterSlaveConnectionManager implements ConnectionManager { } private void connect(final Codec codec, final String channelName, final RedisPubSubListener listener, - final Promise promise, final PubSubType type, final AsyncSemaphore lock) { + final RPromise promise, final PubSubType type, final AsyncSemaphore lock) { final int slot = calcSlot(channelName); - Future connFuture = nextPubSubConnection(slot); + RFuture connFuture = nextPubSubConnection(slot); connFuture.addListener(new FutureListener() { @Override @@ -614,7 +613,7 @@ public class MasterSlaveConnectionManager implements ConnectionManager { } @Override - public Future connectionWriteOp(NodeSource source, RedisCommand command) { + public RFuture connectionWriteOp(NodeSource source, RedisCommand command) { MasterSlaveEntry entry = source.getEntry(); if (entry == null) { entry = getEntry(source); @@ -640,7 +639,7 @@ public class MasterSlaveConnectionManager implements ConnectionManager { } @Override - public Future connectionReadOp(NodeSource source, RedisCommand command) { + public RFuture connectionReadOp(NodeSource source, RedisCommand command) { MasterSlaveEntry entry = source.getEntry(); if (entry == null && source.getSlot() != null) { entry = getEntry(source.getSlot()); @@ -651,7 +650,7 @@ public class MasterSlaveConnectionManager implements ConnectionManager { return entry.connectionReadOp(); } - Future nextPubSubConnection(int slot) { + RFuture nextPubSubConnection(int slot) { return getEntry(slot).nextPubSubConnection(); } @@ -746,7 +745,7 @@ public class MasterSlaveConnectionManager implements ConnectionManager { } @Override - public Future getShutdownPromise() { + public RFuture getShutdownPromise() { return shutdownPromise; } diff --git a/redisson/src/main/java/org/redisson/connection/MasterSlaveEntry.java b/redisson/src/main/java/org/redisson/connection/MasterSlaveEntry.java index 61cc713dc..d28a0979d 100644 --- a/redisson/src/main/java/org/redisson/connection/MasterSlaveEntry.java +++ b/redisson/src/main/java/org/redisson/connection/MasterSlaveEntry.java @@ -25,6 +25,7 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import org.redisson.api.NodeType; +import org.redisson.api.RFuture; import org.redisson.client.RedisClient; import org.redisson.client.RedisConnection; import org.redisson.client.RedisPubSubConnection; @@ -79,13 +80,13 @@ public class MasterSlaveEntry { writeConnectionHolder = new MasterConnectionPool(config, connectionManager, this); } - public List> initSlaveBalancer(Collection disconnectedNodes) { + public List> initSlaveBalancer(Collection disconnectedNodes) { boolean freezeMasterAsSlave = !config.getSlaveAddresses().isEmpty() && config.getReadMode() == ReadMode.SLAVE && disconnectedNodes.size() < config.getSlaveAddresses().size(); - List> result = new LinkedList>(); - Future f = addSlave(config.getMasterAddress().getHost(), config.getMasterAddress().getPort(), freezeMasterAsSlave, NodeType.MASTER); + List> result = new LinkedList>(); + RFuture f = addSlave(config.getMasterAddress().getHost(), config.getMasterAddress().getPort(), freezeMasterAsSlave, NodeType.MASTER); result.add(f); for (URI address : config.getSlaveAddresses()) { f = addSlave(address.getHost(), address.getPort(), disconnectedNodes.contains(address), NodeType.SLAVE); @@ -94,7 +95,7 @@ public class MasterSlaveEntry { return result; } - public Future setupMasterEntry(String host, int port) { + public RFuture setupMasterEntry(String host, int port) { RedisClient client = connectionManager.createClient(NodeType.MASTER, host, port); masterEntry = new ClientConnectionsEntry(client, config.getMasterConnectionMinimumIdleSize(), config.getMasterConnectionPoolSize(), 0, 0, connectionManager, NodeType.MASTER); @@ -180,7 +181,7 @@ public class MasterSlaveEntry { return; } - Future subscribeFuture = connectionManager.subscribe(subscribeCodec, channelName, null); + RFuture subscribeFuture = connectionManager.subscribe(subscribeCodec, channelName, null); subscribeFuture.addListener(new FutureListener() { @Override @@ -203,7 +204,7 @@ public class MasterSlaveEntry { final Collection> listeners) { Codec subscribeCodec = connectionManager.punsubscribe(channelName); if (!listeners.isEmpty()) { - Future future = connectionManager.psubscribe(channelName, subscribeCodec, null); + RFuture future = connectionManager.psubscribe(channelName, subscribeCodec, null); future.addListener(new FutureListener() { @Override public void operationComplete(Future future) @@ -231,7 +232,7 @@ public class MasterSlaveEntry { return; } - Future newConnection = connectionReadOp(); + RFuture newConnection = connectionReadOp(); newConnection.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -268,11 +269,11 @@ public class MasterSlaveEntry { }); } - public Future addSlave(String host, int port) { + public RFuture addSlave(String host, int port) { return addSlave(host, port, true, NodeType.SLAVE); } - private Future addSlave(String host, int port, boolean freezed, NodeType mode) { + private RFuture addSlave(String host, int port, boolean freezed, NodeType mode) { RedisClient client = connectionManager.createClient(NodeType.SLAVE, host, port); ClientConnectionsEntry entry = new ClientConnectionsEntry(client, this.config.getSlaveConnectionMinimumIdleSize(), @@ -356,20 +357,19 @@ public class MasterSlaveEntry { slaveBalancer.shutdownAsync(); } - public Future connectionWriteOp() { + public RFuture connectionWriteOp() { return writeConnectionHolder.get(); } - public Future connectionReadOp() { + public RFuture connectionReadOp() { return slaveBalancer.nextConnection(); } - public Future connectionReadOp(InetSocketAddress addr) { + public RFuture connectionReadOp(InetSocketAddress addr) { return slaveBalancer.getConnection(addr); } - - Future nextPubSubConnection() { + RFuture nextPubSubConnection() { return slaveBalancer.nextPubSubConnection(); } diff --git a/redisson/src/main/java/org/redisson/connection/RedisClientEntry.java b/redisson/src/main/java/org/redisson/connection/RedisClientEntry.java index eed80a2f8..f0c3dc2b5 100644 --- a/redisson/src/main/java/org/redisson/connection/RedisClientEntry.java +++ b/redisson/src/main/java/org/redisson/connection/RedisClientEntry.java @@ -15,16 +15,15 @@ */ package org.redisson.connection; -import io.netty.util.concurrent.Promise; +import java.net.InetSocketAddress; +import java.util.Map; import org.redisson.api.ClusterNode; import org.redisson.api.NodeType; import org.redisson.client.RedisClient; import org.redisson.client.RedisConnection; import org.redisson.client.protocol.RedisCommands; - -import java.net.InetSocketAddress; -import java.util.Map; +import org.redisson.misc.RPromise; public class RedisClientEntry implements ClusterNode { @@ -55,7 +54,7 @@ public class RedisClientEntry implements ClusterNode { private RedisConnection connect() { RedisConnection c = client.connect(); - Promise future = manager.newPromise(); + RPromise future = manager.newPromise(); manager.getConnectListener().onConnect(future, c, null, manager.getConfig()); future.syncUninterruptibly(); return future.getNow(); diff --git a/redisson/src/main/java/org/redisson/connection/SentinelConnectionManager.java b/redisson/src/main/java/org/redisson/connection/SentinelConnectionManager.java index 8a7d0bda1..227ab8a82 100755 --- a/redisson/src/main/java/org/redisson/connection/SentinelConnectionManager.java +++ b/redisson/src/main/java/org/redisson/connection/SentinelConnectionManager.java @@ -25,6 +25,7 @@ import java.util.Set; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference; +import org.redisson.api.RFuture; import org.redisson.client.BaseRedisPubSubListener; import org.redisson.client.RedisClient; import org.redisson.client.RedisConnection; @@ -115,13 +116,13 @@ public class SentinelConnectionManager extends MasterSlaveConnectionManager { } init(c); - List> connectionFutures = new ArrayList>(cfg.getSentinelAddresses().size()); + List> connectionFutures = new ArrayList>(cfg.getSentinelAddresses().size()); for (URI addr : cfg.getSentinelAddresses()) { - Future future = registerSentinel(cfg, addr, c); + RFuture future = registerSentinel(cfg, addr, c); connectionFutures.add(future); } - for (Future future : connectionFutures) { + for (RFuture future : connectionFutures) { future.awaitUninterruptibly(); } } @@ -130,23 +131,23 @@ public class SentinelConnectionManager extends MasterSlaveConnectionManager { protected MasterSlaveEntry createMasterSlaveEntry(MasterSlaveServersConfig config, HashSet slots) { MasterSlaveEntry entry = new MasterSlaveEntry(slots, this, config); - List> fs = entry.initSlaveBalancer(disconnectedSlaves); - for (Future future : fs) { + List> fs = entry.initSlaveBalancer(disconnectedSlaves); + for (RFuture future : fs) { future.syncUninterruptibly(); } - Future f = entry.setupMasterEntry(config.getMasterAddress().getHost(), config.getMasterAddress().getPort()); + RFuture f = entry.setupMasterEntry(config.getMasterAddress().getHost(), config.getMasterAddress().getPort()); f.syncUninterruptibly(); return entry; } - private Future registerSentinel(final SentinelServersConfig cfg, final URI addr, final MasterSlaveServersConfig c) { + private RFuture registerSentinel(final SentinelServersConfig cfg, final URI addr, final MasterSlaveServersConfig c) { RedisClient client = createClient(addr.getHost(), addr.getPort(), c.getConnectTimeout(), c.getRetryInterval() * c.getRetryAttempts()); RedisClient oldClient = sentinels.putIfAbsent(addr.getHost() + ":" + addr.getPort(), client); if (oldClient != null) { return newSucceededFuture(null); } - Future pubsubFuture = client.connectPubSubAsync(); + RFuture pubsubFuture = client.connectPubSubAsync(); pubsubFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -218,7 +219,7 @@ public class SentinelConnectionManager extends MasterSlaveConnectionManager { // to avoid addition twice if (slaves.putIfAbsent(slaveAddr, true) == null && config.getReadMode() != ReadMode.MASTER) { - Future future = getEntry(singleSlotRange.getStartSlot()).addSlave(ip, Integer.valueOf(port)); + RFuture future = getEntry(singleSlotRange.getStartSlot()).addSlave(ip, Integer.valueOf(port)); future.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { diff --git a/redisson/src/main/java/org/redisson/connection/SingleEntry.java b/redisson/src/main/java/org/redisson/connection/SingleEntry.java index 85ca21e7a..a474a125b 100644 --- a/redisson/src/main/java/org/redisson/connection/SingleEntry.java +++ b/redisson/src/main/java/org/redisson/connection/SingleEntry.java @@ -20,6 +20,7 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import org.redisson.api.NodeType; +import org.redisson.api.RFuture; import org.redisson.client.RedisClient; import org.redisson.client.RedisConnection; import org.redisson.client.RedisPubSubConnection; @@ -27,10 +28,10 @@ import org.redisson.cluster.ClusterSlotRange; import org.redisson.config.MasterSlaveServersConfig; import org.redisson.connection.pool.PubSubConnectionPool; import org.redisson.connection.pool.SinglePubSubConnectionPool; +import org.redisson.misc.RPromise; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; -import io.netty.util.concurrent.Promise; public class SingleEntry extends MasterSlaveEntry { @@ -42,16 +43,16 @@ public class SingleEntry extends MasterSlaveEntry { } @Override - public Future setupMasterEntry(String host, int port) { + public RFuture setupMasterEntry(String host, int port) { RedisClient masterClient = connectionManager.createClient(NodeType.MASTER, host, port); masterEntry = new ClientConnectionsEntry(masterClient, config.getMasterConnectionMinimumIdleSize(), config.getMasterConnectionPoolSize(), config.getSlaveConnectionMinimumIdleSize(), config.getSlaveSubscriptionConnectionPoolSize(), connectionManager, NodeType.MASTER); - final Promise res = connectionManager.newPromise(); - Future f = writeConnectionHolder.add(masterEntry); - Future s = pubSubConnectionHolder.add(masterEntry); + final RPromise res = connectionManager.newPromise(); + RFuture f = writeConnectionHolder.add(masterEntry); + RFuture s = pubSubConnectionHolder.add(masterEntry); FutureListener listener = new FutureListener() { AtomicInteger counter = new AtomicInteger(2); @Override @@ -71,7 +72,7 @@ public class SingleEntry extends MasterSlaveEntry { } @Override - Future nextPubSubConnection() { + RFuture nextPubSubConnection() { return pubSubConnectionHolder.get(); } @@ -81,12 +82,12 @@ public class SingleEntry extends MasterSlaveEntry { } @Override - public Future connectionReadOp(InetSocketAddress addr) { + public RFuture connectionReadOp(InetSocketAddress addr) { return super.connectionWriteOp(); } @Override - public Future connectionReadOp() { + public RFuture connectionReadOp() { return super.connectionWriteOp(); } diff --git a/redisson/src/main/java/org/redisson/connection/balancer/LoadBalancerManager.java b/redisson/src/main/java/org/redisson/connection/balancer/LoadBalancerManager.java index a3342ab90..e80ed8cad 100644 --- a/redisson/src/main/java/org/redisson/connection/balancer/LoadBalancerManager.java +++ b/redisson/src/main/java/org/redisson/connection/balancer/LoadBalancerManager.java @@ -17,16 +17,15 @@ package org.redisson.connection.balancer; import java.net.InetSocketAddress; +import org.redisson.api.RFuture; import org.redisson.client.RedisConnection; import org.redisson.client.RedisPubSubConnection; import org.redisson.connection.ClientConnectionsEntry; import org.redisson.connection.ClientConnectionsEntry.FreezeReason; -import io.netty.util.concurrent.Future; - public interface LoadBalancerManager { - Future getConnection(InetSocketAddress addr); + RFuture getConnection(InetSocketAddress addr); int getAvailableClients(); @@ -40,11 +39,11 @@ public interface LoadBalancerManager { ClientConnectionsEntry freeze(String host, int port, FreezeReason freezeReason); - Future add(ClientConnectionsEntry entry); + RFuture add(ClientConnectionsEntry entry); - Future nextConnection(); + RFuture nextConnection(); - Future nextPubSubConnection(); + RFuture nextPubSubConnection(); void returnConnection(RedisConnection connection); diff --git a/redisson/src/main/java/org/redisson/connection/balancer/LoadBalancerManagerImpl.java b/redisson/src/main/java/org/redisson/connection/balancer/LoadBalancerManagerImpl.java index 921c8b0dd..e39020e54 100644 --- a/redisson/src/main/java/org/redisson/connection/balancer/LoadBalancerManagerImpl.java +++ b/redisson/src/main/java/org/redisson/connection/balancer/LoadBalancerManagerImpl.java @@ -19,6 +19,7 @@ import java.net.InetSocketAddress; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; +import org.redisson.api.RFuture; import org.redisson.client.RedisConnection; import org.redisson.client.RedisConnectionException; import org.redisson.client.RedisPubSubConnection; @@ -29,12 +30,12 @@ import org.redisson.connection.ConnectionManager; import org.redisson.connection.MasterSlaveEntry; import org.redisson.connection.pool.PubSubConnectionPool; import org.redisson.connection.pool.SlaveConnectionPool; +import org.redisson.misc.RPromise; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; -import io.netty.util.concurrent.Promise; import io.netty.util.internal.PlatformDependent; public class LoadBalancerManagerImpl implements LoadBalancerManager { @@ -52,8 +53,8 @@ public class LoadBalancerManagerImpl implements LoadBalancerManager { pubSubConnectionPool = new PubSubConnectionPool(config, connectionManager, entry); } - public Future add(final ClientConnectionsEntry entry) { - final Promise result = connectionManager.newPromise(); + public RFuture add(final ClientConnectionsEntry entry) { + final RPromise result = connectionManager.newPromise(); FutureListener listener = new FutureListener() { AtomicInteger counter = new AtomicInteger(2); @Override @@ -69,9 +70,9 @@ public class LoadBalancerManagerImpl implements LoadBalancerManager { } }; - Future slaveFuture = slaveConnectionPool.add(entry); + RFuture slaveFuture = slaveConnectionPool.add(entry); slaveFuture.addListener(listener); - Future pubSubFuture = pubSubConnectionPool.add(entry); + RFuture pubSubFuture = pubSubConnectionPool.add(entry); pubSubFuture.addListener(listener); return result; } @@ -136,11 +137,11 @@ public class LoadBalancerManagerImpl implements LoadBalancerManager { return connectionEntry; } - public Future nextPubSubConnection() { + public RFuture nextPubSubConnection() { return pubSubConnectionPool.get(); } - public Future getConnection(InetSocketAddress addr) { + public RFuture getConnection(InetSocketAddress addr) { ClientConnectionsEntry entry = addr2Entry.get(addr); if (entry != null) { return slaveConnectionPool.get(entry); @@ -149,7 +150,7 @@ public class LoadBalancerManagerImpl implements LoadBalancerManager { return connectionManager.newFailedFuture(exception); } - public Future nextConnection() { + public RFuture nextConnection() { return slaveConnectionPool.get(); } diff --git a/redisson/src/main/java/org/redisson/connection/pool/ConnectionPool.java b/redisson/src/main/java/org/redisson/connection/pool/ConnectionPool.java index cf1ad64d0..f0bb65715 100644 --- a/redisson/src/main/java/org/redisson/connection/pool/ConnectionPool.java +++ b/redisson/src/main/java/org/redisson/connection/pool/ConnectionPool.java @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import org.redisson.api.NodeType; +import org.redisson.api.RFuture; import org.redisson.client.RedisConnection; import org.redisson.client.RedisConnectionException; import org.redisson.client.protocol.RedisCommands; @@ -31,6 +32,7 @@ import org.redisson.connection.ClientConnectionsEntry; import org.redisson.connection.ClientConnectionsEntry.FreezeReason; import org.redisson.connection.ConnectionManager; import org.redisson.connection.MasterSlaveEntry; +import org.redisson.misc.RPromise; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,7 +40,6 @@ import io.netty.util.Timeout; import io.netty.util.TimerTask; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; -import io.netty.util.concurrent.Promise; abstract class ConnectionPool { @@ -58,8 +59,8 @@ abstract class ConnectionPool { this.connectionManager = connectionManager; } - public Future add(final ClientConnectionsEntry entry) { - final Promise promise = connectionManager.newPromise(); + public RFuture add(final ClientConnectionsEntry entry) { + final RPromise promise = connectionManager.newPromise(); promise.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -70,7 +71,7 @@ abstract class ConnectionPool { return promise; } - private void initConnections(final ClientConnectionsEntry entry, final Promise initPromise, boolean checkFreezed) { + private void initConnections(final ClientConnectionsEntry entry, final RPromise initPromise, boolean checkFreezed) { final int minimumIdleSize = getMinimumIdleSize(entry); if (minimumIdleSize == 0 || (checkFreezed && entry.isFreezed())) { @@ -86,7 +87,7 @@ abstract class ConnectionPool { } } - private void createConnection(final boolean checkFreezed, final AtomicInteger requests, final ClientConnectionsEntry entry, final Promise initPromise, + private void createConnection(final boolean checkFreezed, final AtomicInteger requests, final ClientConnectionsEntry entry, final RPromise initPromise, final int minimumIdleSize, final AtomicInteger initializedConnections) { if ((checkFreezed && entry.isFreezed()) || !tryAcquireConnection(entry)) { @@ -97,7 +98,7 @@ abstract class ConnectionPool { return; } - Future promise = createConnection(entry); + RFuture promise = createConnection(entry); promise.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -135,7 +136,7 @@ abstract class ConnectionPool { return config.getLoadBalancer().getEntry(entries); } - public Future get() { + public RFuture get() { for (int j = entries.size() - 1; j >= 0; j--) { ClientConnectionsEntry entry = getEntry(); if (!entry.isFreezed() && tryAcquireConnection(entry)) { @@ -165,7 +166,7 @@ abstract class ConnectionPool { return connectionManager.newFailedFuture(exception); } - public Future get(ClientConnectionsEntry entry) { + public RFuture get(ClientConnectionsEntry entry) { if (((entry.getNodeType() == NodeType.MASTER && entry.getFreezeReason() == FreezeReason.SYSTEM) || !entry.isFreezed()) && tryAcquireConnection(entry)) { return connectTo(entry); @@ -184,11 +185,11 @@ abstract class ConnectionPool { return (T) entry.pollConnection(); } - protected Future connect(ClientConnectionsEntry entry) { - return (Future) entry.connect(); + protected RFuture connect(ClientConnectionsEntry entry) { + return (RFuture) entry.connect(); } - private Future connectTo(ClientConnectionsEntry entry) { + private RFuture connectTo(ClientConnectionsEntry entry) { T conn = poll(entry); if (conn != null) { if (!conn.isActive()) { @@ -201,9 +202,9 @@ abstract class ConnectionPool { return createConnection(entry); } - private Future createConnection(final ClientConnectionsEntry entry) { - final Promise promise = connectionManager.newPromise(); - Future connFuture = connect(entry); + private RFuture createConnection(final ClientConnectionsEntry entry) { + final RPromise promise = connectionManager.newPromise(); + RFuture connFuture = connect(entry); connFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -226,7 +227,7 @@ abstract class ConnectionPool { return promise; } - private void promiseSuccessful(ClientConnectionsEntry entry, Promise promise, T conn) { + private void promiseSuccessful(ClientConnectionsEntry entry, RPromise promise, T conn) { entry.resetFailedAttempts(); if (!promise.trySuccess(conn)) { releaseConnection(entry, conn); @@ -234,12 +235,12 @@ abstract class ConnectionPool { } } - private Future promiseSuccessful(ClientConnectionsEntry entry, T conn) { + private RFuture promiseSuccessful(ClientConnectionsEntry entry, T conn) { entry.resetFailedAttempts(); - return (Future) conn.getAcquireFuture(); + return (RFuture) conn.getAcquireFuture(); } - private void promiseFailure(ClientConnectionsEntry entry, Promise promise, Throwable cause) { + private void promiseFailure(ClientConnectionsEntry entry, RPromise promise, Throwable cause) { if (entry.incFailedAttempts() == config.getFailedAttempts()) { checkForReconnect(entry); } @@ -247,7 +248,7 @@ abstract class ConnectionPool { promise.tryFailure(cause); } - private void promiseFailure(ClientConnectionsEntry entry, Promise promise, T conn) { + private void promiseFailure(ClientConnectionsEntry entry, RPromise promise, T conn) { int attempts = entry.incFailedAttempts(); if (attempts == config.getFailedAttempts()) { checkForReconnect(entry); @@ -261,7 +262,7 @@ abstract class ConnectionPool { promise.tryFailure(cause); } - private Future promiseFailure(ClientConnectionsEntry entry, T conn) { + private RFuture promiseFailure(ClientConnectionsEntry entry, T conn) { int attempts = entry.incFailedAttempts(); if (attempts == config.getFailedAttempts()) { checkForReconnect(entry); @@ -301,7 +302,7 @@ abstract class ConnectionPool { return; } - Future connectionFuture = entry.getClient().connectAsync(); + RFuture connectionFuture = entry.getClient().connectAsync(); connectionFuture.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { @@ -332,7 +333,7 @@ abstract class ConnectionPool { if (future.isSuccess() && "PONG".equals(future.getNow())) { entry.resetFailedAttempts(); - Promise promise = connectionManager.newPromise(); + RPromise promise = connectionManager.newPromise(); promise.addListener(new FutureListener() { @Override public void operationComplete(Future future) @@ -362,7 +363,7 @@ abstract class ConnectionPool { }; if (entry.getConfig().getPassword() != null) { - Future temp = c.async(RedisCommands.AUTH, config.getPassword()); + RFuture temp = c.async(RedisCommands.AUTH, config.getPassword()); FutureListener listener = new FutureListener () { @Override public void operationComplete (Future < Void > future)throws Exception { @@ -381,7 +382,7 @@ abstract class ConnectionPool { } private void ping(RedisConnection c, final FutureListener pingListener) { - Future f = c.async(RedisCommands.PING); + RFuture f = c.async(RedisCommands.PING); f.addListener(pingListener); } diff --git a/redisson/src/main/java/org/redisson/connection/pool/PubSubConnectionPool.java b/redisson/src/main/java/org/redisson/connection/pool/PubSubConnectionPool.java index 7efd5d0f0..b0d7de32d 100644 --- a/redisson/src/main/java/org/redisson/connection/pool/PubSubConnectionPool.java +++ b/redisson/src/main/java/org/redisson/connection/pool/PubSubConnectionPool.java @@ -15,13 +15,12 @@ */ package org.redisson.connection.pool; +import org.redisson.api.RFuture; import org.redisson.client.RedisPubSubConnection; import org.redisson.config.MasterSlaveServersConfig; +import org.redisson.connection.ClientConnectionsEntry; import org.redisson.connection.ConnectionManager; import org.redisson.connection.MasterSlaveEntry; -import org.redisson.connection.ClientConnectionsEntry; - -import io.netty.util.concurrent.Future; public class PubSubConnectionPool extends ConnectionPool { @@ -40,7 +39,7 @@ public class PubSubConnectionPool extends ConnectionPool } @Override - protected Future connect(ClientConnectionsEntry entry) { + protected RFuture connect(ClientConnectionsEntry entry) { return entry.connectPubSub(); } diff --git a/redisson/src/main/java/org/redisson/executor/ExecutorRemoteService.java b/redisson/src/main/java/org/redisson/executor/ExecutorRemoteService.java index 249badd38..e61ec5467 100644 --- a/redisson/src/main/java/org/redisson/executor/ExecutorRemoteService.java +++ b/redisson/src/main/java/org/redisson/executor/ExecutorRemoteService.java @@ -26,11 +26,11 @@ import org.redisson.client.codec.Codec; import org.redisson.client.codec.LongCodec; import org.redisson.client.protocol.RedisCommands; import org.redisson.command.CommandExecutor; +import org.redisson.misc.RPromise; import org.redisson.remote.RemoteServiceRequest; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; -import io.netty.util.concurrent.Promise; /** * @@ -60,10 +60,10 @@ public class ExecutorRemoteService extends BaseRemoteService { } @Override - protected final Future addAsync(RBlockingQueue requestQueue, + protected final RFuture addAsync(RBlockingQueue requestQueue, RemoteServiceRequest request, RemotePromise result) { - final Promise promise = commandExecutor.getConnectionManager().newPromise(); - Future future = addAsync(requestQueue, request); + final RPromise promise = commandExecutor.getConnectionManager().newPromise(); + RFuture future = addAsync(requestQueue, request); result.setAddFuture(future); future.addListener(new FutureListener() { diff --git a/redisson/src/main/java/org/redisson/executor/RedissonScheduledFuture.java b/redisson/src/main/java/org/redisson/executor/RedissonScheduledFuture.java index 38f3004bb..4dbf1a7f9 100644 --- a/redisson/src/main/java/org/redisson/executor/RedissonScheduledFuture.java +++ b/redisson/src/main/java/org/redisson/executor/RedissonScheduledFuture.java @@ -19,7 +19,7 @@ import java.util.concurrent.Delayed; import java.util.concurrent.TimeUnit; import org.redisson.api.RScheduledFuture; -import org.redisson.misc.RedissonFuture; +import org.redisson.misc.PromiseDelegator; /** * @@ -27,7 +27,7 @@ import org.redisson.misc.RedissonFuture; * * @param */ -public class RedissonScheduledFuture extends RedissonFuture implements RScheduledFuture { +public class RedissonScheduledFuture extends PromiseDelegator implements RScheduledFuture { private final long scheduledExecutionTime; private final String taskId; diff --git a/redisson/src/main/java/org/redisson/executor/RemotePromise.java b/redisson/src/main/java/org/redisson/executor/RemotePromise.java index 4dc5f7fba..62781401e 100644 --- a/redisson/src/main/java/org/redisson/executor/RemotePromise.java +++ b/redisson/src/main/java/org/redisson/executor/RemotePromise.java @@ -15,10 +15,9 @@ */ package org.redisson.executor; +import org.redisson.api.RFuture; import org.redisson.misc.PromiseDelegator; - -import io.netty.util.concurrent.Future; -import io.netty.util.concurrent.Promise; +import org.redisson.misc.RPromise; /** * @@ -28,9 +27,9 @@ import io.netty.util.concurrent.Promise; public class RemotePromise extends PromiseDelegator { private String requestId; - private Future addFuture; + private RFuture addFuture; - public RemotePromise(Promise promise) { + public RemotePromise(RPromise promise) { super(promise); } @@ -41,10 +40,10 @@ public class RemotePromise extends PromiseDelegator { return requestId; } - public void setAddFuture(Future addFuture) { + public void setAddFuture(RFuture addFuture) { this.addFuture = addFuture; } - public Future getAddFuture() { + public RFuture getAddFuture() { return addFuture; } diff --git a/redisson/src/main/java/org/redisson/misc/PromiseDelegator.java b/redisson/src/main/java/org/redisson/misc/PromiseDelegator.java index ee8e88534..9182b5a09 100644 --- a/redisson/src/main/java/org/redisson/misc/PromiseDelegator.java +++ b/redisson/src/main/java/org/redisson/misc/PromiseDelegator.java @@ -19,24 +19,23 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import io.netty.util.concurrent.Future; -import io.netty.util.concurrent.GenericFutureListener; +import io.netty.util.concurrent.FutureListener; import io.netty.util.concurrent.Promise; public class PromiseDelegator implements RPromise { - private final Promise promise; + private final RPromise promise; - public PromiseDelegator(Promise promise) { + public PromiseDelegator(RPromise promise) { super(); this.promise = promise; } - public Promise getInnerPromise() { + public RPromise getInnerPromise() { return promise; } - public Promise setSuccess(T result) { + public RPromise setSuccess(T result) { return promise.setSuccess(result); } @@ -68,35 +67,35 @@ public class PromiseDelegator implements RPromise { return promise.setUncancellable(); } - public Promise addListener(GenericFutureListener> listener) { + public RPromise addListener(FutureListener listener) { return promise.addListener(listener); } - public Promise addListeners(GenericFutureListener>... listeners) { + public RPromise addListeners(FutureListener... listeners) { return promise.addListeners(listeners); } - public Promise removeListener(GenericFutureListener> listener) { + public RPromise removeListener(FutureListener listener) { return promise.removeListener(listener); } - public Promise removeListeners(GenericFutureListener>... listeners) { + public RPromise removeListeners(FutureListener... listeners) { return promise.removeListeners(listeners); } - public Promise await() throws InterruptedException { + public RPromise await() throws InterruptedException { return promise.await(); } - public Promise awaitUninterruptibly() { + public RPromise awaitUninterruptibly() { return promise.awaitUninterruptibly(); } - public Promise sync() throws InterruptedException { + public RPromise sync() throws InterruptedException { return promise.sync(); } - public Promise syncUninterruptibly() { + public RPromise syncUninterruptibly() { return promise.syncUninterruptibly(); } diff --git a/redisson/src/main/java/org/redisson/misc/RPromise.java b/redisson/src/main/java/org/redisson/misc/RPromise.java index 5050f7cfc..100907e78 100644 --- a/redisson/src/main/java/org/redisson/misc/RPromise.java +++ b/redisson/src/main/java/org/redisson/misc/RPromise.java @@ -17,6 +17,7 @@ package org.redisson.misc; import org.redisson.api.RFuture; +import io.netty.util.concurrent.FutureListener; import io.netty.util.concurrent.Promise; /** @@ -25,6 +26,74 @@ import io.netty.util.concurrent.Promise; * * @param */ -public interface RPromise extends Promise, RFuture { +public interface RPromise extends RFuture { + + /** + * Marks this future as a success and notifies all + * listeners. + * + * If it is success or failed already it will throw an {@link IllegalStateException}. + */ + RPromise setSuccess(T result); + + /** + * Marks this future as a success and notifies all + * listeners. + * + * @return {@code true} if and only if successfully marked this future as + * a success. Otherwise {@code false} because this future is + * already marked as either a success or a failure. + */ + boolean trySuccess(T result); + + /** + * Marks this future as a failure and notifies all + * listeners. + * + * If it is success or failed already it will throw an {@link IllegalStateException}. + */ + Promise setFailure(Throwable cause); + + /** + * Marks this future as a failure and notifies all + * listeners. + * + * @return {@code true} if and only if successfully marked this future as + * a failure. Otherwise {@code false} because this future is + * already marked as either a success or a failure. + */ + boolean tryFailure(Throwable cause); + + /** + * Make this future impossible to cancel. + * + * @return {@code true} if and only if successfully marked this future as uncancellable or it is already done + * without being cancelled. {@code false} if this future has been cancelled already. + */ + boolean setUncancellable(); + + @Override + RPromise addListener(FutureListener listener); + + @Override + RPromise addListeners(FutureListener... listeners); + + @Override + RPromise removeListener(FutureListener listener); + + @Override + RPromise removeListeners(FutureListener... listeners); + + @Override + RPromise await() throws InterruptedException; + + @Override + RPromise awaitUninterruptibly(); + + @Override + RPromise sync() throws InterruptedException; + + @Override + RPromise syncUninterruptibly(); } diff --git a/redisson/src/main/java/org/redisson/misc/RedissonFuture.java b/redisson/src/main/java/org/redisson/misc/RedissonFuture.java index cff4d2d78..e41bce293 100644 --- a/redisson/src/main/java/org/redisson/misc/RedissonFuture.java +++ b/redisson/src/main/java/org/redisson/misc/RedissonFuture.java @@ -22,7 +22,7 @@ import java.util.concurrent.TimeoutException; import org.redisson.api.RFuture; import io.netty.util.concurrent.Future; -import io.netty.util.concurrent.GenericFutureListener; +import io.netty.util.concurrent.FutureListener; /** * @@ -55,36 +55,44 @@ public class RedissonFuture implements RFuture { return future.cause(); } - public Future addListener(GenericFutureListener> listener) { - return future.addListener(listener); + public RFuture addListener(FutureListener listener) { + future.addListener(listener); + return this; } - public Future addListeners(GenericFutureListener>... listeners) { - return future.addListeners(listeners); + public RFuture addListeners(FutureListener... listeners) { + future.addListeners(listeners); + return this; } - public Future removeListener(GenericFutureListener> listener) { - return future.removeListener(listener); + public RFuture removeListener(FutureListener listener) { + future.removeListener(listener); + return this; } - public Future removeListeners(GenericFutureListener>... listeners) { - return future.removeListeners(listeners); + public RFuture removeListeners(FutureListener... listeners) { + future.removeListeners(listeners); + return this; } - public Future sync() throws InterruptedException { - return future.sync(); + public RFuture sync() throws InterruptedException { + future.sync(); + return this; } - public Future syncUninterruptibly() { - return future.syncUninterruptibly(); + public RFuture syncUninterruptibly() { + future.syncUninterruptibly(); + return this; } - public Future await() throws InterruptedException { - return future.await(); + public RFuture await() throws InterruptedException { + future.await(); + return this; } - public Future awaitUninterruptibly() { - return future.awaitUninterruptibly(); + public RFuture awaitUninterruptibly() { + future.awaitUninterruptibly(); + return this; } public boolean await(long timeout, TimeUnit unit) throws InterruptedException { diff --git a/redisson/src/main/java/org/redisson/misc/RedissonPromise.java b/redisson/src/main/java/org/redisson/misc/RedissonPromise.java index 814395e47..d0c2c2902 100644 --- a/redisson/src/main/java/org/redisson/misc/RedissonPromise.java +++ b/redisson/src/main/java/org/redisson/misc/RedissonPromise.java @@ -15,6 +15,11 @@ */ package org.redisson.misc; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import io.netty.util.concurrent.FutureListener; import io.netty.util.concurrent.Promise; /** @@ -23,10 +28,155 @@ import io.netty.util.concurrent.Promise; * * @param */ -public class RedissonPromise extends PromiseDelegator implements RPromise { +public class RedissonPromise implements RPromise { + private final Promise promise; + public RedissonPromise(Promise promise) { - super(promise); + this.promise = promise; + } + + public Promise getInnerPromise() { + return promise; + } + + @Override + public RPromise setSuccess(T result) { + promise.setSuccess(result); + return this; + } + + @Override + public boolean isSuccess() { + return promise.isSuccess(); + } + + @Override + public boolean trySuccess(T result) { + return promise.trySuccess(result); + } + + @Override + public boolean isCancellable() { + return promise.isCancellable(); + } + + @Override + public Throwable cause() { + return promise.cause(); + } + + @Override + public Promise setFailure(Throwable cause) { + return promise.setFailure(cause); + } + + @Override + public boolean tryFailure(Throwable cause) { + return promise.tryFailure(cause); + } + + @Override + public boolean setUncancellable() { + return promise.setUncancellable(); + } + + @Override + public RPromise addListener(FutureListener listener) { + promise.addListener(listener); + return this; + } + + @Override + public RPromise addListeners(FutureListener... listeners) { + promise.addListeners(listeners); + return this; + } + + @Override + public RPromise removeListener(FutureListener listener) { + promise.removeListener(listener); + return this; + } + + @Override + public RPromise removeListeners(FutureListener... listeners) { + promise.removeListeners(listeners); + return this; + } + + @Override + public RPromise await() throws InterruptedException { + promise.await(); + return this; + } + + @Override + public RPromise awaitUninterruptibly() { + promise.awaitUninterruptibly(); + return this; + } + + @Override + public RPromise sync() throws InterruptedException { + promise.sync(); + return this; + } + + @Override + public RPromise syncUninterruptibly() { + promise.syncUninterruptibly(); + return this; + } + + @Override + public boolean await(long timeout, TimeUnit unit) throws InterruptedException { + return promise.await(timeout, unit); + } + + @Override + public boolean isCancelled() { + return promise.isCancelled(); + } + + @Override + public boolean isDone() { + return promise.isDone(); + } + + @Override + public boolean await(long timeoutMillis) throws InterruptedException { + return promise.await(timeoutMillis); + } + + @Override + public T get() throws InterruptedException, ExecutionException { + return promise.get(); + } + + @Override + public boolean awaitUninterruptibly(long timeout, TimeUnit unit) { + return promise.awaitUninterruptibly(timeout, unit); + } + + @Override + public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + return promise.get(timeout, unit); + } + + @Override + public boolean awaitUninterruptibly(long timeoutMillis) { + return promise.awaitUninterruptibly(timeoutMillis); + } + + @Override + public T getNow() { + return promise.getNow(); + } + + @Override + public boolean cancel(boolean mayInterruptIfRunning) { + return promise.cancel(mayInterruptIfRunning); } } diff --git a/redisson/src/main/java/org/redisson/pubsub/CountDownLatchPubSub.java b/redisson/src/main/java/org/redisson/pubsub/CountDownLatchPubSub.java index 998bfd6d4..11ab3dd9a 100644 --- a/redisson/src/main/java/org/redisson/pubsub/CountDownLatchPubSub.java +++ b/redisson/src/main/java/org/redisson/pubsub/CountDownLatchPubSub.java @@ -17,13 +17,12 @@ package org.redisson.pubsub; import org.redisson.RedissonCountDownLatch; import org.redisson.RedissonCountDownLatchEntry; - -import io.netty.util.concurrent.Promise; +import org.redisson.misc.RPromise; public class CountDownLatchPubSub extends PublishSubscribe { @Override - protected RedissonCountDownLatchEntry createEntry(Promise newPromise) { + protected RedissonCountDownLatchEntry createEntry(RPromise newPromise) { return new RedissonCountDownLatchEntry(newPromise); } diff --git a/redisson/src/main/java/org/redisson/pubsub/LockPubSub.java b/redisson/src/main/java/org/redisson/pubsub/LockPubSub.java index 04ed5b47c..cbc572741 100644 --- a/redisson/src/main/java/org/redisson/pubsub/LockPubSub.java +++ b/redisson/src/main/java/org/redisson/pubsub/LockPubSub.java @@ -16,15 +16,14 @@ package org.redisson.pubsub; import org.redisson.RedissonLockEntry; - -import io.netty.util.concurrent.Promise; +import org.redisson.misc.RPromise; public class LockPubSub extends PublishSubscribe { public static final Long unlockMessage = 0L; @Override - protected RedissonLockEntry createEntry(Promise newPromise) { + protected RedissonLockEntry createEntry(RPromise newPromise) { return new RedissonLockEntry(newPromise); } diff --git a/redisson/src/main/java/org/redisson/pubsub/PublishSubscribe.java b/redisson/src/main/java/org/redisson/pubsub/PublishSubscribe.java index eeb60fd75..801902d46 100644 --- a/redisson/src/main/java/org/redisson/pubsub/PublishSubscribe.java +++ b/redisson/src/main/java/org/redisson/pubsub/PublishSubscribe.java @@ -19,15 +19,15 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference; import org.redisson.PubSubEntry; +import org.redisson.api.RFuture; import org.redisson.client.BaseRedisPubSubListener; import org.redisson.client.RedisPubSubListener; import org.redisson.client.codec.LongCodec; import org.redisson.client.protocol.pubsub.PubSubType; import org.redisson.connection.ConnectionManager; import org.redisson.misc.PromiseDelegator; +import org.redisson.misc.RPromise; -import io.netty.util.concurrent.Future; -import io.netty.util.concurrent.Promise; import io.netty.util.internal.PlatformDependent; abstract class PublishSubscribe> { @@ -58,10 +58,10 @@ abstract class PublishSubscribe> { return entries.get(entryName); } - public Future subscribe(final String entryName, final String channelName, final ConnectionManager connectionManager) { + public RFuture subscribe(final String entryName, final String channelName, final ConnectionManager connectionManager) { final AtomicReference listenerHolder = new AtomicReference(); final AsyncSemaphore semaphore = connectionManager.getSemaphore(channelName); - final Promise newPromise = new PromiseDelegator(connectionManager.newPromise()) { + final RPromise newPromise = new PromiseDelegator(connectionManager.newPromise()) { @Override public boolean cancel(boolean mayInterruptIfRunning) { return semaphore.remove(listenerHolder.get()); @@ -101,7 +101,7 @@ abstract class PublishSubscribe> { return newPromise; } - protected abstract E createEntry(Promise newPromise); + protected abstract E createEntry(RPromise newPromise); protected abstract void onMessage(E value, Long message); diff --git a/redisson/src/main/java/org/redisson/pubsub/SemaphorePubSub.java b/redisson/src/main/java/org/redisson/pubsub/SemaphorePubSub.java index 84ca12807..6fb3bde77 100644 --- a/redisson/src/main/java/org/redisson/pubsub/SemaphorePubSub.java +++ b/redisson/src/main/java/org/redisson/pubsub/SemaphorePubSub.java @@ -16,13 +16,12 @@ package org.redisson.pubsub; import org.redisson.RedissonLockEntry; - -import io.netty.util.concurrent.Promise; +import org.redisson.misc.RPromise; public class SemaphorePubSub extends PublishSubscribe { @Override - protected RedissonLockEntry createEntry(Promise newPromise) { + protected RedissonLockEntry createEntry(RPromise newPromise) { return new RedissonLockEntry(newPromise); } diff --git a/redisson/src/main/java/org/redisson/pubsub/TransferListener.java b/redisson/src/main/java/org/redisson/pubsub/TransferListener.java index d1cfb0031..52728ee12 100644 --- a/redisson/src/main/java/org/redisson/pubsub/TransferListener.java +++ b/redisson/src/main/java/org/redisson/pubsub/TransferListener.java @@ -15,9 +15,10 @@ */ package org.redisson.pubsub; +import org.redisson.misc.RPromise; + import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; -import io.netty.util.concurrent.Promise; /** * @@ -27,9 +28,9 @@ import io.netty.util.concurrent.Promise; */ public class TransferListener implements FutureListener { - private Promise promise; + private RPromise promise; - public TransferListener(Promise promise) { + public TransferListener(RPromise promise) { super(); this.promise = promise; } diff --git a/redisson/src/main/java/org/redisson/reactive/NettyFuturePublisher.java b/redisson/src/main/java/org/redisson/reactive/NettyFuturePublisher.java index 971a277e4..1bdda4f80 100644 --- a/redisson/src/main/java/org/redisson/reactive/NettyFuturePublisher.java +++ b/redisson/src/main/java/org/redisson/reactive/NettyFuturePublisher.java @@ -16,6 +16,7 @@ package org.redisson.reactive; import org.reactivestreams.Subscriber; +import org.redisson.api.RFuture; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; @@ -26,9 +27,9 @@ import reactor.rx.subscription.ReactiveSubscription; public class NettyFuturePublisher extends Stream { - private final Future that; + private final RFuture that; - public NettyFuturePublisher(Future that) { + public NettyFuturePublisher(RFuture that) { this.that = that; } diff --git a/redisson/src/main/java/org/redisson/reactive/RedissonListReactive.java b/redisson/src/main/java/org/redisson/reactive/RedissonListReactive.java index df4fc68b0..b9dd297b1 100644 --- a/redisson/src/main/java/org/redisson/reactive/RedissonListReactive.java +++ b/redisson/src/main/java/org/redisson/reactive/RedissonListReactive.java @@ -15,10 +15,8 @@ */ package org.redisson.reactive; -import static org.redisson.client.protocol.RedisCommands.EVAL_OBJECT; import static org.redisson.client.protocol.RedisCommands.LINDEX; import static org.redisson.client.protocol.RedisCommands.LLEN; -import static org.redisson.client.protocol.RedisCommands.LPOP; import static org.redisson.client.protocol.RedisCommands.LREM_SINGLE; import static org.redisson.client.protocol.RedisCommands.RPUSH; @@ -36,7 +34,6 @@ import org.redisson.client.codec.Codec; import org.redisson.client.protocol.RedisCommand; import org.redisson.client.protocol.RedisCommand.ValueType; import org.redisson.client.protocol.RedisCommands; -import org.redisson.client.protocol.convertor.Convertor; import org.redisson.client.protocol.convertor.LongReplayConvertor; import org.redisson.command.CommandReactiveExecutor; @@ -259,17 +256,7 @@ public class RedissonListReactive extends RedissonExpirableReactive implement @Override public Publisher remove(long index) { - if (index == 0) { - return commandExecutor.writeReactive(getName(), codec, LPOP, getName()); - } - - return commandExecutor.evalWriteReactive(getName(), codec, EVAL_OBJECT, - "local v = redis.call('lindex', KEYS[1], ARGV[1]); " + - "local tail = redis.call('lrange', KEYS[1], ARGV[1]);" + - "redis.call('ltrim', KEYS[1], 0, ARGV[1] - 1);" + - "for i, v in ipairs(tail) do redis.call('rpush', KEYS[1], v) end;" + - "return v", - Collections.singletonList(getName()), index); + return reactive(instance.removeAsync(index)); } @Override @@ -277,38 +264,14 @@ public class RedissonListReactive extends RedissonExpirableReactive implement return reactive(instance.containsAsync(o)); } - private Publisher indexOf(Object o, Convertor convertor) { - return commandExecutor.evalReadReactive(getName(), codec, new RedisCommand("EVAL", convertor, 4), - "local key = KEYS[1] " + - "local obj = ARGV[1] " + - "local items = redis.call('lrange', key, 0, -1) " + - "for i=1,#items do " + - "if items[i] == obj then " + - "return i - 1 " + - "end " + - "end " + - "return -1", - Collections.singletonList(getName()), o); - } - @Override public Publisher indexOf(Object o) { - return indexOf(o, new LongReplayConvertor()); + return reactive(instance.indexOfAsync(o, new LongReplayConvertor())); } @Override public Publisher lastIndexOf(Object o) { - return commandExecutor.evalReadReactive(getName(), codec, new RedisCommand("EVAL", 4), - "local key = KEYS[1] " + - "local obj = ARGV[1] " + - "local items = redis.call('lrange', key, 0, -1) " + - "for i = table.getn(items), 0, -1 do " + - "if items[i] == obj then " + - "return i - 1 " + - "end " + - "end " + - "return -1", - Collections.singletonList(getName()), o); + return reactive(instance.lastIndexOfAsync(o, new LongReplayConvertor())); } @Override diff --git a/redisson/src/main/java/org/redisson/reactive/RedissonObjectReactive.java b/redisson/src/main/java/org/redisson/reactive/RedissonObjectReactive.java index 7931af3de..6c4d70970 100644 --- a/redisson/src/main/java/org/redisson/reactive/RedissonObjectReactive.java +++ b/redisson/src/main/java/org/redisson/reactive/RedissonObjectReactive.java @@ -16,12 +16,12 @@ package org.redisson.reactive; import org.reactivestreams.Publisher; +import org.redisson.api.RFuture; import org.redisson.api.RObjectReactive; import org.redisson.client.codec.Codec; import org.redisson.client.protocol.RedisCommands; import org.redisson.command.CommandReactiveExecutor; -import io.netty.util.concurrent.Future; import reactor.rx.Stream; import reactor.rx.Streams; @@ -43,7 +43,7 @@ abstract class RedissonObjectReactive implements RObjectReactive { this.commandExecutor = commandExecutor; } - public Publisher reactive(Future future) { + public Publisher reactive(RFuture future) { return commandExecutor.reactive(future); } diff --git a/redisson/src/main/java/org/redisson/reactive/RedissonPatternTopicReactive.java b/redisson/src/main/java/org/redisson/reactive/RedissonPatternTopicReactive.java index 744de3995..00a8140a3 100644 --- a/redisson/src/main/java/org/redisson/reactive/RedissonPatternTopicReactive.java +++ b/redisson/src/main/java/org/redisson/reactive/RedissonPatternTopicReactive.java @@ -21,6 +21,7 @@ import java.util.List; import org.reactivestreams.Publisher; import org.redisson.PubSubPatternMessageListener; import org.redisson.PubSubPatternStatusListener; +import org.redisson.api.RFuture; import org.redisson.api.RPatternTopicReactive; import org.redisson.api.listener.PatternMessageListener; import org.redisson.api.listener.PatternStatusListener; @@ -28,11 +29,11 @@ import org.redisson.client.RedisPubSubListener; import org.redisson.client.codec.Codec; import org.redisson.command.CommandReactiveExecutor; import org.redisson.connection.PubSubConnectionEntry; +import org.redisson.misc.RPromise; import org.redisson.pubsub.AsyncSemaphore; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; -import io.netty.util.concurrent.Promise; /** * Distributed topic implementation. Messages are delivered to all message listeners across Redis cluster. @@ -59,21 +60,21 @@ public class RedissonPatternTopicReactive implements RPatternTopicReactive @Override public Publisher addListener(PatternStatusListener listener) { - Promise promise = commandExecutor.getConnectionManager().newPromise(); + RPromise promise = commandExecutor.getConnectionManager().newPromise(); addListener(new PubSubPatternStatusListener(listener, name), promise); return new NettyFuturePublisher(promise); }; @Override public Publisher addListener(PatternMessageListener listener) { - Promise promise = commandExecutor.getConnectionManager().newPromise(); + RPromise promise = commandExecutor.getConnectionManager().newPromise(); PubSubPatternMessageListener pubSubListener = new PubSubPatternMessageListener(listener, name); addListener(pubSubListener, promise); return new NettyFuturePublisher(promise); } - private void addListener(final RedisPubSubListener pubSubListener, final Promise promise) { - Future future = commandExecutor.getConnectionManager().psubscribe(name, codec, pubSubListener); + private void addListener(final RedisPubSubListener pubSubListener, final RPromise promise) { + RFuture future = commandExecutor.getConnectionManager().psubscribe(name, codec, pubSubListener); future.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { diff --git a/redisson/src/main/java/org/redisson/reactive/RedissonTopicReactive.java b/redisson/src/main/java/org/redisson/reactive/RedissonTopicReactive.java index 326e38956..71ce5f70a 100644 --- a/redisson/src/main/java/org/redisson/reactive/RedissonTopicReactive.java +++ b/redisson/src/main/java/org/redisson/reactive/RedissonTopicReactive.java @@ -21,6 +21,7 @@ import java.util.List; import org.reactivestreams.Publisher; import org.redisson.PubSubMessageListener; import org.redisson.PubSubStatusListener; +import org.redisson.api.RFuture; import org.redisson.api.RTopicReactive; import org.redisson.api.listener.MessageListener; import org.redisson.api.listener.StatusListener; @@ -29,11 +30,11 @@ import org.redisson.client.codec.Codec; import org.redisson.client.protocol.RedisCommands; import org.redisson.command.CommandReactiveExecutor; import org.redisson.connection.PubSubConnectionEntry; +import org.redisson.misc.RPromise; import org.redisson.pubsub.AsyncSemaphore; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; -import io.netty.util.concurrent.Promise; /** * Distributed topic implementation. Messages are delivered to all message listeners across Redis cluster. @@ -80,8 +81,8 @@ public class RedissonTopicReactive implements RTopicReactive { } private Publisher addListener(final RedisPubSubListener pubSubListener) { - final Promise promise = commandExecutor.getConnectionManager().newPromise(); - Future future = commandExecutor.getConnectionManager().subscribe(codec, name, pubSubListener); + final RPromise promise = commandExecutor.getConnectionManager().newPromise(); + RFuture future = commandExecutor.getConnectionManager().subscribe(codec, name, pubSubListener); future.addListener(new FutureListener() { @Override public void operationComplete(Future future) throws Exception { diff --git a/redisson/src/test/java/org/redisson/BaseTest.java b/redisson/src/test/java/org/redisson/BaseTest.java index 4430f711a..e22c1df61 100644 --- a/redisson/src/test/java/org/redisson/BaseTest.java +++ b/redisson/src/test/java/org/redisson/BaseTest.java @@ -6,6 +6,7 @@ import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; import org.redisson.api.RedissonClient; +import org.redisson.codec.MsgPackJacksonCodec; import org.redisson.config.Config; public abstract class BaseTest { @@ -24,8 +25,8 @@ public abstract class BaseTest { @AfterClass public static void afterClass() throws IOException, InterruptedException { if (!RedissonRuntimeEnvironment.isTravis) { - RedisRunner.shutDownDefaultRedisServerInstance(); defaultRedisson.shutdown(); + RedisRunner.shutDownDefaultRedisServerInstance(); } } diff --git a/redisson/src/test/java/org/redisson/RedisClientTest.java b/redisson/src/test/java/org/redisson/RedisClientTest.java index 5e68d80d3..9419e07ee 100644 --- a/redisson/src/test/java/org/redisson/RedisClientTest.java +++ b/redisson/src/test/java/org/redisson/RedisClientTest.java @@ -18,6 +18,7 @@ import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; +import org.redisson.api.RFuture; import org.redisson.client.RedisClient; import org.redisson.client.RedisConnection; import org.redisson.client.RedisPubSubConnection; @@ -28,10 +29,11 @@ import org.redisson.client.protocol.CommandData; import org.redisson.client.protocol.CommandsData; import org.redisson.client.protocol.RedisCommands; import org.redisson.client.protocol.pubsub.PubSubType; +import org.redisson.misc.RPromise; +import org.redisson.misc.RedissonPromise; -import io.netty.util.concurrent.Future; import io.netty.util.concurrent.FutureListener; -import io.netty.util.concurrent.Promise; +import io.netty.util.concurrent.ImmediateEventExecutor; public class RedisClientTest { @@ -66,7 +68,7 @@ public class RedisClientTest { @Test public void testConnectAsync() throws InterruptedException { RedisClient c = new RedisClient("localhost", 6379); - Future f = c.connectAsync(); + RFuture f = c.connectAsync(); final CountDownLatch l = new CountDownLatch(1); f.addListener((FutureListener) future -> { RedisConnection conn = future.get(); @@ -143,7 +145,7 @@ public class RedisClientTest { CommandData cmd4 = conn.create(null, RedisCommands.PING); commands.add(cmd4); - Promise p = c.getBootstrap().group().next().newPromise(); + RPromise p = new RedissonPromise(ImmediateEventExecutor.INSTANCE.newPromise()); conn.send(new CommandsData(p, commands)); assertThat(cmd1.getPromise().get()).isEqualTo("PONG"); @@ -180,7 +182,7 @@ public class RedisClientTest { commands.add(cmd1); } - Promise p = c.getBootstrap().group().next().newPromise(); + RPromise p = new RedissonPromise(ImmediateEventExecutor.INSTANCE.newPromise()); conn.send(new CommandsData(p, commands)); for (CommandData commandData : commands) { diff --git a/redisson/src/test/java/org/redisson/RedissonBatchTest.java b/redisson/src/test/java/org/redisson/RedissonBatchTest.java index 1576f5ee7..32ebce925 100644 --- a/redisson/src/test/java/org/redisson/RedissonBatchTest.java +++ b/redisson/src/test/java/org/redisson/RedissonBatchTest.java @@ -12,14 +12,13 @@ import java.util.concurrent.atomic.AtomicLong; import org.junit.Assert; import org.junit.Test; import org.redisson.api.RBatch; +import org.redisson.api.RFuture; import org.redisson.api.RListAsync; import org.redisson.api.RScript; import org.redisson.api.RScript.Mode; import org.redisson.client.RedisException; import org.redisson.client.codec.StringCodec; -import io.netty.util.concurrent.Future; - public class RedissonBatchTest extends BaseTest { // @Test @@ -56,8 +55,8 @@ public class RedissonBatchTest extends BaseTest { RBatch b = redisson.createBatch(); b.getMap("test1").putAsync("1", "2"); b.getMap("test2", StringCodec.INSTANCE).putAsync("21", "3"); - Future val1 = b.getMap("test1").getAsync("1"); - Future val2 = b.getMap("test2", StringCodec.INSTANCE).getAsync("21"); + RFuture val1 = b.getMap("test1").getAsync("1"); + RFuture val2 = b.getMap("test2", StringCodec.INSTANCE).getAsync("21"); b.execute(); Assert.assertEquals("2", val1.getNow()); @@ -117,7 +116,7 @@ public class RedissonBatchTest extends BaseTest { ExecutorService e = Executors.newFixedThreadPool(16); final RBatch batch = redisson.createBatch(); final AtomicLong index = new AtomicLong(-1); - final List> futures = new CopyOnWriteArrayList<>(); + final List> futures = new CopyOnWriteArrayList<>(); for (int i = 0; i < 500; i++) { futures.add(null); } @@ -129,7 +128,7 @@ public class RedissonBatchTest extends BaseTest { synchronized (RedissonBatchTest.this) { int i = (int) index.incrementAndGet(); int ind = j % 3; - Future f1 = batch.getAtomicLong("test" + ind).addAndGetAsync(j); + RFuture f1 = batch.getAtomicLong("test" + ind).addAndGetAsync(j); futures.set(i, f1); } } @@ -141,7 +140,7 @@ public class RedissonBatchTest extends BaseTest { int i = 0; for (Object element : s) { - Future a = futures.get(i); + RFuture a = futures.get(i); Assert.assertEquals(a.getNow(), element); i++; } diff --git a/redisson/src/test/java/org/redisson/RedissonBlockingQueueTest.java b/redisson/src/test/java/org/redisson/RedissonBlockingQueueTest.java index 394fac947..32f29757a 100644 --- a/redisson/src/test/java/org/redisson/RedissonBlockingQueueTest.java +++ b/redisson/src/test/java/org/redisson/RedissonBlockingQueueTest.java @@ -1,5 +1,8 @@ package org.redisson; +import static com.jayway.awaitility.Awaitility.await; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import java.util.ArrayList; import java.util.HashSet; @@ -19,14 +22,10 @@ import org.junit.Assert; import org.junit.Test; import org.redisson.RedisRunner.RedisProcess; import org.redisson.api.RBlockingQueue; +import org.redisson.api.RFuture; import org.redisson.api.RedissonClient; import org.redisson.config.Config; -import io.netty.util.concurrent.Future; - -import static com.jayway.awaitility.Awaitility.await; -import static org.assertj.core.api.Assertions.assertThat; - public class RedissonBlockingQueueTest extends BaseTest { @Test @@ -41,7 +40,7 @@ public class RedissonBlockingQueueTest extends BaseTest { config.useSingleServer().setAddress("127.0.0.1:6319"); RedissonClient redisson = Redisson.create(config); final RBlockingQueue queue1 = redisson.getBlockingQueue("queue:pollTimeout"); - Future f = queue1.pollAsync(5, TimeUnit.SECONDS); + RFuture f = queue1.pollAsync(5, TimeUnit.SECONDS); Assert.assertFalse(f.await(1, TimeUnit.SECONDS)); runner.stop(); @@ -116,7 +115,7 @@ public class RedissonBlockingQueueTest extends BaseTest { RedissonClient redisson = Redisson.create(config); RBlockingQueue queue1 = redisson.getBlockingQueue("queue:pollany"); - Future f = queue1.pollAsync(10, TimeUnit.SECONDS); + RFuture f = queue1.pollAsync(10, TimeUnit.SECONDS); f.await(1, TimeUnit.SECONDS); runner.stop(); @@ -151,7 +150,7 @@ public class RedissonBlockingQueueTest extends BaseTest { config.useSingleServer().setAddress("127.0.0.1:6319"); RedissonClient redisson = Redisson.create(config); RBlockingQueue queue1 = redisson.getBlockingQueue("testTakeReattach"); - Future f = queue1.takeAsync(); + RFuture f = queue1.takeAsync(); f.await(1, TimeUnit.SECONDS); runner.stop(); @@ -181,7 +180,7 @@ public class RedissonBlockingQueueTest extends BaseTest { RedissonClient redisson = Redisson.create(config); RBlockingQueue queue1 = redisson.getBlockingQueue("testTakeAsyncCancel"); for (int i = 0; i < 10; i++) { - Future f = queue1.takeAsync(); + RFuture f = queue1.takeAsync(); f.cancel(true); } assertThat(queue1.add(1)).isTrue(); @@ -199,7 +198,7 @@ public class RedissonBlockingQueueTest extends BaseTest { RedissonClient redisson = Redisson.create(config); RBlockingQueue queue1 = redisson.getBlockingQueue("queue:pollany"); for (int i = 0; i < 10; i++) { - Future f = queue1.pollAsync(1, TimeUnit.SECONDS); + RFuture f = queue1.pollAsync(1, TimeUnit.SECONDS); f.cancel(true); } assertThat(queue1.add(1)).isTrue(); diff --git a/redisson/src/test/java/org/redisson/RedissonBoundedBlockingQueueTest.java b/redisson/src/test/java/org/redisson/RedissonBoundedBlockingQueueTest.java index 0cf850dad..628362b11 100644 --- a/redisson/src/test/java/org/redisson/RedissonBoundedBlockingQueueTest.java +++ b/redisson/src/test/java/org/redisson/RedissonBoundedBlockingQueueTest.java @@ -23,6 +23,7 @@ import org.junit.Assert; import org.junit.Test; import org.redisson.RedisRunner.RedisProcess; import org.redisson.api.RBoundedBlockingQueue; +import org.redisson.api.RFuture; import org.redisson.api.RedissonClient; import org.redisson.client.RedisException; import org.redisson.config.Config; @@ -229,7 +230,7 @@ public class RedissonBoundedBlockingQueueTest extends BaseTest { RedissonClient redisson = Redisson.create(config); final RBoundedBlockingQueue queue1 = redisson.getBoundedBlockingQueue("bounded-queue:pollTimeout"); assertThat(queue1.trySetCapacity(5)).isTrue(); - Future f = queue1.pollAsync(5, TimeUnit.SECONDS); + RFuture f = queue1.pollAsync(5, TimeUnit.SECONDS); Assert.assertFalse(f.await(1, TimeUnit.SECONDS)); runner.stop(); @@ -307,7 +308,7 @@ public class RedissonBoundedBlockingQueueTest extends BaseTest { RedissonClient redisson = Redisson.create(config); RBoundedBlockingQueue queue1 = redisson.getBoundedBlockingQueue("queue:pollany"); - Future f = queue1.pollAsync(10, TimeUnit.SECONDS); + RFuture f = queue1.pollAsync(10, TimeUnit.SECONDS); f.await(1, TimeUnit.SECONDS); runner.stop(); @@ -346,7 +347,7 @@ public class RedissonBoundedBlockingQueueTest extends BaseTest { RBoundedBlockingQueue queue1 = redisson.getBoundedBlockingQueue("testTakeReattach"); assertThat(queue1.trySetCapacity(15)).isTrue(); - Future f = queue1.takeAsync(); + RFuture f = queue1.takeAsync(); f.await(1, TimeUnit.SECONDS); runner.stop(); @@ -380,7 +381,7 @@ public class RedissonBoundedBlockingQueueTest extends BaseTest { RBoundedBlockingQueue queue1 = redisson.getBoundedBlockingQueue("testTakeAsyncCancel"); assertThat(queue1.trySetCapacity(15)).isTrue(); for (int i = 0; i < 10; i++) { - Future f = queue1.takeAsync(); + RFuture f = queue1.takeAsync(); f.cancel(true); } assertThat(queue1.add(1)).isTrue(); @@ -401,7 +402,7 @@ public class RedissonBoundedBlockingQueueTest extends BaseTest { RBoundedBlockingQueue queue1 = redisson.getBoundedBlockingQueue("queue:pollany"); assertThat(queue1.trySetCapacity(15)).isTrue(); for (int i = 0; i < 10; i++) { - Future f = queue1.pollAsync(1, TimeUnit.SECONDS); + RFuture f = queue1.pollAsync(1, TimeUnit.SECONDS); f.cancel(true); } assertThat(queue1.add(1)).isTrue(); diff --git a/redisson/src/test/java/org/redisson/RedissonMapCacheTest.java b/redisson/src/test/java/org/redisson/RedissonMapCacheTest.java index fce98ee55..1e64420ca 100644 --- a/redisson/src/test/java/org/redisson/RedissonMapCacheTest.java +++ b/redisson/src/test/java/org/redisson/RedissonMapCacheTest.java @@ -16,6 +16,7 @@ import java.util.concurrent.TimeUnit; import org.junit.Assert; import org.junit.Test; +import org.redisson.api.RFuture; import org.redisson.api.RMapCache; import org.redisson.codec.JsonJacksonCodec; import org.redisson.codec.MsgPackJacksonCodec; @@ -571,12 +572,12 @@ public class RedissonMapCacheTest extends BaseTest { @Test public void testPutAsync() throws InterruptedException, ExecutionException { RMapCache map = redisson.getMapCache("simple"); - Future future = map.putAsync(2, 3); + RFuture future = map.putAsync(2, 3); Assert.assertNull(future.get()); Assert.assertEquals((Integer) 3, map.get(2)); - Future future1 = map.putAsync(2, 4); + RFuture future1 = map.putAsync(2, 4); Assert.assertEquals((Integer) 3, future1.get()); Assert.assertEquals((Integer) 4, map.get(2)); diff --git a/redisson/src/test/java/org/redisson/RedissonMapTest.java b/redisson/src/test/java/org/redisson/RedissonMapTest.java index 9b2775650..c57434c73 100644 --- a/redisson/src/test/java/org/redisson/RedissonMapTest.java +++ b/redisson/src/test/java/org/redisson/RedissonMapTest.java @@ -16,14 +16,13 @@ import java.util.concurrent.ExecutionException; import org.junit.Assert; import org.junit.Test; +import org.redisson.api.RFuture; import org.redisson.api.RMap; import org.redisson.api.RedissonClient; import org.redisson.client.codec.StringCodec; import org.redisson.codec.JsonJacksonCodec; import org.redisson.config.Config; -import io.netty.util.concurrent.Future; - public class RedissonMapTest extends BaseTest { public static class SimpleKey implements Serializable { @@ -596,12 +595,12 @@ public class RedissonMapTest extends BaseTest { @Test public void testPutAsync() throws InterruptedException, ExecutionException { RMap map = redisson.getMap("simple"); - Future future = map.putAsync(2, 3); + RFuture future = map.putAsync(2, 3); Assert.assertNull(future.get()); Assert.assertEquals((Integer) 3, map.get(2)); - Future future1 = map.putAsync(2, 4); + RFuture future1 = map.putAsync(2, 4); Assert.assertEquals((Integer) 3, future1.get()); Assert.assertEquals((Integer) 4, map.get(2)); diff --git a/redisson/src/test/java/org/redisson/RedissonRemoteServiceTest.java b/redisson/src/test/java/org/redisson/RedissonRemoteServiceTest.java index 6d0fae8cb..efc06f454 100644 --- a/redisson/src/test/java/org/redisson/RedissonRemoteServiceTest.java +++ b/redisson/src/test/java/org/redisson/RedissonRemoteServiceTest.java @@ -198,7 +198,7 @@ public class RedissonRemoteServiceTest extends BaseTest { RedissonClient r2 = createInstance(); RemoteInterfaceAsync ri = r2.getRemoteSerivce().get(RemoteInterfaceAsync.class); - Future f = ri.cancelMethod(); + RFuture f = ri.cancelMethod(); Thread.sleep(500); assertThat(f.cancel(true)).isTrue(); @@ -230,9 +230,9 @@ public class RedissonRemoteServiceTest extends BaseTest { RedissonClient r2 = createInstance(); RemoteInterfaceAsync ri = r2.getRemoteSerivce().get(RemoteInterfaceAsync.class); - Future f = ri.voidMethod("someName", 100L); + RFuture f = ri.voidMethod("someName", 100L); f.sync(); - Future resFuture = ri.resultMethod(100L); + RFuture resFuture = ri.resultMethod(100L); resFuture.sync(); assertThat(resFuture.getNow()).isEqualTo(200); @@ -249,9 +249,9 @@ public class RedissonRemoteServiceTest extends BaseTest { RedissonClient r2 = createInstance(); RemoteInterfaceAsync ri = r2.getRemoteSerivce().get(RemoteInterfaceAsync.class); - Future f = ri.voidMethod("someName", 100L); + RFuture f = ri.voidMethod("someName", 100L); f.sync(); - Future resFuture = ri.resultMethod(100L); + RFuture resFuture = ri.resultMethod(100L); resFuture.sync(); assertThat(resFuture.getNow()).isEqualTo(200); diff --git a/redisson/src/test/java/org/redisson/RedissonScoredSortedSetTest.java b/redisson/src/test/java/org/redisson/RedissonScoredSortedSetTest.java index ca0a9aa61..de17bdf09 100644 --- a/redisson/src/test/java/org/redisson/RedissonScoredSortedSetTest.java +++ b/redisson/src/test/java/org/redisson/RedissonScoredSortedSetTest.java @@ -16,14 +16,13 @@ import java.util.concurrent.ExecutionException; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; +import org.redisson.api.RFuture; import org.redisson.api.RLexSortedSet; import org.redisson.api.RScoredSortedSet; import org.redisson.api.RSortedSet; import org.redisson.client.codec.StringCodec; import org.redisson.client.protocol.ScoredEntry; -import io.netty.util.concurrent.Future; - public class RedissonScoredSortedSetTest extends BaseTest { @Test @@ -206,9 +205,9 @@ public class RedissonScoredSortedSetTest extends BaseTest { @Test public void testAddAsync() throws InterruptedException, ExecutionException { RScoredSortedSet set = redisson.getScoredSortedSet("simple"); - Future future = set.addAsync(0.323, 2); + RFuture future = set.addAsync(0.323, 2); Assert.assertTrue(future.get()); - Future future2 = set.addAsync(0.323, 2); + RFuture future2 = set.addAsync(0.323, 2); Assert.assertFalse(future2.get()); Assert.assertTrue(set.contains(2)); diff --git a/redisson/src/test/java/org/redisson/RedissonScriptTest.java b/redisson/src/test/java/org/redisson/RedissonScriptTest.java index 776273fab..e67e676de 100644 --- a/redisson/src/test/java/org/redisson/RedissonScriptTest.java +++ b/redisson/src/test/java/org/redisson/RedissonScriptTest.java @@ -1,13 +1,13 @@ package org.redisson; -import io.netty.util.concurrent.Future; +import static org.assertj.core.api.Assertions.assertThat; import java.util.Collections; import java.util.List; -import static org.assertj.core.api.Assertions.*; import org.junit.Assert; import org.junit.Test; +import org.redisson.api.RFuture; import org.redisson.api.RScript; import org.redisson.api.RScript.Mode; import org.redisson.client.RedisException; @@ -24,7 +24,7 @@ public class RedissonScriptTest extends BaseTest { @Test public void testEvalAsync() { RScript script = redisson.getScript(); - Future> res = script.evalAsync(RScript.Mode.READ_ONLY, "return {1,2,3.3333,'\"foo\"',nil,'bar'}", RScript.ReturnType.MULTI, Collections.emptyList()); + RFuture> res = script.evalAsync(RScript.Mode.READ_ONLY, "return {1,2,3.3333,'\"foo\"',nil,'bar'}", RScript.ReturnType.MULTI, Collections.emptyList()); assertThat(res.awaitUninterruptibly().getNow()).containsExactly(1L, 2L, 3L, "foo"); } @@ -73,7 +73,7 @@ public class RedissonScriptTest extends BaseTest { @Test public void testScriptLoadAsync() { redisson.getBucket("foo").set("bar"); - Future r = redisson.getScript().scriptLoadAsync("return redis.call('get', 'foo')"); + RFuture r = redisson.getScript().scriptLoadAsync("return redis.call('get', 'foo')"); Assert.assertEquals("282297a0228f48cd3fc6a55de6316f31422f5d17", r.awaitUninterruptibly().getNow()); String r1 = redisson.getScript().evalSha(Mode.READ_ONLY, "282297a0228f48cd3fc6a55de6316f31422f5d17", RScript.ReturnType.VALUE, Collections.emptyList()); Assert.assertEquals("bar", r1); @@ -99,7 +99,7 @@ public class RedissonScriptTest extends BaseTest { redisson.getBucket("foo").set("bar"); String r = redisson.getScript().eval(Mode.READ_ONLY, "return redis.call('get', 'foo')", RScript.ReturnType.VALUE); Assert.assertEquals("bar", r); - Future r1 = redisson.getScript().evalShaAsync(Mode.READ_ONLY, "282297a0228f48cd3fc6a55de6316f31422f5d17", RScript.ReturnType.VALUE, Collections.emptyList()); + RFuture r1 = redisson.getScript().evalShaAsync(Mode.READ_ONLY, "282297a0228f48cd3fc6a55de6316f31422f5d17", RScript.ReturnType.VALUE, Collections.emptyList()); Assert.assertEquals("bar", r1.awaitUninterruptibly().getNow()); } diff --git a/redisson/src/test/java/org/redisson/RedissonSetTest.java b/redisson/src/test/java/org/redisson/RedissonSetTest.java index eb620ed5d..5a26005cd 100644 --- a/redisson/src/test/java/org/redisson/RedissonSetTest.java +++ b/redisson/src/test/java/org/redisson/RedissonSetTest.java @@ -12,10 +12,8 @@ import java.util.concurrent.ExecutionException; import org.junit.Assert; import org.junit.Test; +import org.redisson.api.RFuture; import org.redisson.api.RSet; -import org.redisson.api.RSetCache; - -import io.netty.util.concurrent.Future; public class RedissonSetTest extends BaseTest { @@ -89,7 +87,7 @@ public class RedissonSetTest extends BaseTest { @Test public void testAddAsync() throws InterruptedException, ExecutionException { RSet set = redisson.getSet("simple"); - Future future = set.addAsync(2); + RFuture future = set.addAsync(2); Assert.assertTrue(future.get()); Assert.assertTrue(set.contains(2)); diff --git a/redisson/src/test/java/org/redisson/RedissonSortedSetTest.java b/redisson/src/test/java/org/redisson/RedissonSortedSetTest.java index 381852d08..a4078b993 100644 --- a/redisson/src/test/java/org/redisson/RedissonSortedSetTest.java +++ b/redisson/src/test/java/org/redisson/RedissonSortedSetTest.java @@ -1,7 +1,5 @@ package org.redisson; -import io.netty.util.concurrent.Future; - import java.util.Arrays; import java.util.Collections; import java.util.HashSet; @@ -15,6 +13,7 @@ import org.hamcrest.MatcherAssert; import org.hamcrest.Matchers; import org.junit.Assert; import org.junit.Test; +import org.redisson.api.RFuture; import org.redisson.api.RSortedSet; public class RedissonSortedSetTest extends BaseTest { @@ -22,7 +21,7 @@ public class RedissonSortedSetTest extends BaseTest { @Test public void testAddAsync() throws InterruptedException, ExecutionException { RSortedSet set = redisson.getSortedSet("simple"); - Future future = set.addAsync(2); + RFuture future = set.addAsync(2); Assert.assertTrue(future.get()); Assert.assertTrue(set.contains(2)); diff --git a/redisson/src/test/java/org/redisson/TimeoutTest.java b/redisson/src/test/java/org/redisson/TimeoutTest.java index b5faa2025..d6311e127 100644 --- a/redisson/src/test/java/org/redisson/TimeoutTest.java +++ b/redisson/src/test/java/org/redisson/TimeoutTest.java @@ -1,16 +1,16 @@ package org.redisson; -import io.netty.util.concurrent.Future; - import java.util.ArrayList; import java.util.List; import java.util.concurrent.ExecutionException; -import org.junit.Test; +import org.redisson.api.RFuture; import org.redisson.api.RMap; import org.redisson.api.RTopic; import org.redisson.api.listener.MessageListener; +import io.netty.util.concurrent.Future; + public class TimeoutTest extends BaseTest { // @Test @@ -59,15 +59,15 @@ public class TimeoutTest extends BaseTest { // @Test public void testPutAsyncTimeout() throws InterruptedException, ExecutionException { RMap map = redisson.getMap("simple"); - List> futures = new ArrayList>(); + List> futures = new ArrayList<>(); for (int i = 0; i < 1000; i++) { - Future future = map.putAsync(i, i*1000); + RFuture future = map.putAsync(i, i*1000); Thread.sleep(1000); futures.add(future); System.out.println(i); } - for (Future future : futures) { + for (RFuture future : futures) { future.get(); } @@ -80,19 +80,19 @@ public class TimeoutTest extends BaseTest { // @Test public void testGetAsyncTimeout() throws InterruptedException, ExecutionException { RMap map = redisson.getMap("simple"); - List> futures = new ArrayList>(); + List> futures = new ArrayList<>(); for (int i = 0; i < 10; i++) { map.put(i, i*1000); } for (int i = 0; i < 10; i++) { - Future future = map.getAsync(i); + RFuture future = map.getAsync(i); Thread.sleep(1000); System.out.println(i); futures.add(future); } - for (Future future : futures) { + for (RFuture future : futures) { Integer res = future.get(); System.out.println(res); } From c9294e1774970d7f1d7f21a5be3a949a58fb575c Mon Sep 17 00:00:00 2001 From: Nikita Date: Wed, 24 Aug 2016 14:50:08 +0300 Subject: [PATCH 02/34] refactoring --- .../src/main/java/org/redisson/RedissonExecutorService.java | 4 ++-- .../java/org/redisson/connection/pool/ConnectionPool.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/redisson/src/main/java/org/redisson/RedissonExecutorService.java b/redisson/src/main/java/org/redisson/RedissonExecutorService.java index 92bda305c..1dc7bb0c7 100644 --- a/redisson/src/main/java/org/redisson/RedissonExecutorService.java +++ b/redisson/src/main/java/org/redisson/RedissonExecutorService.java @@ -660,7 +660,7 @@ public class RedissonExecutorService implements RScheduledExecutorService { } }; for (Future future : futures) { - io.netty.util.concurrent.Future f = (io.netty.util.concurrent.Future) future; + RFuture f = (RFuture) future; f.addListener(listener); } @@ -671,7 +671,7 @@ public class RedissonExecutorService implements RScheduledExecutorService { } for (Future future : futures) { - io.netty.util.concurrent.Future f = (io.netty.util.concurrent.Future) future; + RFuture f = (RFuture) future; f.removeListener(listener); } diff --git a/redisson/src/main/java/org/redisson/connection/pool/ConnectionPool.java b/redisson/src/main/java/org/redisson/connection/pool/ConnectionPool.java index f0bb65715..700fc1644 100644 --- a/redisson/src/main/java/org/redisson/connection/pool/ConnectionPool.java +++ b/redisson/src/main/java/org/redisson/connection/pool/ConnectionPool.java @@ -365,8 +365,8 @@ abstract class ConnectionPool { if (entry.getConfig().getPassword() != null) { RFuture temp = c.async(RedisCommands.AUTH, config.getPassword()); - FutureListener listener = new FutureListener () { - @Override public void operationComplete (Future < Void > future)throws Exception { + FutureListener listener = new FutureListener() { + @Override public void operationComplete(Future future)throws Exception { ping(c, pingListener); } }; From 4deedba56214b5a5b586fbec3f7f49eba0e903fc Mon Sep 17 00:00:00 2001 From: Nikita Date: Wed, 24 Aug 2016 14:52:40 +0300 Subject: [PATCH 03/34] version updated --- pom.xml | 2 +- redisson-all/pom.xml | 2 +- redisson/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 379061757..fa6c00bd7 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ org.redisson redisson-parent - 0.9.1-SNAPSHOT + 2.3.0-SNAPSHOT pom Redisson diff --git a/redisson-all/pom.xml b/redisson-all/pom.xml index 5abfda8c5..8bf3ec215 100644 --- a/redisson-all/pom.xml +++ b/redisson-all/pom.xml @@ -4,7 +4,7 @@ org.redisson redisson-parent - 0.9.1-SNAPSHOT + 2.3.0-SNAPSHOT ../ diff --git a/redisson/pom.xml b/redisson/pom.xml index 7aaa878eb..415dd26f1 100644 --- a/redisson/pom.xml +++ b/redisson/pom.xml @@ -4,7 +4,7 @@ org.redisson redisson-parent - 0.9.1-SNAPSHOT + 2.3.0-SNAPSHOT ../ From 7b4594f682083f3e1c9dd08b6c5328611618c69d Mon Sep 17 00:00:00 2001 From: Nikita Date: Wed, 24 Aug 2016 15:02:56 +0300 Subject: [PATCH 04/34] comments added --- .../main/java/org/redisson/api/RFuture.java | 65 ++++++++++++++++++- 1 file changed, 62 insertions(+), 3 deletions(-) diff --git a/redisson/src/main/java/org/redisson/api/RFuture.java b/redisson/src/main/java/org/redisson/api/RFuture.java index bbe9c6005..bc0a1c0f6 100644 --- a/redisson/src/main/java/org/redisson/api/RFuture.java +++ b/redisson/src/main/java/org/redisson/api/RFuture.java @@ -76,26 +76,85 @@ public interface RFuture extends java.util.concurrent.Future { */ boolean await(long timeoutMillis) throws InterruptedException; + /** + * Adds the specified listener to this future. The + * specified listener is notified when this future is + * {@linkplain #isDone() done}. If this future is already + * completed, the specified listener is notified immediately. + */ RFuture addListener(FutureListener listener); + /** + * Adds the specified listeners to this future. The + * specified listeners are notified when this future is + * {@linkplain #isDone() done}. If this future is already + * completed, the specified listeners are notified immediately. + */ RFuture addListeners(FutureListener... listeners); + /** + * Removes the first occurrence of the specified listener from this future. + * The specified listener is no longer notified when this + * future is {@linkplain #isDone() done}. If the specified + * listener is not associated with this future, this method + * does nothing and returns silently. + */ RFuture removeListener(FutureListener listener); + /** + * Removes the first occurrence for each of the listeners from this future. + * The specified listeners are no longer notified when this + * future is {@linkplain #isDone() done}. If the specified + * listeners are not associated with this future, this method + * does nothing and returns silently. + */ RFuture removeListeners(FutureListener... listeners); - boolean isCancellable(); - + /** + * Waits for this future until it is done, and rethrows the cause of the failure if this future + * failed. + */ RFuture sync() throws InterruptedException; + /** + * Waits for this future until it is done, and rethrows the cause of the failure if this future + * failed. + */ RFuture syncUninterruptibly(); + /** + * Waits for this future to be completed. + * + * @throws InterruptedException + * if the current thread was interrupted + */ RFuture await() throws InterruptedException; + /** + * Waits for this future to be completed without + * interruption. This method catches an {@link InterruptedException} and + * discards it silently. + */ RFuture awaitUninterruptibly(); - + + /** + * Waits for this future to be completed within the + * specified time limit without interruption. This method catches an + * {@link InterruptedException} and discards it silently. + * + * @return {@code true} if and only if the future was completed within + * the specified time limit + */ boolean awaitUninterruptibly(long timeout, TimeUnit unit); + /** + * Waits for this future to be completed within the + * specified time limit without interruption. This method catches an + * {@link InterruptedException} and discards it silently. + * + * @return {@code true} if and only if the future was completed within + * the specified time limit + */ boolean awaitUninterruptibly(long timeoutMillis); From 49b23d4e28213c151685f9588acf14eca9c8e8d9 Mon Sep 17 00:00:00 2001 From: Nikita Date: Wed, 24 Aug 2016 15:09:06 +0300 Subject: [PATCH 05/34] compilation fixed --- .../org/redisson/misc/PromiseDelegator.java | 29 ++++++++++++++++--- .../org/redisson/misc/RedissonPromise.java | 5 ---- 2 files changed, 25 insertions(+), 9 deletions(-) diff --git a/redisson/src/main/java/org/redisson/misc/PromiseDelegator.java b/redisson/src/main/java/org/redisson/misc/PromiseDelegator.java index 9182b5a09..b6cf398ba 100644 --- a/redisson/src/main/java/org/redisson/misc/PromiseDelegator.java +++ b/redisson/src/main/java/org/redisson/misc/PromiseDelegator.java @@ -35,106 +35,127 @@ public class PromiseDelegator implements RPromise { return promise; } + @Override public RPromise setSuccess(T result) { return promise.setSuccess(result); } + @Override public boolean isSuccess() { return promise.isSuccess(); } + @Override public boolean trySuccess(T result) { return promise.trySuccess(result); } - public boolean isCancellable() { - return promise.isCancellable(); - } - + @Override public Throwable cause() { return promise.cause(); } + @Override public Promise setFailure(Throwable cause) { return promise.setFailure(cause); } + @Override public boolean tryFailure(Throwable cause) { return promise.tryFailure(cause); } + @Override public boolean setUncancellable() { return promise.setUncancellable(); } + @Override public RPromise addListener(FutureListener listener) { return promise.addListener(listener); } + @Override public RPromise addListeners(FutureListener... listeners) { return promise.addListeners(listeners); } + @Override public RPromise removeListener(FutureListener listener) { return promise.removeListener(listener); } + @Override public RPromise removeListeners(FutureListener... listeners) { return promise.removeListeners(listeners); } + @Override public RPromise await() throws InterruptedException { return promise.await(); } + @Override public RPromise awaitUninterruptibly() { return promise.awaitUninterruptibly(); } + @Override public RPromise sync() throws InterruptedException { return promise.sync(); } + @Override public RPromise syncUninterruptibly() { return promise.syncUninterruptibly(); } + @Override public boolean await(long timeout, TimeUnit unit) throws InterruptedException { return promise.await(timeout, unit); } + @Override public boolean isCancelled() { return promise.isCancelled(); } + @Override public boolean isDone() { return promise.isDone(); } + @Override public boolean await(long timeoutMillis) throws InterruptedException { return promise.await(timeoutMillis); } + @Override public T get() throws InterruptedException, ExecutionException { return promise.get(); } + @Override public boolean awaitUninterruptibly(long timeout, TimeUnit unit) { return promise.awaitUninterruptibly(timeout, unit); } + @Override public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { return promise.get(timeout, unit); } + @Override public boolean awaitUninterruptibly(long timeoutMillis) { return promise.awaitUninterruptibly(timeoutMillis); } + @Override public T getNow() { return promise.getNow(); } + @Override public boolean cancel(boolean mayInterruptIfRunning) { return promise.cancel(mayInterruptIfRunning); } diff --git a/redisson/src/main/java/org/redisson/misc/RedissonPromise.java b/redisson/src/main/java/org/redisson/misc/RedissonPromise.java index d0c2c2902..95303e46e 100644 --- a/redisson/src/main/java/org/redisson/misc/RedissonPromise.java +++ b/redisson/src/main/java/org/redisson/misc/RedissonPromise.java @@ -56,11 +56,6 @@ public class RedissonPromise implements RPromise { return promise.trySuccess(result); } - @Override - public boolean isCancellable() { - return promise.isCancellable(); - } - @Override public Throwable cause() { return promise.cause(); From 651cd907fc2a2aff6e07af4613afc36ebb0110b2 Mon Sep 17 00:00:00 2001 From: Nikita Date: Wed, 24 Aug 2016 15:11:26 +0300 Subject: [PATCH 06/34] [maven-release-plugin] prepare release redisson-parent-2.3.0 --- pom.xml | 4 ++-- redisson-all/pom.xml | 4 ++-- redisson/pom.xml | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index fa6c00bd7..0eb8311f1 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ org.redisson redisson-parent - 2.3.0-SNAPSHOT + 2.3.0 pom Redisson @@ -20,7 +20,7 @@ scm:git:git@github.com:mrniko/redisson.git scm:git:git@github.com:mrniko/redisson.git scm:git:git@github.com:mrniko/redisson.git - redisson-parent-0.9.0 + redisson-parent-2.3.0 diff --git a/redisson-all/pom.xml b/redisson-all/pom.xml index 8bf3ec215..ac210db38 100644 --- a/redisson-all/pom.xml +++ b/redisson-all/pom.xml @@ -4,7 +4,7 @@ org.redisson redisson-parent - 2.3.0-SNAPSHOT + 2.3.0 ../ @@ -26,7 +26,7 @@ scm:git:git@github.com:mrniko/redisson.git scm:git:git@github.com:mrniko/redisson.git scm:git:git@github.com:mrniko/redisson.git - redisson-parent-0.9.0 + redisson-parent-2.3.0 diff --git a/redisson/pom.xml b/redisson/pom.xml index 415dd26f1..1ae7b84dd 100644 --- a/redisson/pom.xml +++ b/redisson/pom.xml @@ -4,7 +4,7 @@ org.redisson redisson-parent - 2.3.0-SNAPSHOT + 2.3.0 ../ From 29c17dbb61de54706e2336287ff05ed4c00c055a Mon Sep 17 00:00:00 2001 From: Nikita Date: Wed, 24 Aug 2016 15:11:33 +0300 Subject: [PATCH 07/34] [maven-release-plugin] prepare for next development iteration --- pom.xml | 4 ++-- redisson-all/pom.xml | 4 ++-- redisson/pom.xml | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index 0eb8311f1..e1f0daf9a 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ org.redisson redisson-parent - 2.3.0 + 2.3.1-SNAPSHOT pom Redisson @@ -20,7 +20,7 @@ scm:git:git@github.com:mrniko/redisson.git scm:git:git@github.com:mrniko/redisson.git scm:git:git@github.com:mrniko/redisson.git - redisson-parent-2.3.0 + redisson-parent-0.9.0 diff --git a/redisson-all/pom.xml b/redisson-all/pom.xml index ac210db38..1259ecce2 100644 --- a/redisson-all/pom.xml +++ b/redisson-all/pom.xml @@ -4,7 +4,7 @@ org.redisson redisson-parent - 2.3.0 + 2.3.1-SNAPSHOT ../ @@ -26,7 +26,7 @@ scm:git:git@github.com:mrniko/redisson.git scm:git:git@github.com:mrniko/redisson.git scm:git:git@github.com:mrniko/redisson.git - redisson-parent-2.3.0 + redisson-parent-0.9.0 diff --git a/redisson/pom.xml b/redisson/pom.xml index 1ae7b84dd..c0f072099 100644 --- a/redisson/pom.xml +++ b/redisson/pom.xml @@ -4,7 +4,7 @@ org.redisson redisson-parent - 2.3.0 + 2.3.1-SNAPSHOT ../ From 87bd4d2fabb8c520e09d209381d6e8ef3fe5dd2d Mon Sep 17 00:00:00 2001 From: Nikita Koksharov Date: Wed, 24 Aug 2016 16:34:01 +0300 Subject: [PATCH 08/34] Update CHANGELOG.md --- CHANGELOG.md | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 5860dc4d4..32b190f55 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,24 @@ Redisson Releases History ================================ ####Please Note: trunk is current development branch. +####24-Aug-2016 - version 2.3.0 released +Starting from this version Redisson could be run as standalone node to execute distributed tasks and do other things in future. More info about it [here](https://github.com/mrniko/redisson/wiki/12.-Standalone-node) + +Feature - __new service added__ `RExecutorService`. More info about it [here](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#93-executor-service) +Feature - __new service added__ `RScheduledExecutorService`. More info about it +[here](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#94-scheduled-executor-service) +Feature - __new service added__ `RLiveObjectService`. More info about it +[here](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#92-live-object-service) (big thanks to Rui Gu for this amazin feature) +Feature - __new object added__ `RBoundedBlockingQueue`. More info about it [here](https://github.com/mrniko/redisson/wiki/7.-distributed-collections/#711-bounded-blocking-queue) +Feature - __Redis deployment tool__. More info about it +[here](https://github.com/mrniko/redisson/wiki/13.-Tools#131-redis-deployment-tool) +Feature - __Cluster management tool__. More info about it [here](https://github.com/mrniko/redisson/wiki/13.-Tools#132-cluster-management-tool) +Feature - Avro and Smile codecs added +__Breaking api change__ - all config classes moved to `org.redisson.config` package +__Breaking api change__ - all classes moved from `org.redisson.core` to `org.redisson.api` package +__Breaking api change__ - switched from `io.netty.util.concurrent.Future` to `org.redisson.api.RFuture` interface +Fixed - division by zero in WeightedRoundRobinBalancer (thanks to Shailender R Bathula) + ####08-Aug-2016 - version 2.2.24 released Fixed - PubSub connection in cluster mode should be connected to node according slot derived from channel name Fixed - `RLock.tryLock` could block forever under some conditions From f99d868eb0996e486fdfc356a0ee492351f9e866 Mon Sep 17 00:00:00 2001 From: Nikita Koksharov Date: Wed, 24 Aug 2016 16:35:21 +0300 Subject: [PATCH 09/34] Update CHANGELOG.md --- CHANGELOG.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 32b190f55..78290e09d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -3,13 +3,13 @@ Redisson Releases History ####Please Note: trunk is current development branch. ####24-Aug-2016 - version 2.3.0 released -Starting from this version Redisson could be run as standalone node to execute distributed tasks and do other things in future. More info about it [here](https://github.com/mrniko/redisson/wiki/12.-Standalone-node) +Starting from this version Redisson could be run as standalone node to execute distributed tasks. More features will be added to it in future. Read docs about it [here](https://github.com/mrniko/redisson/wiki/12.-Standalone-node) Feature - __new service added__ `RExecutorService`. More info about it [here](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#93-executor-service) Feature - __new service added__ `RScheduledExecutorService`. More info about it [here](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#94-scheduled-executor-service) Feature - __new service added__ `RLiveObjectService`. More info about it -[here](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#92-live-object-service) (big thanks to Rui Gu for this amazin feature) +[here](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#92-live-object-service) (big thanks to Rui Gu for this amazing feature) Feature - __new object added__ `RBoundedBlockingQueue`. More info about it [here](https://github.com/mrniko/redisson/wiki/7.-distributed-collections/#711-bounded-blocking-queue) Feature - __Redis deployment tool__. More info about it [here](https://github.com/mrniko/redisson/wiki/13.-Tools#131-redis-deployment-tool) From 522788bdf008c2592bd75f03c812c09b7d482319 Mon Sep 17 00:00:00 2001 From: Nikita Koksharov Date: Wed, 24 Aug 2016 16:48:15 +0300 Subject: [PATCH 10/34] Update README.md --- README.md | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index e393e6243..ce0a6d2e1 100644 --- a/README.md +++ b/README.md @@ -81,12 +81,16 @@ Include the following to your dependency list: org.redisson redisson - 2.2.24 + 2.3.0 ### Gradle - compile 'org.redisson:redisson:2.2.24' + compile 'org.redisson:redisson:2.3.0' + +### Downloads + +[Redisson node 2.3.0](https://repository.sonatype.org/service/local/artifact/maven/redirect?r=central-proxy&g=org.redisson&a=redisson-all&v=2.3.0&e=jar) ### Supported by From 9137a3430884b30f1b1396223e3f9f5dd2346afc Mon Sep 17 00:00:00 2001 From: Nikita Koksharov Date: Wed, 24 Aug 2016 17:07:06 +0300 Subject: [PATCH 11/34] Update README.md --- README.md | 25 ++++++------------------- 1 file changed, 6 insertions(+), 19 deletions(-) diff --git a/README.md b/README.md index ce0a6d2e1..58f7a38e0 100644 --- a/README.md +++ b/README.md @@ -37,32 +37,19 @@ Features 5. automatic sentinel servers discovery * Master with Slave servers mode * Single server mode -* Distributed implementation of `java.util.BitSet` -* Distributed implementation of `java.util.List` -* Distributed implementation of `java.util.Set` with TTL support for each entry -* Distributed implementation of `java.util.SortedSet` -* Distributed implementation of `java.util.Queue` -* Distributed implementation of `java.util.concurrent.BlockingQueue` -* Distributed implementation of `java.util.Deque` -* Distributed implementation of `java.util.concurrent.BlockingDeque` -* Distributed implementation of `java.util.Map` with TTL support for each entry -* Distributed implementation of `java.util.concurrent.ConcurrentMap` with TTL support for each entry -* Distributed implementation of reentrant `java.util.concurrent.locks.Lock` with TTL support -* Distributed implementation of reentrant `java.util.concurrent.locks.ReadWriteLock` with TTL support -* Distributed alternative to the `java.util.concurrent.atomic.AtomicLong` -* Distributed alternative to the `java.util.concurrent.CountDownLatch` -* Distributed alternative to the `java.util.concurrent.Semaphore` -* Distributed publish/subscribe messaging via `org.redisson.core.RTopic` -* Distributed HyperLogLog via `org.redisson.core.RHyperLogLog` +* Over 30+ data structures and services * Asynchronous interface for each object * Asynchronous connection pool * Thread-safe implementation -* All commands executes in an atomic way +* All commands execute in an atomic way * Lua scripting * [Spring cache](https://github.com/mrniko/redisson/wiki/10.-additional-features/#104-spring-cache-integration) integration * Supports [Reactive Streams](http://www.reactive-streams.org) * Supports [Redis pipelining](http://redis.io/topics/pipelining) (command batches) -* Supports [Remote services](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#91-remote-service) +* Implements [Remote service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#91-remote-service) +* Implements [Executor service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#93-executor-service) +* Implements [Scheduled executor service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#94-scheduled-executor-service) +* Implements [Live Object service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#92-live-object-service) * Supports Android platform * Supports auto-reconnect * Supports failed to send command auto-retry From d38e8a1e007e2b1d4e0439384b2b0c1629d92997 Mon Sep 17 00:00:00 2001 From: Nikita Koksharov Date: Wed, 24 Aug 2016 17:07:28 +0300 Subject: [PATCH 12/34] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 58f7a38e0..5ad7e0012 100644 --- a/README.md +++ b/README.md @@ -55,7 +55,7 @@ Features * Supports failed to send command auto-retry * Supports OSGi * Supports many popular codecs ([Jackson JSON](https://github.com/FasterXML/jackson), [Avro](http://avro.apache.org/), [Smile](http://wiki.fasterxml.com/SmileFormatSpec), [CBOR](http://cbor.io/), [MsgPack](http://msgpack.org/), [Kryo](https://github.com/EsotericSoftware/kryo), [FST](https://github.com/RuedigerMoeller/fast-serialization), [LZ4](https://github.com/jpountz/lz4-java), [Snappy](https://github.com/xerial/snappy-java) and JDK Serialization) -* With over 500 unit tests +* With over 900 unit tests Projects using Redisson ================================ From 30851eec1159bd412985870d485480faf5839305 Mon Sep 17 00:00:00 2001 From: Nikita Koksharov Date: Wed, 24 Aug 2016 17:12:48 +0300 Subject: [PATCH 13/34] Update README.md --- README.md | 22 ++++++++++++---------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/README.md b/README.md index 5ad7e0012..266395a60 100644 --- a/README.md +++ b/README.md @@ -35,21 +35,23 @@ Features 3. automatic new slave servers discovery 4. automatic slave servers offline/online discovery 5. automatic sentinel servers discovery -* Master with Slave servers mode -* Single server mode -* Over 30+ data structures and services +* Master with Slave servers mode +* Single server mode * Asynchronous interface for each object * Asynchronous connection pool * Thread-safe implementation -* All commands execute in an atomic way * Lua scripting +* [Distributed objects](https://github.com/mrniko/redisson/wiki/6.-Distributed-objects) +* [Distributed collections](https://github.com/mrniko/redisson/wiki/7.-Distributed-collections) +* [Distributed locks and synchronizers](https://github.com/mrniko/redisson/wiki/8.-Distributed-locks-and-synchronizers) * [Spring cache](https://github.com/mrniko/redisson/wiki/10.-additional-features/#104-spring-cache-integration) integration -* Supports [Reactive Streams](http://www.reactive-streams.org) -* Supports [Redis pipelining](http://redis.io/topics/pipelining) (command batches) -* Implements [Remote service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#91-remote-service) -* Implements [Executor service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#93-executor-service) -* Implements [Scheduled executor service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#94-scheduled-executor-service) -* Implements [Live Object service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#92-live-object-service) +* [Hibernate](https://github.com/mrniko/redisson/wiki/14.-Integration%20with%20frameworks/#142-hibernate) integration +* [Reactive Streams](https://github.com/mrniko/redisson/wiki/3.-operations-execution#32-reactive-way) +* [Redis pipelining](https://github.com/mrniko/redisson/wiki/10.-additional-features#102-execution-batches-of-commands) (command batches) +* [Remote service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#91-remote-service) +* [Executor service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#93-executor-service) +* [Scheduled executor service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#94-scheduled-executor-service) +* [Live Object service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#92-live-object-service) * Supports Android platform * Supports auto-reconnect * Supports failed to send command auto-retry From 3eddd394b221e1eea95b388b90619482eb3cddd2 Mon Sep 17 00:00:00 2001 From: Nikita Koksharov Date: Wed, 24 Aug 2016 17:13:36 +0300 Subject: [PATCH 14/34] Update README.md --- README.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index 266395a60..b2a19b33e 100644 --- a/README.md +++ b/README.md @@ -44,14 +44,14 @@ Features * [Distributed objects](https://github.com/mrniko/redisson/wiki/6.-Distributed-objects) * [Distributed collections](https://github.com/mrniko/redisson/wiki/7.-Distributed-collections) * [Distributed locks and synchronizers](https://github.com/mrniko/redisson/wiki/8.-Distributed-locks-and-synchronizers) -* [Spring cache](https://github.com/mrniko/redisson/wiki/10.-additional-features/#104-spring-cache-integration) integration -* [Hibernate](https://github.com/mrniko/redisson/wiki/14.-Integration%20with%20frameworks/#142-hibernate) integration -* [Reactive Streams](https://github.com/mrniko/redisson/wiki/3.-operations-execution#32-reactive-way) -* [Redis pipelining](https://github.com/mrniko/redisson/wiki/10.-additional-features#102-execution-batches-of-commands) (command batches) * [Remote service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#91-remote-service) * [Executor service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#93-executor-service) * [Scheduled executor service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#94-scheduled-executor-service) * [Live Object service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#92-live-object-service) +* [Spring cache](https://github.com/mrniko/redisson/wiki/10.-additional-features/#104-spring-cache-integration) integration +* [Hibernate](https://github.com/mrniko/redisson/wiki/14.-Integration%20with%20frameworks/#142-hibernate) integration +* [Reactive Streams](https://github.com/mrniko/redisson/wiki/3.-operations-execution#32-reactive-way) +* [Redis pipelining](https://github.com/mrniko/redisson/wiki/10.-additional-features#102-execution-batches-of-commands) (command batches) * Supports Android platform * Supports auto-reconnect * Supports failed to send command auto-retry From b2e900beec546357ce1f2e6ddc936bc36b4952ee Mon Sep 17 00:00:00 2001 From: Nikita Koksharov Date: Wed, 24 Aug 2016 17:52:13 +0300 Subject: [PATCH 15/34] Update README.md --- README.md | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/README.md b/README.md index b2a19b33e..a7658afdc 100644 --- a/README.md +++ b/README.md @@ -3,12 +3,9 @@ Redis based In-Memory Data Grid for Java. Redisson. [![Maven Central](https://img.shields.io/maven-central/v/org.redisson/redisson.svg?style=flat-square)](https://maven-badges.herokuapp.com/maven-central/org.redisson/redisson/) -Use familiar Java data structures with power of [Redis](http://redis.io). - -Based on high-performance async and lock-free Java Redis client and [Netty 4](http://netty.io) framework. +Based on high-performance async and lock-free Java Redis client and [Netty](http://netty.io) framework. Redis 2.8+ and JDK 1.6+ compatible. - Please read [documentation](https://github.com/mrniko/redisson/wiki) for more details. Redisson [releases history](https://github.com/mrniko/redisson/blob/master/CHANGELOG.md). From c48ff1654f668575c1a6465084174f78e5cf7127 Mon Sep 17 00:00:00 2001 From: Nikita Koksharov Date: Wed, 24 Aug 2016 18:35:17 +0300 Subject: [PATCH 16/34] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index a7658afdc..b363586b9 100644 --- a/README.md +++ b/README.md @@ -45,7 +45,7 @@ Features * [Executor service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#93-executor-service) * [Scheduled executor service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#94-scheduled-executor-service) * [Live Object service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#92-live-object-service) -* [Spring cache](https://github.com/mrniko/redisson/wiki/10.-additional-features/#104-spring-cache-integration) integration +* [Spring cache](https://github.com/mrniko/redisson/wiki/14.-Integration%20with%20frameworks/#141-spring-cache) integration * [Hibernate](https://github.com/mrniko/redisson/wiki/14.-Integration%20with%20frameworks/#142-hibernate) integration * [Reactive Streams](https://github.com/mrniko/redisson/wiki/3.-operations-execution#32-reactive-way) * [Redis pipelining](https://github.com/mrniko/redisson/wiki/10.-additional-features#102-execution-batches-of-commands) (command batches) From 9f4187e772e6db30f657b4c58c68b2c9b0f3c903 Mon Sep 17 00:00:00 2001 From: Nikita Koksharov Date: Wed, 24 Aug 2016 22:08:37 +0300 Subject: [PATCH 17/34] Update CHANGELOG.md --- CHANGELOG.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 78290e09d..9300b6675 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -5,9 +5,9 @@ Redisson Releases History ####24-Aug-2016 - version 2.3.0 released Starting from this version Redisson could be run as standalone node to execute distributed tasks. More features will be added to it in future. Read docs about it [here](https://github.com/mrniko/redisson/wiki/12.-Standalone-node) -Feature - __new service added__ `RExecutorService`. More info about it [here](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#93-executor-service) +Feature - __new service added__ `RExecutorService`. More info about it [here](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#93-distributed-executor-service) Feature - __new service added__ `RScheduledExecutorService`. More info about it -[here](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#94-scheduled-executor-service) +[here](https://github.com/mrniko/redisson/wiki/9.-distributed-services#94-distributed-scheduled-executor-service) Feature - __new service added__ `RLiveObjectService`. More info about it [here](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#92-live-object-service) (big thanks to Rui Gu for this amazing feature) Feature - __new object added__ `RBoundedBlockingQueue`. More info about it [here](https://github.com/mrniko/redisson/wiki/7.-distributed-collections/#711-bounded-blocking-queue) From 1cf00d2da247fd408e463700ab34916541637886 Mon Sep 17 00:00:00 2001 From: Nikita Date: Fri, 26 Aug 2016 13:33:07 +0300 Subject: [PATCH 18/34] Fixed wrong object encoding during RedissonScoredSortedSet.addScore usage. #594 --- .../org/redisson/RedissonScoredSortedSet.java | 6 +++--- .../client/protocol/RedisCommands.java | 2 +- .../redisson/RedissonScoredSortedSetTest.java | 19 +++++++++---------- 3 files changed, 13 insertions(+), 14 deletions(-) diff --git a/redisson/src/main/java/org/redisson/RedissonScoredSortedSet.java b/redisson/src/main/java/org/redisson/RedissonScoredSortedSet.java index 227d9bb51..12cf4f771 100644 --- a/redisson/src/main/java/org/redisson/RedissonScoredSortedSet.java +++ b/redisson/src/main/java/org/redisson/RedissonScoredSortedSet.java @@ -30,8 +30,8 @@ import java.util.Map.Entry; import org.redisson.api.RFuture; import org.redisson.api.RScoredSortedSet; import org.redisson.client.codec.Codec; +import org.redisson.client.codec.DoubleCodec; import org.redisson.client.codec.ScoredCodec; -import org.redisson.client.codec.StringCodec; import org.redisson.client.protocol.RedisCommand; import org.redisson.client.protocol.RedisCommand.ValueType; import org.redisson.client.protocol.RedisCommands; @@ -357,8 +357,8 @@ public class RedissonScoredSortedSet extends RedissonExpirable implements RSc @Override public RFuture addScoreAsync(V object, Number value) { - return commandExecutor.writeAsync(getName(), StringCodec.INSTANCE, RedisCommands.ZINCRBY, - getName(), new BigDecimal(value.toString()).toPlainString(), object); + return commandExecutor.writeAsync(getName(), DoubleCodec.INSTANCE, RedisCommands.ZINCRBY, + getName(), new BigDecimal(value.toString()).toPlainString(), encode(object)); } @Override diff --git a/redisson/src/main/java/org/redisson/client/protocol/RedisCommands.java b/redisson/src/main/java/org/redisson/client/protocol/RedisCommands.java index c9b292c6e..01329dfcc 100644 --- a/redisson/src/main/java/org/redisson/client/protocol/RedisCommands.java +++ b/redisson/src/main/java/org/redisson/client/protocol/RedisCommands.java @@ -113,7 +113,7 @@ public interface RedisCommands { RedisCommand>> ZRANGE_ENTRY = new RedisCommand>>("ZRANGE", new ScoredSortedSetReplayDecoder()); RedisCommand>> ZRANGEBYSCORE_ENTRY = new RedisCommand>>("ZRANGEBYSCORE", new ScoredSortedSetReplayDecoder()); RedisCommand> ZSCAN = new RedisCommand>("ZSCAN", new NestedMultiDecoder(new ScoredSortedSetScanDecoder(), new ScoredSortedSetScanReplayDecoder()), ValueType.OBJECT); - RedisStrictCommand ZINCRBY = new RedisStrictCommand("ZINCRBY", new DoubleReplayConvertor(), 4); + RedisStrictCommand ZINCRBY = new RedisStrictCommand("ZINCRBY", new DoubleReplayConvertor()); RedisCommand> SCAN = new RedisCommand>("SCAN", new NestedMultiDecoder(new ObjectListReplayDecoder(), new ListScanResultReplayDecoder()), ValueType.OBJECT); RedisStrictCommand RANDOM_KEY = new RedisStrictCommand("RANDOMKEY", new StringDataDecoder()); diff --git a/redisson/src/test/java/org/redisson/RedissonScoredSortedSetTest.java b/redisson/src/test/java/org/redisson/RedissonScoredSortedSetTest.java index de17bdf09..22176b6dc 100644 --- a/redisson/src/test/java/org/redisson/RedissonScoredSortedSetTest.java +++ b/redisson/src/test/java/org/redisson/RedissonScoredSortedSetTest.java @@ -20,7 +20,6 @@ import org.redisson.api.RFuture; import org.redisson.api.RLexSortedSet; import org.redisson.api.RScoredSortedSet; import org.redisson.api.RSortedSet; -import org.redisson.client.codec.StringCodec; import org.redisson.client.protocol.ScoredEntry; public class RedissonScoredSortedSetTest extends BaseTest { @@ -735,23 +734,23 @@ public class RedissonScoredSortedSetTest extends BaseTest { Assert.assertEquals("c", a[0].getValue()); Assert.assertEquals("d", a[1].getValue()); } - + @Test public void testAddAndGet() throws InterruptedException { - RScoredSortedSet set = redisson.getScoredSortedSet("simple", StringCodec.INSTANCE); - set.add(1, 100); + RScoredSortedSet set = redisson.getScoredSortedSet("simple"); + set.add(1, "100"); - Double res = set.addScore(100, 11); + Double res = set.addScore("100", 11); Assert.assertEquals(12, (double)res, 0); - Double score = set.getScore(100); + Double score = set.getScore("100"); Assert.assertEquals(12, (double)score, 0); - RScoredSortedSet set2 = redisson.getScoredSortedSet("simple", StringCodec.INSTANCE); - set2.add(100.2, 1); + RScoredSortedSet set2 = redisson.getScoredSortedSet("simple"); + set2.add(100.2, "1"); - Double res2 = set2.addScore(1, new Double(12.1)); + Double res2 = set2.addScore("1", new Double(12.1)); Assert.assertTrue(new Double(112.3).compareTo(res2) == 0); - res2 = set2.getScore(1); + res2 = set2.getScore("1"); Assert.assertTrue(new Double(112.3).compareTo(res2) == 0); } From 54b888ba0c1196a583a6a4736a929050d5f76ecd Mon Sep 17 00:00:00 2001 From: Nikita Koksharov Date: Fri, 26 Aug 2016 14:28:50 +0300 Subject: [PATCH 19/34] Update README.md --- README.md | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index b363586b9..e24ad7709 100644 --- a/README.md +++ b/README.md @@ -41,10 +41,8 @@ Features * [Distributed objects](https://github.com/mrniko/redisson/wiki/6.-Distributed-objects) * [Distributed collections](https://github.com/mrniko/redisson/wiki/7.-Distributed-collections) * [Distributed locks and synchronizers](https://github.com/mrniko/redisson/wiki/8.-Distributed-locks-and-synchronizers) -* [Remote service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#91-remote-service) -* [Executor service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#93-executor-service) -* [Scheduled executor service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#94-scheduled-executor-service) -* [Live Object service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#92-live-object-service) +* [Distributed services](https://github.com/mrniko/redisson/wiki/9.-distributed-services) +* [Scheduler service](https://github.com/mrniko/redisson/wiki/9.-distributed-services/#94-scheduled-executor-service) * [Spring cache](https://github.com/mrniko/redisson/wiki/14.-Integration%20with%20frameworks/#141-spring-cache) integration * [Hibernate](https://github.com/mrniko/redisson/wiki/14.-Integration%20with%20frameworks/#142-hibernate) integration * [Reactive Streams](https://github.com/mrniko/redisson/wiki/3.-operations-execution#32-reactive-way) From 4b61ada361f74f6ee4d24dd987d98daa91752949 Mon Sep 17 00:00:00 2001 From: Nikita Koksharov Date: Fri, 26 Aug 2016 14:58:05 +0300 Subject: [PATCH 20/34] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index e24ad7709..aed4ebab5 100644 --- a/README.md +++ b/README.md @@ -74,7 +74,7 @@ Include the following to your dependency list: ### Downloads -[Redisson node 2.3.0](https://repository.sonatype.org/service/local/artifact/maven/redirect?r=central-proxy&g=org.redisson&a=redisson-all&v=2.3.0&e=jar) +[Redisson 2.3.0](https://repository.sonatype.org/service/local/artifact/maven/redirect?r=central-proxy&g=org.redisson&a=redisson&v=2.3.0&e=jar), [Redisson node 2.3.0](https://repository.sonatype.org/service/local/artifact/maven/redirect?r=central-proxy&g=org.redisson&a=redisson-all&v=2.3.0&e=jar) ### Supported by From 8d5fba196774f1a5ecdbe8e02e0c49f229ba36e7 Mon Sep 17 00:00:00 2001 From: Nikita Date: Wed, 31 Aug 2016 14:51:06 +0300 Subject: [PATCH 21/34] LRU and LFU caches implemented. #592 --- .../org/redisson/misc/AbstractCacheMap.java | 524 ++++++++++++++++++ .../main/java/org/redisson/misc/Cache.java | 32 ++ .../java/org/redisson/misc/LFUCacheMap.java | 134 +++++ .../java/org/redisson/misc/LRUCacheMap.java | 68 +++ .../java/org/redisson/misc/NoneCacheMap.java | 48 ++ .../org/redisson/misc/LFUCacheMapTest.java | 78 +++ .../org/redisson/misc/LRUCacheMapTest.java | 73 +++ .../org/redisson/misc/NoneCacheMapTest.java | 51 ++ 8 files changed, 1008 insertions(+) create mode 100644 redisson/src/main/java/org/redisson/misc/AbstractCacheMap.java create mode 100644 redisson/src/main/java/org/redisson/misc/Cache.java create mode 100644 redisson/src/main/java/org/redisson/misc/LFUCacheMap.java create mode 100644 redisson/src/main/java/org/redisson/misc/LRUCacheMap.java create mode 100644 redisson/src/main/java/org/redisson/misc/NoneCacheMap.java create mode 100644 redisson/src/test/java/org/redisson/misc/LFUCacheMapTest.java create mode 100644 redisson/src/test/java/org/redisson/misc/LRUCacheMapTest.java create mode 100644 redisson/src/test/java/org/redisson/misc/NoneCacheMapTest.java diff --git a/redisson/src/main/java/org/redisson/misc/AbstractCacheMap.java b/redisson/src/main/java/org/redisson/misc/AbstractCacheMap.java new file mode 100644 index 000000000..3355f67b4 --- /dev/null +++ b/redisson/src/main/java/org/redisson/misc/AbstractCacheMap.java @@ -0,0 +1,524 @@ +/** + * Copyright 2016 Nikita Koksharov + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.redisson.misc; + +import java.util.AbstractCollection; +import java.util.AbstractMap.SimpleEntry; +import java.util.AbstractSet; +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; + +import io.netty.util.internal.PlatformDependent; + +/** + * + * @author Nikita Koksharov + * + * @param + * @param + */ +public abstract class AbstractCacheMap implements Cache { + + public static class CachedValue { + + private final Object key; + private final Object value; + + long ttl; + long maxIdleTime; + + long creationTime; + long lastAccess; + long accessCount; + + public CachedValue(Object key, Object value, long ttl, long maxIdleTime) { + this.value = value; + this.ttl = ttl; + this.key = key; + this.maxIdleTime = maxIdleTime; + creationTime = System.currentTimeMillis(); + lastAccess = creationTime; + } + + public boolean isExpired() { + boolean result = false; + long currentTime = System.currentTimeMillis(); + if (ttl != 0 && creationTime + ttl < currentTime) { + result = true; + } + if (maxIdleTime != 0 && lastAccess + maxIdleTime < currentTime) { + result = true; + } + return result; + } + + public void addAccessCount(long value) { + accessCount += value; + } + + public Long getAccessCount() { + return accessCount; + } + + public Object getKey() { + return key; + } + + public Object getValue() { + lastAccess = System.currentTimeMillis(); + return value; + } + + @Override + public String toString() { + return "CachedValue [key=" + key + ", value=" + value + ", accessCount=" + accessCount + "]"; + } + + } + + final int size; + final ConcurrentMap map = PlatformDependent.newConcurrentHashMap(); + private final long timeToLiveInMillis; + private final long maxIdleInMillis; + + public AbstractCacheMap(int size, long timeToLiveInMillis, long maxIdleInMillis) { + if (size < 0) { + throw new IllegalArgumentException("Size can't be " + size); + } + this.size = size; + this.maxIdleInMillis = maxIdleInMillis; + this.timeToLiveInMillis = timeToLiveInMillis; + } + + protected void onValueRead(CachedValue value) { + + } + + protected void onValueRemove(CachedValue value) { + + } + + + /* + * (non-Javadoc) + * @see java.util.Map#size() + */ + @Override + public int size() { + return map.size(); + } + + /* + * (non-Javadoc) + * @see java.util.Map#isEmpty() + */ + @Override + public boolean isEmpty() { + return map.isEmpty(); + } + + /* + * (non-Javadoc) + * @see java.util.Map#containsKey(java.lang.Object) + */ + @Override + public boolean containsKey(Object key) { + if (key == null) { + throw new NullPointerException(); + } + + CachedValue entry = map.get(key); + if (entry == null) { + return false; + } + if (entry.isExpired()) { + if (map.remove(key, entry)) { + onValueRemove(entry); + return false; + } + return containsKey(key); + } + return true; + } + + /* + * (non-Javadoc) + * @see java.util.Map#containsValue(java.lang.Object) + */ + @Override + public boolean containsValue(Object value) { + if (value == null) { + throw new NullPointerException(); + } + + for (Map.Entry entry : map.entrySet()) { + CachedValue cachedValue = entry.getValue(); + if (cachedValue.getValue().equals(value)) { + if (cachedValue.isExpired()) { + if (map.remove(cachedValue.getKey(), cachedValue)) { + onValueRemove(cachedValue); + } + } else { + readValue(cachedValue); + return true; + } + } + } + return false; + } + + /* + * (non-Javadoc) + * @see java.util.Map#get(java.lang.Object) + */ + @Override + public V get(Object key) { + if (key == null) { + throw new NullPointerException(); + } + + CachedValue entry = map.get(key); + if (entry == null) { + return null; + } + if (entry.isExpired()) { + if (map.remove(key, entry)) { + onValueRemove(entry); + return null; + } + return get(key); + } + return readValue(entry); + } + + @SuppressWarnings("unchecked") + protected V readValue(CachedValue entry) { + onValueRead(entry); + return (V) entry.getValue(); + } + + /* + * (non-Javadoc) + * @see java.util.Map#put(java.lang.Object, java.lang.Object) + */ + @Override + public V put(K key, V value) { + return put(key, value, timeToLiveInMillis, TimeUnit.MILLISECONDS, maxIdleInMillis, TimeUnit.MILLISECONDS); + } + + @SuppressWarnings("unchecked") + @Override + public V put(K key, V value, long ttl, TimeUnit ttlUnit, long maxIdleTime, TimeUnit maxIdleUnit) { + CachedValue entry = new CachedValue(key, value, ttlUnit.toMillis(ttl), maxIdleUnit.toMillis(maxIdleTime)); + if (isFull(key)) { + if (!removeExpiredEntries()) { + onMapFull(); + } + } + onValueCreate(entry); + CachedValue prevCachedValue = map.put(key, entry); + if (prevCachedValue != null) { + onValueRemove(prevCachedValue); + if (!prevCachedValue.isExpired()) { + return (V) prevCachedValue.getValue(); + } + } + return null; + } + + protected void onValueCreate(CachedValue entry) { + } + + private boolean removeExpiredEntries() { + boolean removed = false; + // TODO optimize + for (CachedValue value : map.values()) { + if (value.isExpired()) { + if (map.remove(value.getKey(), value)) { + onValueRemove(value); + removed = true; + } + } + } + return removed; + } + + protected abstract void onMapFull(); + + boolean isFull() { + if (size == 0) { + return false; + } + return map.size() >= size; + } + + private boolean isFull(K key) { + if (size == 0) { + return false; + } + if (map.size() >= size) { + return !map.containsKey(key); + } + return false; + } + + /* + * (non-Javadoc) + * @see java.util.Map#remove(java.lang.Object) + */ + @SuppressWarnings("unchecked") + @Override + public V remove(Object key) { + CachedValue entry = map.remove(key); + if (entry != null) { + onValueRemove(entry); + if (!entry.isExpired()) { + return (V) entry.getValue(); + } + } + return null; + } + + /* + * (non-Javadoc) + * @see java.util.Map#putAll(java.util.Map) + */ + @Override + public void putAll(Map m) { + removeExpiredEntries(); + for (Map.Entry entry : m.entrySet()) { + put(entry.getKey(), entry.getValue()); + } + } + + /* + * (non-Javadoc) + * @see java.util.Map#clear() + */ + @Override + public void clear() { + map.clear(); + } + + /* + * (non-Javadoc) + * @see java.util.Map#keySet() + */ + @Override + public Set keySet() { + removeExpiredEntries(); + return new KeySet(); + } + + /* + * (non-Javadoc) + * @see java.util.Map#values() + */ + @Override + public Collection values() { + removeExpiredEntries(); + return new Values(); + } + + /* + * (non-Javadoc) + * @see java.util.Map#entrySet() + */ + @Override + public Set> entrySet() { + removeExpiredEntries(); + return new EntrySet(); + } + + abstract class MapIterator implements Iterator { + + final Iterator> keyIterator = map.entrySet().iterator(); + + Map.Entry mapEntry; + + @Override + public boolean hasNext() { + if (mapEntry != null) { + return true; + } + mapEntry = null; + while (keyIterator.hasNext()) { + Map.Entry entry = keyIterator.next(); + if (entry.getValue().isExpired()) { + continue; + } + mapEntry = entry; + break; + } + return mapEntry != null; + } + + } + + final class KeySet extends AbstractSet { + + @Override + public Iterator iterator() { + return new MapIterator() { + @Override + public K next() { + if (mapEntry == null) { + throw new NoSuchElementException(); + } + + K key = mapEntry.getKey(); + mapEntry = null; + return key; + } + + @Override + public void remove() { + if (mapEntry == null) { + throw new IllegalStateException(); + } + map.remove(mapEntry.getKey()); + mapEntry = null; + } + }; + } + + @Override + public boolean contains(Object o) { + return AbstractCacheMap.this.containsKey(o); + } + + @Override + public boolean remove(Object o) { + return AbstractCacheMap.this.remove(o) != null; + } + + @Override + public int size() { + return AbstractCacheMap.this.size(); + } + + @Override + public void clear() { + AbstractCacheMap.this.clear(); + } + + } + + final class Values extends AbstractCollection { + + @Override + public Iterator iterator() { + return new MapIterator() { + @Override + public V next() { + if (mapEntry == null) { + throw new NoSuchElementException(); + } + + V value = readValue(mapEntry.getValue()); + mapEntry = null; + return value; + } + + @Override + public void remove() { + if (mapEntry == null) { + throw new IllegalStateException(); + } + map.remove(mapEntry.getKey(), mapEntry.getValue()); + mapEntry = null; + } + }; + } + + @Override + public boolean contains(Object o) { + return AbstractCacheMap.this.containsValue(o); + } + + @Override + public int size() { + return AbstractCacheMap.this.size(); + } + + @Override + public void clear() { + AbstractCacheMap.this.clear(); + } + + } + + final class EntrySet extends AbstractSet> { + + public final Iterator> iterator() { + return new MapIterator>() { + @Override + public Map.Entry next() { + if (mapEntry == null) { + throw new NoSuchElementException(); + } + + SimpleEntry result = new SimpleEntry(mapEntry.getKey(), readValue(mapEntry.getValue())); + mapEntry = null; + return result; + } + + @Override + public void remove() { + if (mapEntry == null) { + throw new IllegalStateException(); + } + map.remove(mapEntry.getKey(), mapEntry.getValue()); + mapEntry = null; + } + }; + } + + public final boolean contains(Object o) { + if (!(o instanceof Map.Entry)) + return false; + Map.Entry e = (Map.Entry) o; + Object key = e.getKey(); + V value = get(key); + return value != null && value.equals(e); + } + + public final boolean remove(Object o) { + if (o instanceof Map.Entry) { + Map.Entry e = (Map.Entry) o; + Object key = e.getKey(); + Object value = e.getValue(); + return AbstractCacheMap.this.map.remove(key, value); + } + return false; + } + + public final int size() { + return AbstractCacheMap.this.size(); + } + + public final void clear() { + AbstractCacheMap.this.clear(); + } + + } + +} diff --git a/redisson/src/main/java/org/redisson/misc/Cache.java b/redisson/src/main/java/org/redisson/misc/Cache.java new file mode 100644 index 000000000..f3a64c4ca --- /dev/null +++ b/redisson/src/main/java/org/redisson/misc/Cache.java @@ -0,0 +1,32 @@ +/** + * Copyright 2016 Nikita Koksharov + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.redisson.misc; + +import java.util.Map; +import java.util.concurrent.TimeUnit; + +/** + * + * @author Nikita Koksharov + * + * @param + * @param + */ +public interface Cache extends Map { + + V put(K key, V value, long ttl, TimeUnit ttlUnit, long maxIdleTime, TimeUnit maxIdleUnit); + +} diff --git a/redisson/src/main/java/org/redisson/misc/LFUCacheMap.java b/redisson/src/main/java/org/redisson/misc/LFUCacheMap.java new file mode 100644 index 000000000..107f435fd --- /dev/null +++ b/redisson/src/main/java/org/redisson/misc/LFUCacheMap.java @@ -0,0 +1,134 @@ +/** + * Copyright 2016 Nikita Koksharov + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.redisson.misc; + +import java.util.Map; +import java.util.concurrent.ConcurrentNavigableMap; +import java.util.concurrent.ConcurrentSkipListMap; + +/** + * LFU (least frequently used) cache. + * + * @author Nikita Koksharov + * + * @param + * @param + */ +public class LFUCacheMap extends AbstractCacheMap { + + public static class MapKey implements Comparable { + + private Long accessCount; + private CachedValue cachedValue; + + public MapKey(Long accessCount, CachedValue cachedValue) { + super(); + this.accessCount = accessCount; + this.cachedValue = cachedValue; + } + + @Override + public int compareTo(MapKey o) { + int compare = accessCount.compareTo(o.accessCount); + if (compare == 0) { + if (!cachedValue.equals(o.cachedValue)) { + return ((Integer)cachedValue.hashCode()).compareTo(o.cachedValue.hashCode()); + } + return 0; + } + return compare; + } + + @Override + public String toString() { + return "MapKey [accessCount=" + accessCount + "]"; + } + + } + + private final ConcurrentNavigableMap accessMap = new ConcurrentSkipListMap(); + + public LFUCacheMap(int size, long timeToLiveInMillis, long maxIdleInMillis) { + super(size, timeToLiveInMillis, maxIdleInMillis); + } + + @Override + protected void onValueCreate(CachedValue value) { + MapKey key = toKey(value); + accessMap.put(key, value); + } + + @Override + protected void onValueRead(CachedValue value) { + addAccessCount(value, 1); + } + + private MapKey toKey(CachedValue value) { + return new MapKey(value.getAccessCount(), value); + } + + @Override + protected void onValueRemove(CachedValue value) { + MapKey key = toKey(value); + if (accessMap.remove(key) == null) { + throw new IllegalStateException(); + } + } + + private void addAccessCount(CachedValue value, long count) { + synchronized (value) { + if (count < 0 && value.getAccessCount() == 0) { + return; + } + + MapKey key = toKey(value); + if (accessMap.remove(key) == null) { + throw new IllegalStateException(); + } + + if (count < 0) { + count = -Math.min(value.getAccessCount(), -count); + } + value.addAccessCount(count); + + key = toKey(value); + accessMap.put(key, value); + } + } + + @Override + protected void onMapFull() { + Map.Entry entry = accessMap.pollFirstEntry(); + map.remove(entry.getValue().getKey(), entry.getValue()); + + if (entry.getValue().getAccessCount() == 0) { + return; + } + + // TODO optimize + // decrease all values + for (CachedValue value : accessMap.values()) { + addAccessCount(value, -entry.getValue().getAccessCount()); + } + } + + @Override + public void clear() { + accessMap.clear(); + super.clear(); + } + +} diff --git a/redisson/src/main/java/org/redisson/misc/LRUCacheMap.java b/redisson/src/main/java/org/redisson/misc/LRUCacheMap.java new file mode 100644 index 000000000..c47ab0100 --- /dev/null +++ b/redisson/src/main/java/org/redisson/misc/LRUCacheMap.java @@ -0,0 +1,68 @@ +/** + * Copyright 2016 Nikita Koksharov + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.redisson.misc; + +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; + +/** + * LRU (least recently used) cache. + * + * @author Nikita Koksharov + * + * @param + * @param + */ +public class LRUCacheMap extends AbstractCacheMap { + + private final Queue queue = new ConcurrentLinkedQueue(); + + public LRUCacheMap(int size, long timeToLiveInMillis, long maxIdleInMillis) { + super(size, timeToLiveInMillis, maxIdleInMillis); + } + + @Override + protected void onValueCreate(CachedValue value) { + queue.add(value); + } + + @Override + protected void onValueRemove(CachedValue value) { + queue.remove(value); + } + + @Override + protected void onValueRead(CachedValue value) { + // move value to tail of queue + synchronized (value) { + queue.remove(value); + queue.add(value); + } + } + + @Override + protected void onMapFull() { + CachedValue value = queue.poll(); + map.remove(value.getKey(), value); + } + + @Override + public void clear() { + queue.clear(); + super.clear(); + } + +} diff --git a/redisson/src/main/java/org/redisson/misc/NoneCacheMap.java b/redisson/src/main/java/org/redisson/misc/NoneCacheMap.java new file mode 100644 index 000000000..303d597c2 --- /dev/null +++ b/redisson/src/main/java/org/redisson/misc/NoneCacheMap.java @@ -0,0 +1,48 @@ +/** + * Copyright 2016 Nikita Koksharov + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.redisson.misc; + +import java.util.concurrent.TimeUnit; + +/** + * + * @author Nikita Koksharov + * + * @param + * @param + */ +public class NoneCacheMap extends AbstractCacheMap { + + public NoneCacheMap(long timeToLiveInMillis, long maxIdleInMillis) { + super(0, timeToLiveInMillis, maxIdleInMillis); + } + + @Override + public V put(K key, V value, long ttl, TimeUnit ttlUnit, long maxIdleTime, TimeUnit maxIdleUnit) { + CachedValue entry = new CachedValue(key, value, ttlUnit.toMillis(ttl), maxIdleUnit.toMillis(maxIdleTime)); + onValueCreate(entry); + CachedValue prevCachedValue = map.put(key, entry); + if (prevCachedValue != null && !prevCachedValue.isExpired()) { + return (V) prevCachedValue.getValue(); + } + return null; + } + + @Override + protected void onMapFull() { + } + +} diff --git a/redisson/src/test/java/org/redisson/misc/LFUCacheMapTest.java b/redisson/src/test/java/org/redisson/misc/LFUCacheMapTest.java new file mode 100644 index 000000000..f272cc366 --- /dev/null +++ b/redisson/src/test/java/org/redisson/misc/LFUCacheMapTest.java @@ -0,0 +1,78 @@ +package org.redisson.misc; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.concurrent.TimeUnit; + +import org.junit.Test; + +public class LFUCacheMapTest { + + @Test + public void testMaxIdleTimeEviction() throws InterruptedException { + Cache map = new LFUCacheMap(2, 0, 0); + map.put(1, 0, 0, TimeUnit.MILLISECONDS, 400, TimeUnit.MILLISECONDS); + assertThat(map.get(1)).isEqualTo(0); + Thread.sleep(200); + assertThat(map.get(1)).isEqualTo(0); + Thread.sleep(200); + assertThat(map.get(1)).isEqualTo(0); + Thread.sleep(200); + assertThat(map.get(1)).isEqualTo(0); + Thread.sleep(410); + assertThat(map.keySet()).isEmpty(); + } + + @Test + public void testTTLEviction() throws InterruptedException { + Cache map = new LFUCacheMap(2, 0, 0); + map.put(1, 0, 500, TimeUnit.MILLISECONDS, 0, TimeUnit.MILLISECONDS); + assertThat(map.get(1)).isEqualTo(0); + Thread.sleep(100); + assertThat(map.get(1)).isEqualTo(0); + assertThat(map.keySet()).containsOnly(1); + Thread.sleep(500); + assertThat(map.keySet()).isEmpty(); + } + + @Test + public void testSizeLFUEviction() throws InterruptedException { + Cache map = new LFUCacheMap(3, 0, 0); + + map.put(1, 0); + map.put(2, 0); + map.put(6, 0); + + map.get(1); + map.put(3, 0); + + assertThat(map.keySet()).containsOnly(3, 1, 6); + + map.get(1); + map.put(4, 0); + + assertThat(map.keySet()).contains(4, 1).hasSize(3); + } + + @Test + public void testSizeEviction() throws InterruptedException { + Cache map = new LFUCacheMap(2, 0, 0); + map.put(1, 0); + map.put(2, 0); + + assertThat(map.keySet()).containsOnly(1, 2); + + map.put(3, 0); + + assertThat(map.keySet()).contains(3).hasSize(2); + + map.put(4, 0); + + assertThat(map.keySet()).contains(4).hasSize(2); + + map.put(5, 0); + + assertThat(map.keySet()).contains(5).hasSize(2); + } + +} diff --git a/redisson/src/test/java/org/redisson/misc/LRUCacheMapTest.java b/redisson/src/test/java/org/redisson/misc/LRUCacheMapTest.java new file mode 100644 index 000000000..530827416 --- /dev/null +++ b/redisson/src/test/java/org/redisson/misc/LRUCacheMapTest.java @@ -0,0 +1,73 @@ +package org.redisson.misc; + +import static org.assertj.core.api.Assertions.*; + +import java.util.concurrent.TimeUnit; + +import org.junit.Test; + +public class LRUCacheMapTest { + + @Test + public void testMaxIdleTimeEviction() throws InterruptedException { + Cache map = new LRUCacheMap(2, 0, 0); + map.put(1, 0, 0, TimeUnit.MILLISECONDS, 400, TimeUnit.MILLISECONDS); + assertThat(map.get(1)).isEqualTo(0); + Thread.sleep(200); + assertThat(map.get(1)).isEqualTo(0); + Thread.sleep(200); + assertThat(map.get(1)).isEqualTo(0); + Thread.sleep(200); + assertThat(map.get(1)).isEqualTo(0); + Thread.sleep(410); + assertThat(map.keySet()).isEmpty(); + } + + @Test + public void testTTLEviction() throws InterruptedException { + Cache map = new LRUCacheMap(2, 0, 0); + map.put(1, 0, 500, TimeUnit.MILLISECONDS, 0, TimeUnit.MILLISECONDS); + assertThat(map.get(1)).isEqualTo(0); + Thread.sleep(100); + assertThat(map.get(1)).isEqualTo(0); + assertThat(map.keySet()).containsOnly(1); + Thread.sleep(500); + assertThat(map.keySet()).isEmpty(); + } + + @Test + public void testSizeLRUEviction() throws InterruptedException { + Cache map = new LRUCacheMap(3, 0, 0); + map.put(1, 0); + map.put(2, 0); + map.put(5, 0); + + map.get(1); + map.put(3, 0); + + assertThat(map.keySet()).containsOnly(3, 1, 5); + + map.get(1); + map.put(4, 0); + + assertThat(map.keySet()).containsOnly(4, 1, 3); + } + + @Test + public void testSizeEviction() throws InterruptedException { + Cache map = new LRUCacheMap(2, 0, 0); + map.put(1, 0); + map.put(2, 0); + + assertThat(map.keySet()).containsOnly(1, 2); + + map.put(3, 0); + + assertThat(map.keySet()).containsOnly(3, 2); + + map.put(4, 0); + + assertThat(map.keySet()).containsOnly(4, 3); + } + +} diff --git a/redisson/src/test/java/org/redisson/misc/NoneCacheMapTest.java b/redisson/src/test/java/org/redisson/misc/NoneCacheMapTest.java new file mode 100644 index 000000000..63f5a58e8 --- /dev/null +++ b/redisson/src/test/java/org/redisson/misc/NoneCacheMapTest.java @@ -0,0 +1,51 @@ +package org.redisson.misc; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.concurrent.TimeUnit; + +import org.junit.Test; + +public class NoneCacheMapTest { + + @Test + public void testMaxIdleTimeEviction() throws InterruptedException { + Cache map = new NoneCacheMap(0, 0); + map.put(1, 0, 0, TimeUnit.MILLISECONDS, 400, TimeUnit.MILLISECONDS); + assertThat(map.get(1)).isEqualTo(0); + Thread.sleep(200); + assertThat(map.get(1)).isEqualTo(0); + Thread.sleep(200); + assertThat(map.get(1)).isEqualTo(0); + Thread.sleep(200); + assertThat(map.get(1)).isEqualTo(0); + Thread.sleep(410); + assertThat(map.keySet()).isEmpty(); + } + + @Test + public void testTTLEviction() throws InterruptedException { + Cache map = new NoneCacheMap(0, 0); + map.put(1, 0, 500, TimeUnit.MILLISECONDS, 0, TimeUnit.MILLISECONDS); + assertThat(map.get(1)).isEqualTo(0); + Thread.sleep(100); + assertThat(map.get(1)).isEqualTo(0); + assertThat(map.keySet()).containsOnly(1); + Thread.sleep(500); + assertThat(map.keySet()).isEmpty(); + } + + @Test + public void testSizeEviction() { + Cache map = new NoneCacheMap(0, 0); + map.put(1, 0); + map.put(2, 0); + + assertThat(map.keySet()).containsOnly(1, 2); + map.put(3, 0); + map.put(4, 0); + + assertThat(map.keySet()).containsOnly(1, 2, 3, 4); + } + +} From 038bedcbca1cdbafb76c400379fc5c6d754cc1aa Mon Sep 17 00:00:00 2001 From: Nikita Date: Wed, 31 Aug 2016 15:59:47 +0300 Subject: [PATCH 22/34] Executor should reject non-static inner task class. #602 --- .../org/redisson/RedissonExecutorService.java | 5 ++ .../executor/RedissonExecutorServiceTest.java | 55 +++++++++++++++++++ 2 files changed, 60 insertions(+) diff --git a/redisson/src/main/java/org/redisson/RedissonExecutorService.java b/redisson/src/main/java/org/redisson/RedissonExecutorService.java index 1dc7bb0c7..8447e51fd 100644 --- a/redisson/src/main/java/org/redisson/RedissonExecutorService.java +++ b/redisson/src/main/java/org/redisson/RedissonExecutorService.java @@ -19,6 +19,7 @@ import java.io.DataInputStream; import java.io.IOException; import java.io.InputStream; import java.lang.reflect.Field; +import java.lang.reflect.Modifier; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -438,6 +439,10 @@ public class RedissonExecutorService implements RScheduledExecutorService { if (task.getClass().isAnonymousClass()) { throw new IllegalArgumentException("Task can't be created using anonymous class"); } + if (task.getClass().isMemberClass() + && !Modifier.isStatic(task.getClass().getModifiers())) { + throw new IllegalArgumentException("Task class is an inner class and it should be static"); + } } private void execute(RemotePromise promise) { diff --git a/redisson/src/test/java/org/redisson/executor/RedissonExecutorServiceTest.java b/redisson/src/test/java/org/redisson/executor/RedissonExecutorServiceTest.java index fcf386e7a..000745a35 100644 --- a/redisson/src/test/java/org/redisson/executor/RedissonExecutorServiceTest.java +++ b/redisson/src/test/java/org/redisson/executor/RedissonExecutorServiceTest.java @@ -239,6 +239,61 @@ public class RedissonExecutorServiceTest extends BaseTest { } }); } + + public class TaskCallableClass implements Callable { + + @Override + public String call() throws Exception { + return "123"; + } + + } + + @Test(expected = IllegalArgumentException.class) + public void testNonStaticInnerClassCallable() { + redisson.getExecutorService("test").submit(new TaskCallableClass()); + } + + public static class TaskStaticCallableClass implements Callable { + + @Override + public String call() throws Exception { + return "123"; + } + + } + + @Test + public void testInnerClassCallable() throws InterruptedException, ExecutionException { + String res = redisson.getExecutorService("test").submit(new TaskStaticCallableClass()).get(); + assertThat(res).isEqualTo("123"); + } + + public class TaskRunnableClass implements Runnable { + + @Override + public void run() { + } + + } + + @Test(expected = IllegalArgumentException.class) + public void testNonStaticInnerClassRunnable() { + redisson.getExecutorService("test").submit(new TaskRunnableClass()); + } + + public static class TaskStaticRunnableClass implements Runnable { + + @Override + public void run() { + } + + } + + @Test + public void testInnerClassRunnable() throws InterruptedException, ExecutionException { + redisson.getExecutorService("test").submit(new TaskStaticRunnableClass()).get(); + } @Test(expected = IllegalArgumentException.class) public void testAnonymousRunnableExecute() { From 564258db791f1308b12b861aab415b3f0a472282 Mon Sep 17 00:00:00 2001 From: Nikita Date: Wed, 31 Aug 2016 16:58:46 +0300 Subject: [PATCH 23/34] refactoring --- .../org/redisson/misc/AbstractCacheMap.java | 18 +++--- .../java/org/redisson/misc/LFUCacheMap.java | 60 ++++++++++++------- .../java/org/redisson/misc/NoneCacheMap.java | 13 ---- 3 files changed, 47 insertions(+), 44 deletions(-) diff --git a/redisson/src/main/java/org/redisson/misc/AbstractCacheMap.java b/redisson/src/main/java/org/redisson/misc/AbstractCacheMap.java index 3355f67b4..78ab10e52 100644 --- a/redisson/src/main/java/org/redisson/misc/AbstractCacheMap.java +++ b/redisson/src/main/java/org/redisson/misc/AbstractCacheMap.java @@ -47,7 +47,6 @@ public abstract class AbstractCacheMap implements Cache { long creationTime; long lastAccess; - long accessCount; public CachedValue(Object key, Object value, long ttl, long maxIdleTime) { this.value = value; @@ -70,14 +69,6 @@ public abstract class AbstractCacheMap implements Cache { return result; } - public void addAccessCount(long value) { - accessCount += value; - } - - public Long getAccessCount() { - return accessCount; - } - public Object getKey() { return key; } @@ -89,7 +80,7 @@ public abstract class AbstractCacheMap implements Cache { @Override public String toString() { - return "CachedValue [key=" + key + ", value=" + value + ", accessCount=" + accessCount + "]"; + return "CachedValue [key=" + key + ", value=" + value + "]"; } } @@ -99,6 +90,7 @@ public abstract class AbstractCacheMap implements Cache { private final long timeToLiveInMillis; private final long maxIdleInMillis; + public AbstractCacheMap(int size, long timeToLiveInMillis, long maxIdleInMillis) { if (size < 0) { throw new IllegalArgumentException("Size can't be " + size); @@ -227,7 +219,7 @@ public abstract class AbstractCacheMap implements Cache { @SuppressWarnings("unchecked") @Override public V put(K key, V value, long ttl, TimeUnit ttlUnit, long maxIdleTime, TimeUnit maxIdleUnit) { - CachedValue entry = new CachedValue(key, value, ttlUnit.toMillis(ttl), maxIdleUnit.toMillis(maxIdleTime)); + CachedValue entry = create(key, value, ttlUnit.toMillis(ttl), maxIdleUnit.toMillis(maxIdleTime)); if (isFull(key)) { if (!removeExpiredEntries()) { onMapFull(); @@ -244,6 +236,10 @@ public abstract class AbstractCacheMap implements Cache { return null; } + protected CachedValue create(K key, V value, long ttl, long maxIdleTime) { + return new CachedValue(key, value, ttl, maxIdleTime); + } + protected void onValueCreate(CachedValue entry) { } diff --git a/redisson/src/main/java/org/redisson/misc/LFUCacheMap.java b/redisson/src/main/java/org/redisson/misc/LFUCacheMap.java index 107f435fd..1a1449926 100644 --- a/redisson/src/main/java/org/redisson/misc/LFUCacheMap.java +++ b/redisson/src/main/java/org/redisson/misc/LFUCacheMap.java @@ -18,6 +18,7 @@ package org.redisson.misc; import java.util.Map; import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.atomic.AtomicLong; /** * LFU (least frequently used) cache. @@ -32,9 +33,9 @@ public class LFUCacheMap extends AbstractCacheMap { public static class MapKey implements Comparable { private Long accessCount; - private CachedValue cachedValue; + private LFUCachedValue cachedValue; - public MapKey(Long accessCount, CachedValue cachedValue) { + public MapKey(Long accessCount, LFUCachedValue cachedValue) { super(); this.accessCount = accessCount; this.cachedValue = cachedValue; @@ -44,10 +45,7 @@ public class LFUCacheMap extends AbstractCacheMap { public int compareTo(MapKey o) { int compare = accessCount.compareTo(o.accessCount); if (compare == 0) { - if (!cachedValue.equals(o.cachedValue)) { - return ((Integer)cachedValue.hashCode()).compareTo(o.cachedValue.hashCode()); - } - return 0; + return cachedValue.id.compareTo(o.cachedValue.id); } return compare; } @@ -59,38 +57,60 @@ public class LFUCacheMap extends AbstractCacheMap { } - private final ConcurrentNavigableMap accessMap = new ConcurrentSkipListMap(); + public static class LFUCachedValue extends CachedValue { + + Long id; + long accessCount; + + public LFUCachedValue(long id, Object key, Object value, long ttl, long maxIdleTime) { + super(key, value, ttl, maxIdleTime); + this.id = id; + } + + public void addAccessCount(long value) { + accessCount += value; + } + + } + + private final AtomicLong idGenerator = new AtomicLong(); + private final ConcurrentNavigableMap accessMap = new ConcurrentSkipListMap(); public LFUCacheMap(int size, long timeToLiveInMillis, long maxIdleInMillis) { super(size, timeToLiveInMillis, maxIdleInMillis); } + @Override + protected CachedValue create(K key, V value, long ttl, long maxIdleTime) { + return new LFUCachedValue(idGenerator.incrementAndGet(), key, value, ttl, maxIdleTime); + } + @Override protected void onValueCreate(CachedValue value) { - MapKey key = toKey(value); - accessMap.put(key, value); + MapKey key = toKey((LFUCachedValue)value); + accessMap.put(key, (LFUCachedValue)value); } @Override protected void onValueRead(CachedValue value) { - addAccessCount(value, 1); + addAccessCount((LFUCachedValue)value, 1); } - private MapKey toKey(CachedValue value) { - return new MapKey(value.getAccessCount(), value); + private MapKey toKey(LFUCachedValue value) { + return new MapKey(value.accessCount, value); } @Override protected void onValueRemove(CachedValue value) { - MapKey key = toKey(value); + MapKey key = toKey((LFUCachedValue)value); if (accessMap.remove(key) == null) { throw new IllegalStateException(); } } - private void addAccessCount(CachedValue value, long count) { + private void addAccessCount(LFUCachedValue value, long count) { synchronized (value) { - if (count < 0 && value.getAccessCount() == 0) { + if (count < 0 && value.accessCount == 0) { return; } @@ -100,7 +120,7 @@ public class LFUCacheMap extends AbstractCacheMap { } if (count < 0) { - count = -Math.min(value.getAccessCount(), -count); + count = -Math.min(value.accessCount, -count); } value.addAccessCount(count); @@ -111,17 +131,17 @@ public class LFUCacheMap extends AbstractCacheMap { @Override protected void onMapFull() { - Map.Entry entry = accessMap.pollFirstEntry(); + Map.Entry entry = accessMap.pollFirstEntry(); map.remove(entry.getValue().getKey(), entry.getValue()); - if (entry.getValue().getAccessCount() == 0) { + if (entry.getValue().accessCount == 0) { return; } // TODO optimize // decrease all values - for (CachedValue value : accessMap.values()) { - addAccessCount(value, -entry.getValue().getAccessCount()); + for (LFUCachedValue value : accessMap.values()) { + addAccessCount(value, -entry.getValue().accessCount); } } diff --git a/redisson/src/main/java/org/redisson/misc/NoneCacheMap.java b/redisson/src/main/java/org/redisson/misc/NoneCacheMap.java index 303d597c2..e6533defa 100644 --- a/redisson/src/main/java/org/redisson/misc/NoneCacheMap.java +++ b/redisson/src/main/java/org/redisson/misc/NoneCacheMap.java @@ -15,8 +15,6 @@ */ package org.redisson.misc; -import java.util.concurrent.TimeUnit; - /** * * @author Nikita Koksharov @@ -30,17 +28,6 @@ public class NoneCacheMap extends AbstractCacheMap { super(0, timeToLiveInMillis, maxIdleInMillis); } - @Override - public V put(K key, V value, long ttl, TimeUnit ttlUnit, long maxIdleTime, TimeUnit maxIdleUnit) { - CachedValue entry = new CachedValue(key, value, ttlUnit.toMillis(ttl), maxIdleUnit.toMillis(maxIdleTime)); - onValueCreate(entry); - CachedValue prevCachedValue = map.put(key, entry); - if (prevCachedValue != null && !prevCachedValue.isExpired()) { - return (V) prevCachedValue.getValue(); - } - return null; - } - @Override protected void onMapFull() { } From 60b1c8fa1549fa30b8de1f6e90d444c7cd980d41 Mon Sep 17 00:00:00 2001 From: Nikita Date: Wed, 31 Aug 2016 17:46:52 +0300 Subject: [PATCH 24/34] Fixed - Can't find slave error in cluster mode if failed slave hasn't been added before. #593 --- .../cluster/ClusterConnectionManager.java | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/redisson/src/main/java/org/redisson/cluster/ClusterConnectionManager.java b/redisson/src/main/java/org/redisson/cluster/ClusterConnectionManager.java index e0d6249a5..a47b47506 100644 --- a/redisson/src/main/java/org/redisson/cluster/ClusterConnectionManager.java +++ b/redisson/src/main/java/org/redisson/cluster/ClusterConnectionManager.java @@ -296,9 +296,6 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { if (nodesIterator == null) { List nodes = new ArrayList(); List slaves = new ArrayList(); - if (lastPartitions.isEmpty()) { - System.out.println("lastPartitions.isEmpty()"); - } for (ClusterPartition partition : getLastPartitions()) { if (!partition.isMasterFail()) { @@ -391,17 +388,18 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { MasterSlaveEntry entry = getEntry(currentPart.getMasterAddr()); // should be invoked first in order to remove stale failedSlaveAddresses - addRemoveSlaves(entry, currentPart, newPart); - // Does some slaves change failed state to alive? - upDownSlaves(entry, currentPart, newPart); + Set addedSlaves = addRemoveSlaves(entry, currentPart, newPart); + // Do some slaves have changed state from failed to alive? + upDownSlaves(entry, currentPart, newPart, addedSlaves); break; } } } - private void upDownSlaves(final MasterSlaveEntry entry, final ClusterPartition currentPart, final ClusterPartition newPart) { + private void upDownSlaves(final MasterSlaveEntry entry, final ClusterPartition currentPart, final ClusterPartition newPart, Set addedSlaves) { Set aliveSlaves = new HashSet(currentPart.getFailedSlaveAddresses()); + aliveSlaves.removeAll(addedSlaves); aliveSlaves.removeAll(newPart.getFailedSlaveAddresses()); for (URI uri : aliveSlaves) { currentPart.removeFailedSlaveAddress(uri); @@ -420,7 +418,7 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { } } - private void addRemoveSlaves(final MasterSlaveEntry entry, final ClusterPartition currentPart, final ClusterPartition newPart) { + private Set addRemoveSlaves(final MasterSlaveEntry entry, final ClusterPartition currentPart, final ClusterPartition newPart) { Set removedSlaves = new HashSet(currentPart.getSlaveAddresses()); removedSlaves.removeAll(newPart.getSlaveAddresses()); @@ -450,6 +448,7 @@ public class ClusterConnectionManager extends MasterSlaveConnectionManager { } }); } + return addedSlaves; } private Collection slots(Collection partitions) { From d4c6716f851b6ee1da796c20017d40619926ab5a Mon Sep 17 00:00:00 2001 From: Nikita Koksharov Date: Wed, 31 Aug 2016 19:37:34 +0300 Subject: [PATCH 25/34] Update README.md --- README.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index aed4ebab5..94f118a57 100644 --- a/README.md +++ b/README.md @@ -74,7 +74,9 @@ Include the following to your dependency list: ### Downloads -[Redisson 2.3.0](https://repository.sonatype.org/service/local/artifact/maven/redirect?r=central-proxy&g=org.redisson&a=redisson&v=2.3.0&e=jar), [Redisson node 2.3.0](https://repository.sonatype.org/service/local/artifact/maven/redirect?r=central-proxy&g=org.redisson&a=redisson-all&v=2.3.0&e=jar) +[Redisson 2.3.0](https://repository.sonatype.org/service/local/artifact/maven/redirect?r=central-proxy&g=org.redisson&a=redisson&v=2.3.0&e=jar) +[Redisson node 2.3.0](https://repository.sonatype.org/service/local/artifact/maven/redirect?r=central-proxy&g=org.redisson&a=redisson-all&v=2.3.0&e=jar) +[Java data structures powered by Redis. Introduction to Redisson @ Redis Lightning Talks, May 9](http://redisson.org/Redisson.pdf) ### Supported by From e9d1c2214d49ec901e84ef78deb4f654ad6ec983 Mon Sep 17 00:00:00 2001 From: Nikita Koksharov Date: Wed, 31 Aug 2016 19:37:49 +0300 Subject: [PATCH 26/34] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 94f118a57..1666d13c1 100644 --- a/README.md +++ b/README.md @@ -75,7 +75,7 @@ Include the following to your dependency list: ### Downloads [Redisson 2.3.0](https://repository.sonatype.org/service/local/artifact/maven/redirect?r=central-proxy&g=org.redisson&a=redisson&v=2.3.0&e=jar) -[Redisson node 2.3.0](https://repository.sonatype.org/service/local/artifact/maven/redirect?r=central-proxy&g=org.redisson&a=redisson-all&v=2.3.0&e=jar) +[Redisson node 2.3.0](https://repository.sonatype.org/service/local/artifact/maven/redirect?r=central-proxy&g=org.redisson&a=redisson-all&v=2.3.0&e=jar) [Java data structures powered by Redis. Introduction to Redisson @ Redis Lightning Talks, May 9](http://redisson.org/Redisson.pdf) ### Supported by From fdac15673891152d68e1ae28a39c9509ae04463e Mon Sep 17 00:00:00 2001 From: Nikita Koksharov Date: Wed, 31 Aug 2016 19:38:17 +0300 Subject: [PATCH 27/34] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 1666d13c1..db95696b3 100644 --- a/README.md +++ b/README.md @@ -76,7 +76,7 @@ Include the following to your dependency list: [Redisson 2.3.0](https://repository.sonatype.org/service/local/artifact/maven/redirect?r=central-proxy&g=org.redisson&a=redisson&v=2.3.0&e=jar) [Redisson node 2.3.0](https://repository.sonatype.org/service/local/artifact/maven/redirect?r=central-proxy&g=org.redisson&a=redisson-all&v=2.3.0&e=jar) -[Java data structures powered by Redis. Introduction to Redisson @ Redis Lightning Talks, May 9](http://redisson.org/Redisson.pdf) +[Java data structures powered by Redis. Introduction to Redisson. PDF](http://redisson.org/Redisson.pdf) ### Supported by From cda8a00b2144441050933220a84250f49d2e3dcc Mon Sep 17 00:00:00 2001 From: Nikita Koksharov Date: Wed, 31 Aug 2016 19:44:18 +0300 Subject: [PATCH 28/34] Update README.md --- README.md | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index db95696b3..ca9347815 100644 --- a/README.md +++ b/README.md @@ -58,6 +58,13 @@ Projects using Redisson ================================ [Setronica](http://setronica.com/), [Monits](http://monits.com/), [Brookhaven National Laboratory](http://bnl.gov/), [Netflix Dyno client] (https://github.com/Netflix/dyno), [武林Q传](http://www.nbrpg.com/), [Ocous](http://www.ocous.com/), [Invaluable](http://www.invaluable.com/), [Clover](https://www.clover.com/) , [Apache Karaf Decanter](https://karaf.apache.org/projects.html#decanter), [Atmosphere Framework](http://async-io.org/) +### Articles + +[Java data structures powered by Redis. Introduction to Redisson. PDF](http://redisson.org/Redisson.pdf) +[Introducing Redisson Live Objects (Object Hash Mapping)](https://dzone.com/articles/introducing-redisson-live-object-object-hash-mappi) +[Java Remote Method Invocation with Redisson](https://dzone.com/articles/java-remote-method-invocation-with-redisson) +[Java Multimaps With Redis](https://dzone.com/articles/multimaps-with-redis) + ### Maven Include the following to your dependency list: @@ -76,7 +83,6 @@ Include the following to your dependency list: [Redisson 2.3.0](https://repository.sonatype.org/service/local/artifact/maven/redirect?r=central-proxy&g=org.redisson&a=redisson&v=2.3.0&e=jar) [Redisson node 2.3.0](https://repository.sonatype.org/service/local/artifact/maven/redirect?r=central-proxy&g=org.redisson&a=redisson-all&v=2.3.0&e=jar) -[Java data structures powered by Redis. Introduction to Redisson. PDF](http://redisson.org/Redisson.pdf) ### Supported by From 79f0e9c45986fb070d6e4bc428bcdb5e6ae64b31 Mon Sep 17 00:00:00 2001 From: Nikita Koksharov Date: Wed, 31 Aug 2016 20:07:40 +0300 Subject: [PATCH 29/34] Create README.md --- redisson-all/README.md | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 redisson-all/README.md diff --git a/redisson-all/README.md b/redisson-all/README.md new file mode 100644 index 000000000..20eb81897 --- /dev/null +++ b/redisson-all/README.md @@ -0,0 +1,5 @@ +## Redisson standalone node + +Redisson offers ability to run as standalone node and participate in distributed computing. Such standalone nodes could be used to run [ExecutorService](./9.-distributed-services#93-executor-service), [ScheduledExecutorService](https://github.com/mrniko/redisson/wiki/9.-distributed-services#94-scheduled-executor-service) tasks or [RemoteService](./9.-distributed-services#91-remote-service) services. It's just a single jar and could be downloaded from [here](https://repository.sonatype.org/service/local/artifact/maven/redirect?r=central-proxy&g=org.redisson&a=redisson-all&v=2.3.0&e=jar) + +[Documenation](https://github.com/mrniko/redisson/wiki/12.-Standalone-node) about Redisson standalone node. From edf2d346bfe0295440b412fc5e25e8f013bbe711 Mon Sep 17 00:00:00 2001 From: Nikita Koksharov Date: Wed, 31 Aug 2016 20:33:26 +0300 Subject: [PATCH 30/34] Update README.md --- redisson-all/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/redisson-all/README.md b/redisson-all/README.md index 20eb81897..3d734b00c 100644 --- a/redisson-all/README.md +++ b/redisson-all/README.md @@ -2,4 +2,4 @@ Redisson offers ability to run as standalone node and participate in distributed computing. Such standalone nodes could be used to run [ExecutorService](./9.-distributed-services#93-executor-service), [ScheduledExecutorService](https://github.com/mrniko/redisson/wiki/9.-distributed-services#94-scheduled-executor-service) tasks or [RemoteService](./9.-distributed-services#91-remote-service) services. It's just a single jar and could be downloaded from [here](https://repository.sonatype.org/service/local/artifact/maven/redirect?r=central-proxy&g=org.redisson&a=redisson-all&v=2.3.0&e=jar) -[Documenation](https://github.com/mrniko/redisson/wiki/12.-Standalone-node) about Redisson standalone node. +[Documentation](https://github.com/mrniko/redisson/wiki/12.-Standalone-node) about Redisson standalone node. From ebec4744c043210b05e7546935df7dac4236a04f Mon Sep 17 00:00:00 2001 From: Nikita Date: Fri, 2 Sep 2016 11:12:20 +0300 Subject: [PATCH 31/34] Fixed race condition during cache entry removing. #592 --- .../main/java/org/redisson/misc/LFUCacheMap.java | 11 +++++++---- .../main/java/org/redisson/misc/LRUCacheMap.java | 13 +++++++++---- 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/redisson/src/main/java/org/redisson/misc/LFUCacheMap.java b/redisson/src/main/java/org/redisson/misc/LFUCacheMap.java index 1a1449926..49b6a2d1d 100644 --- a/redisson/src/main/java/org/redisson/misc/LFUCacheMap.java +++ b/redisson/src/main/java/org/redisson/misc/LFUCacheMap.java @@ -102,9 +102,9 @@ public class LFUCacheMap extends AbstractCacheMap { @Override protected void onValueRemove(CachedValue value) { - MapKey key = toKey((LFUCachedValue)value); - if (accessMap.remove(key) == null) { - throw new IllegalStateException(); + synchronized (value) { + MapKey key = toKey((LFUCachedValue)value); + accessMap.remove(key); } } @@ -116,7 +116,7 @@ public class LFUCacheMap extends AbstractCacheMap { MapKey key = toKey(value); if (accessMap.remove(key) == null) { - throw new IllegalStateException(); + return; } if (count < 0) { @@ -132,6 +132,9 @@ public class LFUCacheMap extends AbstractCacheMap { @Override protected void onMapFull() { Map.Entry entry = accessMap.pollFirstEntry(); + if (entry == null) { + return; + } map.remove(entry.getValue().getKey(), entry.getValue()); if (entry.getValue().accessCount == 0) { diff --git a/redisson/src/main/java/org/redisson/misc/LRUCacheMap.java b/redisson/src/main/java/org/redisson/misc/LRUCacheMap.java index c47ab0100..f526837ca 100644 --- a/redisson/src/main/java/org/redisson/misc/LRUCacheMap.java +++ b/redisson/src/main/java/org/redisson/misc/LRUCacheMap.java @@ -41,22 +41,27 @@ public class LRUCacheMap extends AbstractCacheMap { @Override protected void onValueRemove(CachedValue value) { - queue.remove(value); + synchronized (value) { + queue.remove(value); + } } @Override protected void onValueRead(CachedValue value) { // move value to tail of queue synchronized (value) { - queue.remove(value); - queue.add(value); + if (queue.remove(value)) { + queue.add(value); + } } } @Override protected void onMapFull() { CachedValue value = queue.poll(); - map.remove(value.getKey(), value); + if (value != null) { + map.remove(value.getKey(), value); + } } @Override From 825a65371f7bca65f384479f5ca75e12c8fed337 Mon Sep 17 00:00:00 2001 From: Nikita Date: Fri, 2 Sep 2016 11:29:49 +0300 Subject: [PATCH 32/34] comments added --- .../src/main/java/org/redisson/client/codec/BitSetCodec.java | 5 +++++ .../main/java/org/redisson/client/codec/ByteArrayCodec.java | 5 +++++ .../src/main/java/org/redisson/client/codec/ScanCodec.java | 5 +++++ redisson/src/main/java/org/redisson/codec/KryoCodec.java | 5 +++++ 4 files changed, 20 insertions(+) diff --git a/redisson/src/main/java/org/redisson/client/codec/BitSetCodec.java b/redisson/src/main/java/org/redisson/client/codec/BitSetCodec.java index 356472a70..6a45ab499 100644 --- a/redisson/src/main/java/org/redisson/client/codec/BitSetCodec.java +++ b/redisson/src/main/java/org/redisson/client/codec/BitSetCodec.java @@ -23,6 +23,11 @@ import org.redisson.client.protocol.Encoder; import io.netty.buffer.ByteBuf; +/** + * + * @author Nikita Koksharov + * + */ public class BitSetCodec implements Codec { public static final BitSetCodec INSTANCE = new BitSetCodec(); diff --git a/redisson/src/main/java/org/redisson/client/codec/ByteArrayCodec.java b/redisson/src/main/java/org/redisson/client/codec/ByteArrayCodec.java index b6057aca5..3768d638e 100644 --- a/redisson/src/main/java/org/redisson/client/codec/ByteArrayCodec.java +++ b/redisson/src/main/java/org/redisson/client/codec/ByteArrayCodec.java @@ -23,6 +23,11 @@ import org.redisson.client.protocol.Encoder; import io.netty.buffer.ByteBuf; +/** + * + * @author Nikita Koksharov + * + */ public class ByteArrayCodec implements Codec { public static final ByteArrayCodec INSTANCE = new ByteArrayCodec(); diff --git a/redisson/src/main/java/org/redisson/client/codec/ScanCodec.java b/redisson/src/main/java/org/redisson/client/codec/ScanCodec.java index ac760c0bb..407eb13ce 100644 --- a/redisson/src/main/java/org/redisson/client/codec/ScanCodec.java +++ b/redisson/src/main/java/org/redisson/client/codec/ScanCodec.java @@ -25,6 +25,11 @@ import org.redisson.client.protocol.decoder.ScanObjectEntry; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +/** + * + * @author Nikita Koksharov + * + */ public class ScanCodec implements Codec { private final Codec delegate; diff --git a/redisson/src/main/java/org/redisson/codec/KryoCodec.java b/redisson/src/main/java/org/redisson/codec/KryoCodec.java index d368509d0..6469106e5 100755 --- a/redisson/src/main/java/org/redisson/codec/KryoCodec.java +++ b/redisson/src/main/java/org/redisson/codec/KryoCodec.java @@ -34,6 +34,11 @@ import com.esotericsoftware.kryo.io.Output; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufInputStream; +/** + * + * @author Nikita Koksharov + * + */ public class KryoCodec implements Codec { public interface KryoPool { From 3d4b5d799bdd7d91b21f125f01801cc26058ac10 Mon Sep 17 00:00:00 2001 From: Nikita Date: Fri, 2 Sep 2016 12:21:51 +0300 Subject: [PATCH 33/34] removed unnecessary sync in LRUCacheMap #592 --- .../src/main/java/org/redisson/misc/LRUCacheMap.java | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/redisson/src/main/java/org/redisson/misc/LRUCacheMap.java b/redisson/src/main/java/org/redisson/misc/LRUCacheMap.java index f526837ca..112190b6c 100644 --- a/redisson/src/main/java/org/redisson/misc/LRUCacheMap.java +++ b/redisson/src/main/java/org/redisson/misc/LRUCacheMap.java @@ -41,18 +41,14 @@ public class LRUCacheMap extends AbstractCacheMap { @Override protected void onValueRemove(CachedValue value) { - synchronized (value) { - queue.remove(value); - } + queue.remove(value); } @Override protected void onValueRead(CachedValue value) { // move value to tail of queue - synchronized (value) { - if (queue.remove(value)) { - queue.add(value); - } + if (queue.remove(value)) { + queue.add(value); } } From 5261be134ed276d105f9336dc239ca9b87e64c57 Mon Sep 17 00:00:00 2001 From: Nikita Date: Fri, 2 Sep 2016 16:12:32 +0300 Subject: [PATCH 34/34] RLocalCachedMap implemented. #592 --- redisson/pom.xml | 18 +- .../src/main/java/org/redisson/Redisson.java | 12 + .../org/redisson/RedissonLocalCachedMap.java | 690 ++++++++++++++++++ .../java/org/redisson/RedissonObject.java | 16 + .../redisson/api/LocalCachedMapOptions.java | 174 +++++ .../org/redisson/api/RLocalCachedMap.java | 58 ++ .../redisson/api/RLocalCachedMapAsync.java | 100 +++ .../java/org/redisson/api/RedissonClient.java | 21 + .../src/main/java/org/redisson/misc/Hash.java | 13 + .../redisson/RedissonLocalCachedMapTest.java | 249 +++++++ 10 files changed, 1345 insertions(+), 6 deletions(-) create mode 100644 redisson/src/main/java/org/redisson/RedissonLocalCachedMap.java create mode 100644 redisson/src/main/java/org/redisson/api/LocalCachedMapOptions.java create mode 100644 redisson/src/main/java/org/redisson/api/RLocalCachedMap.java create mode 100644 redisson/src/main/java/org/redisson/api/RLocalCachedMapAsync.java create mode 100644 redisson/src/test/java/org/redisson/RedissonLocalCachedMapTest.java diff --git a/redisson/pom.xml b/redisson/pom.xml index c0f072099..ca9da989f 100644 --- a/redisson/pom.xml +++ b/redisson/pom.xml @@ -41,33 +41,33 @@ io.netty netty-transport-native-epoll - 4.0.40.Final + 4.0.41.Final provided io.netty netty-common - 4.0.40.Final + 4.0.41.Final io.netty netty-codec - 4.0.40.Final + 4.0.41.Final io.netty netty-buffer - 4.0.40.Final + 4.0.41.Final io.netty netty-transport - 4.0.40.Final + 4.0.41.Final io.netty netty-handler - 4.0.40.Final + 4.0.41.Final @@ -106,6 +106,12 @@ 1.7.12 test + + org.jmockit + jmockit + 1.27 + test + net.jpountz.lz4 diff --git a/redisson/src/main/java/org/redisson/Redisson.java b/redisson/src/main/java/org/redisson/Redisson.java index 6e20e819f..d4336f6e7 100755 --- a/redisson/src/main/java/org/redisson/Redisson.java +++ b/redisson/src/main/java/org/redisson/Redisson.java @@ -20,6 +20,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; import org.redisson.api.ClusterNodesGroup; +import org.redisson.api.LocalCachedMapOptions; import org.redisson.api.Node; import org.redisson.api.NodesGroup; import org.redisson.api.RAtomicDouble; @@ -42,6 +43,7 @@ import org.redisson.api.RList; import org.redisson.api.RListMultimap; import org.redisson.api.RListMultimapCache; import org.redisson.api.RLiveObjectService; +import org.redisson.api.RLocalCachedMap; import org.redisson.api.RLock; import org.redisson.api.RMap; import org.redisson.api.RMapCache; @@ -216,6 +218,16 @@ public class Redisson implements RedissonClient { return new RedissonListMultimap(codec, commandExecutor, name); } + @Override + public RLocalCachedMap getLocalCachedMap(String name, LocalCachedMapOptions options) { + return new RedissonLocalCachedMap(this, commandExecutor, name, options); + } + + @Override + public RLocalCachedMap getLocalCachedMap(String name, Codec codec, LocalCachedMapOptions options) { + return new RedissonLocalCachedMap(this, codec, commandExecutor, name, options); + } + @Override public RMap getMap(String name) { return new RedissonMap(commandExecutor, name); diff --git a/redisson/src/main/java/org/redisson/RedissonLocalCachedMap.java b/redisson/src/main/java/org/redisson/RedissonLocalCachedMap.java new file mode 100644 index 000000000..503085e21 --- /dev/null +++ b/redisson/src/main/java/org/redisson/RedissonLocalCachedMap.java @@ -0,0 +1,690 @@ +/** + * Copyright 2016 Nikita Koksharov + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.redisson; + +import java.util.AbstractCollection; +import java.util.AbstractMap; +import java.util.AbstractSet; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; + +import org.redisson.api.LocalCachedMapOptions; +import org.redisson.api.LocalCachedMapOptions.EvictionPolicy; +import org.redisson.api.RFuture; +import org.redisson.api.RLocalCachedMap; +import org.redisson.api.RMap; +import org.redisson.api.RTopic; +import org.redisson.api.RedissonClient; +import org.redisson.api.listener.MessageListener; +import org.redisson.client.codec.Codec; +import org.redisson.client.codec.LongCodec; +import org.redisson.client.protocol.RedisCommand; +import org.redisson.client.protocol.RedisCommand.ValueType; +import org.redisson.client.protocol.RedisCommands; +import org.redisson.command.CommandAsyncExecutor; +import org.redisson.misc.Cache; +import org.redisson.misc.Hash; +import org.redisson.misc.LFUCacheMap; +import org.redisson.misc.LRUCacheMap; +import org.redisson.misc.NoneCacheMap; + +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.FutureListener; + +/** + * + * @author Nikita Koksharov + * + */ +public class RedissonLocalCachedMap extends RedissonExpirable implements RLocalCachedMap { + + public static class LocalCachedMapClear { + + } + + public static class LocalCachedMapInvalidate { + + private byte[] keyHash; + + public LocalCachedMapInvalidate() { + } + + public LocalCachedMapInvalidate(byte[] keyHash) { + super(); + this.keyHash = keyHash; + } + + public byte[] getKeyHash() { + return keyHash; + } + + } + + public static class CacheKey { + + private final byte[] keyHash; + + public CacheKey(byte[] keyHash) { + super(); + this.keyHash = keyHash; + } + + public byte[] getKeyHash() { + return keyHash; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + Arrays.hashCode(keyHash); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + CacheKey other = (CacheKey) obj; + if (!Arrays.equals(keyHash, other.keyHash)) + return false; + return true; + } + + @Override + public String toString() { + return "CacheKey [keyHash=" + Arrays.toString(keyHash) + "]"; + } + + } + + public static class CacheValue { + + private final Object key; + private final Object value; + + public CacheValue(Object key, Object value) { + super(); + this.key = key; + this.value = value; + } + + public Object getKey() { + return key; + } + + public Object getValue() { + return value; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + CacheValue other = (CacheValue) obj; + if (value == null) { + if (other.value != null) + return false; + } else if (!value.equals(other.value)) + return false; + return true; + } + + @Override + public String toString() { + return "CacheValue [key=" + key + ", value=" + value + "]"; + } + + } + + private static final RedisCommand EVAL_PUT = new RedisCommand("EVAL", -1, ValueType.OBJECT, ValueType.MAP_VALUE); + private static final RedisCommand EVAL_REMOVE = new RedisCommand("EVAL", -1, ValueType.OBJECT, ValueType.MAP_VALUE); + + private RTopic invalidationTopic; + private RMap map; + private Cache cache; + private int invalidateEntryOnChange; + + protected RedissonLocalCachedMap(RedissonClient redisson, CommandAsyncExecutor commandExecutor, String name, LocalCachedMapOptions options) { + super(commandExecutor, name); + init(redisson, name, options); + } + + protected RedissonLocalCachedMap(RedissonClient redisson, Codec codec, CommandAsyncExecutor connectionManager, String name, LocalCachedMapOptions options) { + super(codec, connectionManager, name); + init(redisson, name, options); + } + + private void init(RedissonClient redisson, String name, LocalCachedMapOptions options) { + map = redisson.getMap(name); + + if (options.isInvalidateEntryOnChange()) { + invalidateEntryOnChange = 1; + } + if (options.getEvictionPolicy() == EvictionPolicy.NONE) { + cache = new NoneCacheMap(options.getTimeToLiveInMillis(), options.getMaxIdleInMillis()); + } + if (options.getEvictionPolicy() == EvictionPolicy.LRU) { + cache = new LRUCacheMap(options.getCacheSize(), options.getTimeToLiveInMillis(), options.getMaxIdleInMillis()); + } + if (options.getEvictionPolicy() == EvictionPolicy.LFU) { + cache = new LFUCacheMap(options.getCacheSize(), options.getTimeToLiveInMillis(), options.getMaxIdleInMillis()); + } + + invalidationTopic = redisson.getTopic(name + ":topic"); + invalidationTopic.addListener(new MessageListener() { + @Override + public void onMessage(String channel, Object msg) { + if (msg instanceof LocalCachedMapClear) { + cache.clear(); + } + if (msg instanceof LocalCachedMapInvalidate) { + CacheKey key = new CacheKey(((LocalCachedMapInvalidate)msg).getKeyHash()); + cache.remove(key); + } + } + }); + } + + @Override + public int size() { + return get(sizeAsync()); + } + + @Override + public RFuture sizeAsync() { + return map.sizeAsync(); + } + + @Override + public boolean isEmpty() { + return map.isEmpty(); + } + + @Override + public boolean containsKey(Object key) { + return get(containsKeyAsync(key)); + } + + private CacheKey toCacheKey(Object key) { + byte[] encoded = encodeMapKey(key); + return toCacheKey(encoded); + } + + private CacheKey toCacheKey(byte[] encodedKey) { + return new CacheKey(Hash.hash(encodedKey)); + } + + @Override + public RFuture containsKeyAsync(Object key) { + CacheKey cacheKey = toCacheKey(key); + if (!cache.containsKey(cacheKey)) { + return map.containsKeyAsync(key); + } + return newSucceededFuture(true); + } + + @Override + public boolean containsValue(Object value) { + return get(containsValueAsync(value)); + } + + @Override + public RFuture containsValueAsync(Object value) { + CacheValue cacheValue = new CacheValue(null, value); + if (!cache.containsValue(cacheValue)) { + return map.containsValueAsync(value); + } + return newSucceededFuture(true); + } + + @Override + public V get(Object key) { + return get(getAsync(key)); + } + + @Override + public RFuture getAsync(final Object key) { + if (key == null) { + throw new NullPointerException(); + } + + final CacheKey cacheKey = toCacheKey(key); + CacheValue cacheValue = cache.get(cacheKey); + if (cacheValue != null && cacheValue.getValue() != null) { + return newSucceededFuture((V)cacheValue.getValue()); + } + + RFuture future = map.getAsync((K)key); + future.addListener(new FutureListener() { + @Override + public void operationComplete(Future future) throws Exception { + if (!future.isSuccess()) { + return; + } + + V value = future.getNow(); + if (value != null) { + cache.put(cacheKey, new CacheValue(key, value)); + } + } + }); + return future; + } + + + @Override + public V put(K key, V value) { + return get(putAsync(key, value)); + } + + @Override + public RFuture putAsync(K key, V value) { + if (key == null) { + throw new NullPointerException(); + } + if (value == null) { + throw new NullPointerException(); + } + + byte[] mapKey = encodeMapKey(key); + CacheKey cacheKey = toCacheKey(mapKey); + byte[] msg = encode(new LocalCachedMapInvalidate(cacheKey.getKeyHash())); + CacheValue cacheValue = new CacheValue(key, value); + cache.put(cacheKey, cacheValue); + return commandExecutor.evalWriteAsync(getName(), codec, EVAL_PUT, + "local v = redis.call('hget', KEYS[1], ARGV[1]); " + + "if redis.call('hset', KEYS[1], ARGV[1], ARGV[2]) == 0 and ARGV[4] == '1' then " + + "redis.call('publish', KEYS[2], ARGV[3]); " + + "end; " + + "return v; ", + Arrays.asList(getName(), invalidationTopic.getChannelNames().get(0)), + mapKey, encodeMapValue(value), msg, invalidateEntryOnChange); + } + + @Override + public boolean fastPut(K key, V value) { + return get(fastPutAsync(key, value)); + } + + @Override + public RFuture fastPutAsync(K key, V value) { + if (key == null) { + throw new NullPointerException(); + } + if (value == null) { + throw new NullPointerException(); + } + + byte[] encodedKey = encodeMapKey(key); + byte[] encodedValue = encodeMapKey(value); + CacheKey cacheKey = toCacheKey(encodedKey); + byte[] msg = encode(new LocalCachedMapInvalidate(cacheKey.getKeyHash())); + CacheValue cacheValue = new CacheValue(key, value); + cache.put(cacheKey, cacheValue); + return commandExecutor.evalWriteAsync(getName(), codec, RedisCommands.EVAL_BOOLEAN, + "if redis.call('hset', KEYS[1], ARGV[1], ARGV[2]) == 0 then " + + "if ARGV[4] == '1' then " + + "redis.call('publish', KEYS[2], ARGV[3]); " + + "end;" + + "return 0; " + + "end; " + + "return 1; ", + Arrays.asList(getName(), invalidationTopic.getChannelNames().get(0)), + encodedKey, encodedValue, msg, invalidateEntryOnChange); + } + + @Override + public V remove(Object key) { + return get(removeAsync((K)key)); + } + + @Override + public RFuture removeAsync(K key) { + if (key == null) { + throw new NullPointerException(); + } + + byte[] keyEncoded = encodeMapKey(key); + CacheKey cacheKey = toCacheKey(keyEncoded); + byte[] msgEncoded = encode(new LocalCachedMapInvalidate(cacheKey.getKeyHash())); + cache.remove(cacheKey); + return commandExecutor.evalWriteAsync(getName(), codec, EVAL_REMOVE, + "local v = redis.call('hget', KEYS[1], ARGV[1]); " + + "if redis.call('hdel', KEYS[1], ARGV[1]) == 1 and ARGV[3] == '1' then " + + "redis.call('publish', KEYS[2], ARGV[2]); " + + "end; " + + "return v", + Arrays.asList(getName(), invalidationTopic.getChannelNames().get(0)), + keyEncoded, msgEncoded, invalidateEntryOnChange); + } + + @Override + public boolean fastRemove(Object key) { + return get(fastRemoveAsync((K)key)); + } + + @Override + public RFuture fastRemoveAsync(K key) { + if (key == null) { + throw new NullPointerException(); + } + + byte[] keyEncoded = encodeMapKey(key); + CacheKey cacheKey = toCacheKey(keyEncoded); + byte[] msgEncoded = encode(new LocalCachedMapInvalidate(cacheKey.getKeyHash())); + cache.remove(cacheKey); + return commandExecutor.evalWriteAsync(getName(), codec, RedisCommands.EVAL_BOOLEAN, + "if redis.call('hdel', KEYS[1], ARGV[1]) == 1 then " + + "if ARGV[3] == '1' then " + + "redis.call('publish', KEYS[2], ARGV[2]); " + + "end; " + + "return 1;" + + "end;" + + "return 0;", + Arrays.asList(getName(), invalidationTopic.getChannelNames().get(0)), + keyEncoded, msgEncoded, invalidateEntryOnChange); + } + + + @Override + public void putAll(Map m) { + Map cacheMap = new HashMap(m.size()); + for (java.util.Map.Entry entry : m.entrySet()) { + CacheKey cacheKey = toCacheKey(entry.getKey()); + CacheValue cacheValue = new CacheValue(entry.getKey(), entry.getValue()); + cacheMap.put(cacheKey, cacheValue); + } + cache.putAll(cacheMap); + map.putAll(m); + for (CacheKey cacheKey : cacheMap.keySet()) { + invalidationTopic.publish(new LocalCachedMapInvalidate(cacheKey.getKeyHash())); + } + } + + @Override + public void clear() { + delete(); + } + + @Override + public RFuture deleteAsync() { + cache.clear(); + byte[] msgEncoded = encode(new LocalCachedMapClear()); + return commandExecutor.evalWriteAsync(getName(), LongCodec.INSTANCE, RedisCommands.EVAL_BOOLEAN, + "if redis.call('del', KEYS[1]) == 1 and ARGV[2] == '1' then " + + "redis.call('publish', KEYS[2], ARGV[1]); " + + "end; ", + Arrays.asList(getName(), invalidationTopic.getChannelNames().get(0)), + msgEncoded, invalidateEntryOnChange); + } + + @Override + public Set keySet() { + return new KeySet(); + } + + @Override + public Collection values() { + return new Values(); + } + + @Override + public Set> entrySet() { + return new EntrySet(); + } + + private Iterator> cacheEntrySetIterator() { + final Iterator> iter = cache.entrySet().iterator(); + + return new Iterator>() { + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public java.util.Map.Entry next() { + Map.Entry entry = iter.next(); + return new AbstractMap.SimpleEntry(entry.getValue().getKey(), entry.getValue().getValue()); + } + + @Override + public void remove() { + iter.remove(); + } + + }; + } + + private Iterator cacheKeySetIterator() { + final Iterator iter = cache.values().iterator(); + + return new Iterator() { + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public K next() { + CacheValue value = iter.next(); + return (K) value.getKey(); + } + + @Override + public void remove() { + iter.remove(); + } + + }; + } + + + final class KeySet extends AbstractSet { + + @Override + public Iterator iterator() { + return new CompositeIterable(cacheKeySetIterator(), map.keySet().iterator()) { + + @Override + boolean isCacheContains(Object object) { + CacheKey cacheKey = toCacheKey(object); + return cache.containsKey(cacheKey); + } + + }; + } + + @Override + public boolean contains(Object o) { + return RedissonLocalCachedMap.this.containsKey(o); + } + + @Override + public boolean remove(Object o) { + return RedissonLocalCachedMap.this.remove(o) != null; + } + + @Override + public int size() { + return RedissonLocalCachedMap.this.size(); + } + + @Override + public void clear() { + RedissonLocalCachedMap.this.clear(); + } + + } + + final class Values extends AbstractCollection { + + @Override + public Iterator iterator() { + final Iterator> iter = RedissonLocalCachedMap.this.entrySet().iterator(); + return new Iterator() { + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public V next() { + return iter.next().getValue(); + } + + @Override + public void remove() { + iter.remove(); + } + }; + } + + @Override + public boolean contains(Object o) { + return RedissonLocalCachedMap.this.containsValue(o); + } + + @Override + public int size() { + return RedissonLocalCachedMap.this.size(); + } + + @Override + public void clear() { + RedissonLocalCachedMap.this.clear(); + } + + } + + final class EntrySet extends AbstractSet> { + + public final Iterator> iterator() { + return new CompositeIterable>(cacheEntrySetIterator(), map.entrySet().iterator()) { + + @Override + boolean isCacheContains(Map.Entry entry) { + CacheKey cacheKey = toCacheKey(entry.getKey()); + return cache.containsKey(cacheKey); + } + + }; + } + + public final boolean contains(Object o) { + if (!(o instanceof Map.Entry)) + return false; + Map.Entry e = (Map.Entry) o; + Object key = e.getKey(); + V value = get(key); + return value != null && value.equals(e); + } + + public final boolean remove(Object o) { + if (o instanceof Map.Entry) { + Map.Entry e = (Map.Entry) o; + Object key = e.getKey(); + Object value = e.getValue(); + return RedissonLocalCachedMap.this.map.remove(key, value); + } + return false; + } + + public final int size() { + return RedissonLocalCachedMap.this.size(); + } + + public final void clear() { + RedissonLocalCachedMap.this.clear(); + } + + } + + abstract class CompositeIterable implements Iterator { + + private T currentObject; + private Iterator cacheIterator; + private Iterator mapIterator; + + public CompositeIterable(Iterator cacheIterator, Iterator mapIterator) { + this.cacheIterator = cacheIterator; + this.mapIterator = mapIterator; + } + + @Override + public boolean hasNext() { + if (!cacheIterator.hasNext()) { + while (true) { + if (mapIterator.hasNext()) { + currentObject = mapIterator.next(); + if (!isCacheContains(currentObject)) { + return true; + } + } else { + break; + } + } + return false; + } + return true; + } + + abstract boolean isCacheContains(T object); + + @Override + public T next() { + if (currentObject != null) { + T val = currentObject; + currentObject = null; + return val; + } + if (!hasNext()) { + throw new NoSuchElementException(); + } + return cacheIterator.next(); + } + + @Override + public void remove() { + if (currentObject != null) { + mapIterator.remove(); + currentObject = null; + return; + } + + cacheIterator.remove(); + } + + } + +} diff --git a/redisson/src/main/java/org/redisson/RedissonObject.java b/redisson/src/main/java/org/redisson/RedissonObject.java index 6f756967d..ce077a0d0 100644 --- a/redisson/src/main/java/org/redisson/RedissonObject.java +++ b/redisson/src/main/java/org/redisson/RedissonObject.java @@ -140,5 +140,21 @@ abstract class RedissonObject implements RObject { throw new IllegalArgumentException(e); } } + + protected byte[] encodeMapKey(Object value) { + try { + return codec.getMapKeyEncoder().encode(value); + } catch (IOException e) { + throw new IllegalArgumentException(e); + } + } + + protected byte[] encodeMapValue(Object value) { + try { + return codec.getMapValueEncoder().encode(value); + } catch (IOException e) { + throw new IllegalArgumentException(e); + } + } } diff --git a/redisson/src/main/java/org/redisson/api/LocalCachedMapOptions.java b/redisson/src/main/java/org/redisson/api/LocalCachedMapOptions.java new file mode 100644 index 000000000..62d76a4a1 --- /dev/null +++ b/redisson/src/main/java/org/redisson/api/LocalCachedMapOptions.java @@ -0,0 +1,174 @@ +/** + * Copyright 2016 Nikita Koksharov + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.redisson.api; + +import java.util.concurrent.TimeUnit; + +/** + * RLocalCachedMap options object. Used to specify RLocalCachedMap settings. + * + * @author Nikita Koksharov + * + */ +public class LocalCachedMapOptions { + + public enum EvictionPolicy {NONE, LRU, LFU}; + + private boolean invalidateEntryOnChange; + private EvictionPolicy evictionPolicy; + private int cacheSize; + private long timeToLiveInMillis; + private long maxIdleInMillis; + + private LocalCachedMapOptions() { + } + + protected LocalCachedMapOptions(LocalCachedMapOptions copy) { + this.invalidateEntryOnChange = copy.invalidateEntryOnChange; + this.evictionPolicy = copy.evictionPolicy; + this.cacheSize = copy.cacheSize; + this.timeToLiveInMillis = copy.timeToLiveInMillis; + this.maxIdleInMillis = copy.maxIdleInMillis; + } + + /** + * Creates a new instance of LocalCachedMapOptions with default options. + *

+ * This is equivalent to: + *

+     *     new LocalCachedMapOptions()
+     *      .cacheSize(0).timeToLive(0).maxIdle(0)
+     *      .evictionPolicy(EvictionPolicy.NONE)
+     *      .invalidateEntryOnChange(true);
+     * 
+ */ + public static LocalCachedMapOptions defaults() { + return new LocalCachedMapOptions() + .cacheSize(0).timeToLive(0).maxIdle(0) + .evictionPolicy(EvictionPolicy.NONE) + .invalidateEntryOnChange(true); + } + + public boolean isInvalidateEntryOnChange() { + return invalidateEntryOnChange; + } + + public EvictionPolicy getEvictionPolicy() { + return evictionPolicy; + } + + public int getCacheSize() { + return cacheSize; + } + + public long getTimeToLiveInMillis() { + return timeToLiveInMillis; + } + + public long getMaxIdleInMillis() { + return maxIdleInMillis; + } + + /** + * Sets cache size. If size is 0 then cache is unbounded. + * + * @param cacheSize + * @return + */ + public LocalCachedMapOptions cacheSize(int cacheSize) { + this.cacheSize = cacheSize; + return this; + } + + /** + * Sets entry invalidation behavior. + * + * @param value - if true then invalidation message which removes corresponding entry from cache + * will be sent to all other RLocalCachedMap instances on each entry update/remove operation. + * if false then invalidation message won't be sent + * @return + */ + public LocalCachedMapOptions invalidateEntryOnChange(boolean value) { + this.invalidateEntryOnChange = value; + return this; + } + + /** + * Sets eviction policy. + * + * @param evictionPolicy + *

LRU - uses cache with LRU (least recently used) eviction policy. + *

LFU - uses cache with LFU (least frequently used) eviction policy. + *

NONE - doesn't use eviction policy, but timeToLive and maxIdleTime params are still working. + * @return + */ + public LocalCachedMapOptions evictionPolicy(EvictionPolicy evictionPolicy) { + if (evictionPolicy == null) { + throw new NullPointerException("evictionPolicy can't be null"); + } + this.evictionPolicy = evictionPolicy; + return this; + } + + /** + * Sets time to live in milliseconds for each map entry in cache. + * If value equals to 0 then timeout is not applied + * + * @param timeToLiveInMillis + * @return + */ + public LocalCachedMapOptions timeToLive(long timeToLiveInMillis) { + this.timeToLiveInMillis = timeToLiveInMillis; + return this; + } + + /** + * Sets time to live for each map entry in cache. + * If value equals to 0 then timeout is not applied + * + * @param timeToLive + * @param timeUnit + * @return + */ + public LocalCachedMapOptions timeToLive(long timeToLive, TimeUnit timeUnit) { + return timeToLive(timeUnit.toMillis(timeToLive)); + } + + /** + * Sets max idle time in milliseconds for each map entry in cache. + * If value equals to 0 then timeout is not applied + * + * @param maxIdleInMillis + * @return + */ + public LocalCachedMapOptions maxIdle(long maxIdleInMillis) { + this.maxIdleInMillis = maxIdleInMillis; + return this; + } + + /** + * Sets max idle time for each map entry in cache. + * If value equals to 0 then timeout is not applied + * + * @param maxIdleInMillis + * @return + */ + public LocalCachedMapOptions maxIdle(long maxIdle, TimeUnit timeUnit) { + return timeToLive(timeUnit.toMillis(maxIdle)); + } + + +} diff --git a/redisson/src/main/java/org/redisson/api/RLocalCachedMap.java b/redisson/src/main/java/org/redisson/api/RLocalCachedMap.java new file mode 100644 index 000000000..6cfab311b --- /dev/null +++ b/redisson/src/main/java/org/redisson/api/RLocalCachedMap.java @@ -0,0 +1,58 @@ +/** + * Copyright 2016 Nikita Koksharov + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.redisson.api; + +import java.util.Map; + +/** + * Map object with entry cache support. + *

+ * Each instance maintains local cache to achieve fast read operations. + * Suitable for maps which used mostly for read operations and network roundtrip delays are undesirable. + * + * @author Nikita Koksharov + * + * @param + * @param + */ +public interface RLocalCachedMap extends Map, RExpirable, RLocalCachedMapAsync { + + /** + * Associates the specified value with the specified key. + *

+ * Works faster than RLocalCachedMap.put but not returning + * the previous value associated with key + * + * @param key + * @param value + * @return true if key is a new key in the hash and value was set. + * false if key already exists in the hash and the value was updated. + */ + boolean fastPut(K key, V value); + + /** + * Removes key from map + *

+ * Works faster than RLocalCachedMap.remove but not returning + * the value associated with key + * + * @param key + * @return true if key has been deleted. + * false if key doesn't exist. + */ + boolean fastRemove(Object key); + +} diff --git a/redisson/src/main/java/org/redisson/api/RLocalCachedMapAsync.java b/redisson/src/main/java/org/redisson/api/RLocalCachedMapAsync.java new file mode 100644 index 000000000..a7f51a594 --- /dev/null +++ b/redisson/src/main/java/org/redisson/api/RLocalCachedMapAsync.java @@ -0,0 +1,100 @@ +/** + * Copyright 2016 Nikita Koksharov + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.redisson.api; + +/** + * + * @author Nikita Koksharov + * + * @param + * @param + */ +public interface RLocalCachedMapAsync extends RExpirableAsync { + + /** + * Returns map size + * + * @return + */ + RFuture sizeAsync(); + + /** + * Checks if map contains the specified key + * + * @return true if map contains key. + * false if map doesn't contain key. + */ + RFuture containsKeyAsync(Object key); + + /** + * Checks if map contains the specified value + * + * @return true if map contains value. + * false if map doesn't contain value. + */ + RFuture containsValueAsync(Object value); + + /** + * Returns value associated with key + * + * @param key + * @return + */ + RFuture getAsync(Object key); + + /** + * Associates the specified value with the specified key. + * + * @param key + * @param value + * @return previous value associated with key + */ + RFuture putAsync(K key, V value); + + /** + * Removes key from map. + * + * @param key + * @return removed value associated with key + */ + RFuture removeAsync(K key); + + /** + * Removes key from map + *

+ * Works faster than RLocalCachedMap.remove but not returning + * the value associated with key + * + * @param key + * @return true if key has been deleted. + * false if key doesn't exist. + */ + RFuture fastRemoveAsync(K key); + + /** + * Associates the specified value with the specified key. + *

+ * Works faster than RLocalCachedMap.put but not returning + * the previous value associated with key + * + * @param key + * @param value + * @return true if key is a new key in the hash and value was set. + * false if key already exists in the hash and the value was updated. + */ + RFuture fastPutAsync(K key, V value); + +} diff --git a/redisson/src/main/java/org/redisson/api/RedissonClient.java b/redisson/src/main/java/org/redisson/api/RedissonClient.java index ac53260ff..c8a4e130c 100755 --- a/redisson/src/main/java/org/redisson/api/RedissonClient.java +++ b/redisson/src/main/java/org/redisson/api/RedissonClient.java @@ -207,6 +207,27 @@ public interface RedissonClient { */ RListMultimapCache getListMultimapCache(String name, Codec codec); + /** + * Returns local cached map instance by name. + * Configured by parameters of options-object. + * + * @param name + * @param options + * @return + */ + RLocalCachedMap getLocalCachedMap(String name, LocalCachedMapOptions options); + + /** + * Returns local cached map instance by name + * using provided codec. Configured by parameters of options-object. + * + * @param name + * @param codec + * @param options + * @return + */ + RLocalCachedMap getLocalCachedMap(String name, Codec codec, LocalCachedMapOptions options); + /** * Returns map instance by name. * diff --git a/redisson/src/main/java/org/redisson/misc/Hash.java b/redisson/src/main/java/org/redisson/misc/Hash.java index 03fce6b90..df7da118c 100644 --- a/redisson/src/main/java/org/redisson/misc/Hash.java +++ b/redisson/src/main/java/org/redisson/misc/Hash.java @@ -25,6 +25,19 @@ public class Hash { private Hash() { } + + public static byte[] hash(byte[] objectState) { + long h1 = LongHashFunction.farmUo().hashBytes(objectState); + long h2 = LongHashFunction.xx_r39().hashBytes(objectState); + + ByteBuf buf = Unpooled.buffer((2 * Long.SIZE) / Byte.SIZE).writeLong(h1).writeLong(h2); + try { + return buf.array(); + } finally { + buf.release(); + } + } + public static String hashToBase64(byte[] objectState) { long h1 = LongHashFunction.farmUo().hashBytes(objectState); diff --git a/redisson/src/test/java/org/redisson/RedissonLocalCachedMapTest.java b/redisson/src/test/java/org/redisson/RedissonLocalCachedMapTest.java new file mode 100644 index 000000000..7430cf875 --- /dev/null +++ b/redisson/src/test/java/org/redisson/RedissonLocalCachedMapTest.java @@ -0,0 +1,249 @@ +package org.redisson; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import java.util.concurrent.ExecutionException; + +import org.junit.Assert; +import org.junit.Test; +import org.redisson.RedissonLocalCachedMap.CacheKey; +import org.redisson.RedissonLocalCachedMap.CacheValue; +import org.redisson.api.LocalCachedMapOptions; +import org.redisson.api.LocalCachedMapOptions.EvictionPolicy; +import org.redisson.api.RLocalCachedMap; +import org.redisson.misc.Cache; + +import mockit.Deencapsulation; + +public class RedissonLocalCachedMapTest extends BaseTest { + +// @Test + public void testPerf() { + LocalCachedMapOptions options = LocalCachedMapOptions.defaults().evictionPolicy(EvictionPolicy.LFU).cacheSize(100000).invalidateEntryOnChange(true); + Map map = redisson.getLocalCachedMap("test", options); + +// Map map = redisson.getMap("test"); + + + for (int i = 0; i < 100000; i++) { + map.put("" + i, i); + } + + long s = System.currentTimeMillis(); + for (int i = 0; i < 100; i++) { + for (int j = 0; j < 100000; j++) { + map.get("" + j); + } + } + System.out.println(System.currentTimeMillis() - s); + + } + + @Test + public void testInvalidationOnUpdate() throws InterruptedException { + LocalCachedMapOptions options = LocalCachedMapOptions.defaults().evictionPolicy(EvictionPolicy.LFU).cacheSize(5).invalidateEntryOnChange(true); + RLocalCachedMap map1 = redisson.getLocalCachedMap("test", options); + Cache cache1 = Deencapsulation.getField(map1, "cache"); + + RLocalCachedMap map2 = redisson.getLocalCachedMap("test", options); + Cache cache2 = Deencapsulation.getField(map2, "cache"); + + map1.put("1", 1); + map1.put("2", 2); + + assertThat(map2.get("1")).isEqualTo(1); + assertThat(map2.get("2")).isEqualTo(2); + + assertThat(cache1.size()).isEqualTo(2); + assertThat(cache2.size()).isEqualTo(2); + + map1.put("1", 3); + map2.put("2", 4); + Thread.sleep(50); + + assertThat(cache1.size()).isEqualTo(0); + assertThat(cache2.size()).isEqualTo(0); + } + + @Test + public void testNoInvalidationOnUpdate() throws InterruptedException { + LocalCachedMapOptions options = LocalCachedMapOptions.defaults().evictionPolicy(EvictionPolicy.LFU).cacheSize(5).invalidateEntryOnChange(false); + RLocalCachedMap map1 = redisson.getLocalCachedMap("test", options); + Cache cache1 = Deencapsulation.getField(map1, "cache"); + + RLocalCachedMap map2 = redisson.getLocalCachedMap("test", options); + Cache cache2 = Deencapsulation.getField(map2, "cache"); + + map1.put("1", 1); + map1.put("2", 2); + + assertThat(map2.get("1")).isEqualTo(1); + assertThat(map2.get("2")).isEqualTo(2); + + assertThat(cache1.size()).isEqualTo(2); + assertThat(cache2.size()).isEqualTo(2); + + map1.put("1", 3); + map2.put("2", 4); + Thread.sleep(50); + + assertThat(cache1.size()).isEqualTo(2); + assertThat(cache2.size()).isEqualTo(2); + } + + @Test + public void testNoInvalidationOnRemove() throws InterruptedException { + LocalCachedMapOptions options = LocalCachedMapOptions.defaults().evictionPolicy(EvictionPolicy.LFU).cacheSize(5).invalidateEntryOnChange(false); + RLocalCachedMap map1 = redisson.getLocalCachedMap("test", options); + Cache cache1 = Deencapsulation.getField(map1, "cache"); + + RLocalCachedMap map2 = redisson.getLocalCachedMap("test", options); + Cache cache2 = Deencapsulation.getField(map2, "cache"); + + map1.put("1", 1); + map1.put("2", 2); + + assertThat(map2.get("1")).isEqualTo(1); + assertThat(map2.get("2")).isEqualTo(2); + + assertThat(cache1.size()).isEqualTo(2); + assertThat(cache2.size()).isEqualTo(2); + + map1.remove("1"); + map2.remove("2"); + Thread.sleep(50); + + assertThat(cache1.size()).isEqualTo(1); + assertThat(cache2.size()).isEqualTo(1); + } + + @Test + public void testInvalidationOnRemove() throws InterruptedException { + LocalCachedMapOptions options = LocalCachedMapOptions.defaults().evictionPolicy(EvictionPolicy.LFU).cacheSize(5).invalidateEntryOnChange(true); + RLocalCachedMap map1 = redisson.getLocalCachedMap("test", options); + Cache cache1 = Deencapsulation.getField(map1, "cache"); + + RLocalCachedMap map2 = redisson.getLocalCachedMap("test", options); + Cache cache2 = Deencapsulation.getField(map2, "cache"); + + map1.put("1", 1); + map1.put("2", 2); + + assertThat(map2.get("1")).isEqualTo(1); + assertThat(map2.get("2")).isEqualTo(2); + + assertThat(cache1.size()).isEqualTo(2); + assertThat(cache2.size()).isEqualTo(2); + + map1.remove("1"); + map2.remove("2"); + Thread.sleep(50); + + assertThat(cache1.size()).isEqualTo(0); + assertThat(cache2.size()).isEqualTo(0); + } + + @Test + public void testLFU() { + RLocalCachedMap map = redisson.getLocalCachedMap("test", LocalCachedMapOptions.defaults().evictionPolicy(EvictionPolicy.LFU).cacheSize(5)); + Cache cache = Deencapsulation.getField(map, "cache"); + + map.put("12", 1); + map.put("14", 2); + map.put("15", 3); + map.put("16", 4); + map.put("17", 5); + map.put("18", 6); + + assertThat(cache.size()).isEqualTo(5); + assertThat(map.size()).isEqualTo(6); + assertThat(map.keySet()).containsOnly("12", "14", "15", "16", "17", "18"); + assertThat(map.values()).containsOnly(1, 2, 3, 4, 5, 6); + } + + @Test + public void testLRU() { + RLocalCachedMap map = redisson.getLocalCachedMap("test", LocalCachedMapOptions.defaults().evictionPolicy(EvictionPolicy.LRU).cacheSize(5)); + Cache cache = Deencapsulation.getField(map, "cache"); + + map.put("12", 1); + map.put("14", 2); + map.put("15", 3); + map.put("16", 4); + map.put("17", 5); + map.put("18", 6); + + assertThat(cache.size()).isEqualTo(5); + assertThat(map.size()).isEqualTo(6); + assertThat(map.keySet()).containsOnly("12", "14", "15", "16", "17", "18"); + assertThat(map.values()).containsOnly(1, 2, 3, 4, 5, 6); + } + + @Test + public void testSize() { + RLocalCachedMap map = redisson.getLocalCachedMap("test", LocalCachedMapOptions.defaults()); + Cache cache = Deencapsulation.getField(map, "cache"); + + map.put("12", 1); + map.put("14", 2); + map.put("15", 3); + + assertThat(cache.size()).isEqualTo(3); + assertThat(map.size()).isEqualTo(3); + } + + + @Test + public void testPut() { + RLocalCachedMap map = redisson.getLocalCachedMap("test", LocalCachedMapOptions.defaults()); + + map.put("12", 1); + map.put("14", 2); + map.put("15", 3); + + Deencapsulation.setField(map, "map", null); + + assertThat(map.get("12")).isEqualTo(1); + assertThat(map.get("14")).isEqualTo(2); + assertThat(map.get("15")).isEqualTo(3); + } + + @Test + public void testRemove() { + RLocalCachedMap map = redisson.getLocalCachedMap("test", LocalCachedMapOptions.defaults()); + Cache cache = Deencapsulation.getField(map, "cache"); + map.put("12", 1); + + assertThat(cache.size()).isEqualTo(1); + + assertThat(map.remove("12")).isEqualTo(1); + + assertThat(cache.size()).isEqualTo(0); + + assertThat(map.remove("14")).isNull(); + } + + @Test + public void testFastRemoveAsync() throws InterruptedException, ExecutionException { + RLocalCachedMap map = redisson.getLocalCachedMap("test", LocalCachedMapOptions.defaults()); + map.put(1, 3); + map.put(7, 8); + + assertThat(map.fastRemoveAsync(1).get()).isTrue(); + assertThat(map.fastRemoveAsync(2).get()).isFalse(); + assertThat(map.size()).isEqualTo(1); + } + + @Test + public void testFastPut() { + RLocalCachedMap map = redisson.getLocalCachedMap("test", LocalCachedMapOptions.defaults()); + Assert.assertTrue(map.fastPut("1", 2)); + assertThat(map.get("1")).isEqualTo(2); + Assert.assertFalse(map.fastPut("1", 3)); + assertThat(map.get("1")).isEqualTo(3); + Assert.assertEquals(1, map.size()); + } + + +}