From 23701b2bc9f2f8b557125d9ba7a78599b5ca6b87 Mon Sep 17 00:00:00 2001 From: Jacob Barrett Date: Sun, 15 Mar 2020 18:16:45 -0700 Subject: [PATCH] GEODE-7869: Cleanup warnings in geode-redis --- geode-redis/build.gradle | 1 + .../geode/redis/RedisDistDUnitTest.java | 37 +++--- .../redis/ConcurrentStartIntegrationTest.java | 31 ++--- .../apache/geode/redis/GeodeRedisServer.java | 120 +++++++++--------- .../org/apache/geode/redis/internal/Pair.java | 30 ----- .../geode/redis/internal/PubSubImpl.java | 11 +- .../geode/redis/internal/RegionProvider.java | 116 ++++++++--------- .../internal/executor/RenameExecutor.java | 7 +- .../executor/list/LIndexExecutor.java | 6 +- .../internal/executor/list/LLenExecutor.java | 8 +- .../executor/list/LRangeExecutor.java | 13 +- .../internal/executor/list/LRemExecutor.java | 11 +- .../internal/executor/list/LSetExecutor.java | 13 +- .../internal/executor/list/LTrimExecutor.java | 15 ++- .../internal/executor/list/ListExecutor.java | 20 +-- .../internal/executor/list/PopExecutor.java | 8 +- .../internal/executor/list/PushExecutor.java | 4 +- .../internal/executor/list/PushXExecutor.java | 4 +- .../sortedset/GeoSortedSetExecutor.java | 1 + .../sortedset/ZRangeByLexExecutor.java | 9 +- .../sortedset/ZRangeByScoreExecutor.java | 14 +- .../ExecutionHandlerContextJUnitTest.java | 4 +- .../string/StringSetExecutorJUnitTest.java | 4 +- 23 files changed, 229 insertions(+), 258 deletions(-) delete mode 100644 geode-redis/src/main/java/org/apache/geode/redis/internal/Pair.java diff --git a/geode-redis/build.gradle b/geode-redis/build.gradle index f4bccc2a0a09..b4ffdd385401 100644 --- a/geode-redis/build.gradle +++ b/geode-redis/build.gradle @@ -18,6 +18,7 @@ apply from: "${rootDir}/${scriptDir}/standard-subproject-configuration.gradle" apply from: "${project.projectDir}/../gradle/publish-java.gradle" +apply from: "${rootDir}/${scriptDir}/warnings.gradle" dependencies { compile(platform(project(':boms:geode-all-bom'))) diff --git a/geode-redis/src/distributedTest/java/org/apache/geode/redis/RedisDistDUnitTest.java b/geode-redis/src/distributedTest/java/org/apache/geode/redis/RedisDistDUnitTest.java index 6511be24d7f0..3b01de1e84a8 100644 --- a/geode-redis/src/distributedTest/java/org/apache/geode/redis/RedisDistDUnitTest.java +++ b/geode-redis/src/distributedTest/java/org/apache/geode/redis/RedisDistDUnitTest.java @@ -30,7 +30,7 @@ import org.apache.geode.test.awaitility.GeodeAwaitility; import org.apache.geode.test.dunit.AsyncInvocation; import org.apache.geode.test.dunit.IgnoredException; -import org.apache.geode.test.dunit.SerializableCallable; +import org.apache.geode.test.dunit.SerializableRunnable; import org.apache.geode.test.dunit.VM; import org.apache.geode.test.dunit.rules.ClusterStartupRule; import org.apache.geode.test.dunit.rules.MemberVM; @@ -45,9 +45,6 @@ public class RedisDistDUnitTest implements Serializable { private static String LOCALHOST = "localhost"; public static final String TEST_KEY = "key"; - private static MemberVM locator; - private static MemberVM server1; - private static MemberVM server2; private static VM client1; private static VM client2; @@ -57,7 +54,7 @@ public class RedisDistDUnitTest implements Serializable { private static final int JEDIS_TIMEOUT = Math.toIntExact(GeodeAwaitility.getTimeout().getValueInMS()); - private abstract static class ClientTestBase extends SerializableCallable { + private abstract static class ClientTestBase extends SerializableRunnable { int port; protected ClientTestBase(int port) { @@ -71,15 +68,15 @@ public static void setup() { server1Port = ports[0]; server2Port = ports[1]; - locator = cluster.startLocatorVM(0); + MemberVM locator = cluster.startLocatorVM(0); Properties redisProps = new Properties(); redisProps.setProperty("redis-bind-address", LOCALHOST); redisProps.setProperty("redis-port", Integer.toString(ports[0])); - server1 = cluster.startServerVM(1, redisProps, locator.getPort()); + cluster.startServerVM(1, redisProps, locator.getPort()); redisProps.setProperty("redis-port", Integer.toString(ports[1])); - server2 = cluster.startServerVM(2, redisProps, locator.getPort()); + cluster.startServerVM(2, redisProps, locator.getPort()); client1 = cluster.getVM(3); client2 = cluster.getVM(4); @@ -97,7 +94,7 @@ protected ConcListOps(int port) { } @Override - public Object call() throws Exception { + public void run() { Jedis jedis = new Jedis(LOCALHOST, port, JEDIS_TIMEOUT); Random r = new Random(); for (int i = 0; i < pushes; i++) { @@ -107,13 +104,12 @@ public Object call() throws Exception { jedis.rpush(TEST_KEY, randString()); } } - return null; } - }; + } - AsyncInvocation i = client1.invokeAsync(new ConcListOps(server1Port)); + AsyncInvocation i = client1.invokeAsync(new ConcListOps(server1Port)); client2.invoke(new ConcListOps(server2Port)); - i.get(); + i.await(); long expected = 2 * pushes; long result1 = jedis1.llen(TEST_KEY); long result2 = jedis2.llen(TEST_KEY); @@ -137,7 +133,7 @@ protected ConcCreateDestroy(int port) { } @Override - public Object call() throws Exception { + public void run() { Jedis jedis = new Jedis(LOCALHOST, port, JEDIS_TIMEOUT); Random r = new Random(); for (int i = 0; i < ops; i++) { @@ -168,19 +164,19 @@ public Object call() throws Exception { } } } - return null; } } // Expect to run with no exception - AsyncInvocation i = client1.invokeAsync(new ConcCreateDestroy(server1Port)); + AsyncInvocation i = client1.invokeAsync(new ConcCreateDestroy(server1Port)); client2.invoke(new ConcCreateDestroy(server2Port)); - i.get(); + i.await(); } /** * Just make sure there are no unexpected server crashes */ + @Test public void testConcOps() throws Exception { final int ops = 100; @@ -196,7 +192,7 @@ protected ConcOps(int port) { } @Override - public Object call() throws Exception { + public void run() { Jedis jedis = new Jedis(LOCALHOST, port, JEDIS_TIMEOUT); Random r = new Random(); for (int i = 0; i < ops; i++) { @@ -222,14 +218,13 @@ public Object call() throws Exception { jedis.sunionstore("dst", sKey, "afds"); } } - return null; } } // Expect to run with no exception - AsyncInvocation i = client1.invokeAsync(new ConcOps(server1Port)); + AsyncInvocation i = client1.invokeAsync(new ConcOps(server1Port)); client2.invoke(new ConcOps(server2Port)); - i.getResult(); + i.await(); } private String randString() { diff --git a/geode-redis/src/integrationTest/java/org/apache/geode/redis/ConcurrentStartIntegrationTest.java b/geode-redis/src/integrationTest/java/org/apache/geode/redis/ConcurrentStartIntegrationTest.java index 9ab3ae87e57a..024a751122ea 100644 --- a/geode-redis/src/integrationTest/java/org/apache/geode/redis/ConcurrentStartIntegrationTest.java +++ b/geode-redis/src/integrationTest/java/org/apache/geode/redis/ConcurrentStartIntegrationTest.java @@ -49,16 +49,16 @@ public void setUp() { @After public void tearDown() { - if (this.cache != null) { - this.cache.close(); - this.cache = null; + if (cache != null) { + cache.close(); + cache = null; } } @Test public void testCachelessStart() throws InterruptedException { runNServers(numServers); - GemFireCacheImpl.getInstance().close(); + getCache().close(); } @Test @@ -66,7 +66,7 @@ public void testCachefulStart() throws InterruptedException { CacheFactory cf = new CacheFactory(); cf.set(MCAST_PORT, "0"); cf.set(LOCATORS, ""); - this.cache = cf.create(); + cache = cf.create(); runNServers(numServers); } @@ -76,14 +76,10 @@ private void runNServers(int n) throws InterruptedException { final Thread[] threads = new Thread[n]; for (int i = 0; i < n; i++) { final int j = i; - Runnable r = new Runnable() { - - @Override - public void run() { - GeodeRedisServer s = new GeodeRedisServer(ports[j]); - s.start(); - s.shutdown(); - } + Runnable r = () -> { + GeodeRedisServer s = new GeodeRedisServer(ports[j]); + s.start(); + s.shutdown(); }; Thread t = new Thread(r); @@ -93,7 +89,12 @@ public void run() { } for (Thread t : threads) t.join(); - this.cache = GemFireCacheImpl.getInstance(); - assertFalse(this.cache.isClosed()); + cache = getCache(); + assertFalse(cache.isClosed()); + } + + @SuppressWarnings("deprecation") + private GemFireCacheImpl getCache() { + return GemFireCacheImpl.getInstance(); } } diff --git a/geode-redis/src/main/java/org/apache/geode/redis/GeodeRedisServer.java b/geode-redis/src/main/java/org/apache/geode/redis/GeodeRedisServer.java index d7a5939335e6..4a7d3a324ace 100644 --- a/geode-redis/src/main/java/org/apache/geode/redis/GeodeRedisServer.java +++ b/geode-redis/src/main/java/org/apache/geode/redis/GeodeRedisServer.java @@ -54,16 +54,13 @@ import io.netty.channel.EventLoopGroup; import io.netty.channel.ServerChannel; import io.netty.channel.nio.NioEventLoopGroup; -import io.netty.channel.oio.OioEventLoopGroup; import io.netty.channel.socket.SocketChannel; import io.netty.channel.socket.nio.NioServerSocketChannel; -import io.netty.channel.socket.oio.OioServerSocketChannel; import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslContextBuilder; import io.netty.util.concurrent.Future; import org.apache.geode.GemFireConfigException; -import org.apache.geode.LogWriter; import org.apache.geode.annotations.Experimental; import org.apache.geode.annotations.internal.MakeNotStatic; import org.apache.geode.cache.Cache; @@ -224,7 +221,8 @@ public class GeodeRedisServer { /** * Gem logwriter */ - private LogWriter logger; + @SuppressWarnings("deprecation") + private org.apache.geode.LogWriter logger; private RegionProvider regionCache; @@ -372,15 +370,15 @@ public GeodeRedisServer(String bindAddress, int port) { * @param logLevel The logging level to be used by GemFire */ public GeodeRedisServer(String bindAddress, int port, String logLevel) { - this.serverPort = port <= 0 ? DEFAULT_REDIS_SERVER_PORT : port; + serverPort = port <= 0 ? DEFAULT_REDIS_SERVER_PORT : port; this.bindAddress = bindAddress; this.logLevel = logLevel; - this.numWorkerThreads = setNumWorkerThreads(); - this.singleThreadPerConnection = this.numWorkerThreads == 0; - this.numSelectorThreads = 1; - this.metaListener = new MetaCacheListener(); - this.expirationFutures = new ConcurrentHashMap>(); - this.expirationExecutor = + numWorkerThreads = setNumWorkerThreads(); + singleThreadPerConnection = numWorkerThreads == 0; + numSelectorThreads = 1; + metaListener = new MetaCacheListener(); + expirationFutures = new ConcurrentHashMap<>(); + expirationExecutor = Executors.newScheduledThreadPool(numExpirationThreads, new ThreadFactory() { private final AtomicInteger counter = new AtomicInteger(); @@ -393,9 +391,9 @@ public Thread newThread(Runnable r) { } }); - this.DEFAULT_REGION_TYPE = setRegionType(); - this.shutdown = false; - this.started = false; + DEFAULT_REGION_TYPE = setRegionType(); + shutdown = false; + started = false; } /** @@ -404,8 +402,8 @@ public Thread newThread(Runnable r) { * @return The InetAddress to bind to */ private InetAddress getBindAddress() throws UnknownHostException { - return this.bindAddress == null || this.bindAddress.isEmpty() ? LocalHostUtil.getLocalHost() - : InetAddress.getByName(this.bindAddress); + return bindAddress == null || bindAddress.isEmpty() ? LocalHostUtil.getLocalHost() + : InetAddress.getByName(bindAddress); } /** @@ -417,9 +415,7 @@ public synchronized void start() { startGemFire(); initializeRedis(); startRedisServer(); - } catch (IOException e) { - throw new RuntimeException("Could not start Server", e); - } catch (InterruptedException e) { + } catch (IOException | InterruptedException e) { throw new RuntimeException("Could not start Server", e); } started = true; @@ -431,6 +427,7 @@ public synchronized void start() { * {@link Region} to be protected. Also, every {@code GeodeRedisServer} will check for entries * already in the meta data Region. */ + @SuppressWarnings("deprecation") private void startGemFire() { Cache cache = GemFireCacheImpl.getInstance(); if (cache == null) { @@ -446,11 +443,11 @@ private void startGemFire() { } } this.cache = cache; - this.logger = cache.getLogger(); + logger = cache.getLogger(); } private void initializeRedis() { - synchronized (this.cache) { + synchronized (cache) { Region stringsRegion; Region hLLRegion; @@ -461,24 +458,24 @@ private void initializeRedis() { if ((stringsRegion = cache.getRegion(STRING_REGION)) == null) { RegionFactory regionFactory = - gemFireCache.createRegionFactory(this.DEFAULT_REGION_TYPE); + gemFireCache.createRegionFactory(DEFAULT_REGION_TYPE); stringsRegion = regionFactory.create(STRING_REGION); } if ((hLLRegion = cache.getRegion(HLL_REGION)) == null) { RegionFactory regionFactory = - gemFireCache.createRegionFactory(this.DEFAULT_REGION_TYPE); + gemFireCache.createRegionFactory(DEFAULT_REGION_TYPE); hLLRegion = regionFactory.create(HLL_REGION); } if ((redisHash = cache.getRegion(HASH_REGION)) == null) { RegionFactory> regionFactory = - gemFireCache.createRegionFactory(this.DEFAULT_REGION_TYPE); + gemFireCache.createRegionFactory(DEFAULT_REGION_TYPE); redisHash = regionFactory.create(HASH_REGION); } if ((redisSet = cache.getRegion(SET_REGION)) == null) { RegionFactory> regionFactory = - gemFireCache.createRegionFactory(this.DEFAULT_REGION_TYPE); + gemFireCache.createRegionFactory(DEFAULT_REGION_TYPE); redisSet = regionFactory.create(SET_REGION); } @@ -491,11 +488,11 @@ private void initializeRedis() { redisMetaData = redisMetaDataFactory.create(REDIS_META_DATA_REGION); } - this.keyRegistrar = new KeyRegistrar(redisMetaData); - this.hashLockService = new RedisLockService(); - this.pubSub = new PubSubImpl(new Subscriptions()); - this.regionCache = new RegionProvider(stringsRegion, hLLRegion, this.keyRegistrar, - expirationFutures, expirationExecutor, this.DEFAULT_REGION_TYPE, redisHash, redisSet); + keyRegistrar = new KeyRegistrar(redisMetaData); + hashLockService = new RedisLockService(); + pubSub = new PubSubImpl(new Subscriptions()); + regionCache = new RegionProvider(stringsRegion, hLLRegion, keyRegistrar, + expirationFutures, expirationExecutor, DEFAULT_REGION_TYPE, redisHash, redisSet); redisMetaData.put(REDIS_META_DATA_REGION, RedisDataType.REDIS_PROTECTED); redisMetaData.put(HLL_REGION, RedisDataType.REDIS_PROTECTED); redisMetaData.put(STRING_REGION, RedisDataType.REDIS_PROTECTED); @@ -514,7 +511,7 @@ private void registerLockServiceMBean() { Set names = platformMBeanServer.queryNames(mbeanON, null); if (names.isEmpty()) { - platformMBeanServer.registerMBean(this.hashLockService, mbeanON); + platformMBeanServer.registerMBean(hashLockService, mbeanON); logger.info("Registered RedisLockServiceMBean on " + mbeanON); } } catch (InstanceAlreadyExistsException | MBeanRegistrationException @@ -532,7 +529,7 @@ private void checkForRegions() { if (newRegion == null && type != RedisDataType.REDIS_STRING && type != RedisDataType.REDIS_HLL && type != RedisDataType.REDIS_PROTECTED) { try { - this.regionCache + regionCache .createRemoteRegionReferenceLocally(Coder.stringToByteArrayWrapper(regionName), type); } catch (Exception e) { if (logger.errorEnabled()) { @@ -573,14 +570,13 @@ public Thread newThread(Runnable r) { bossGroup = null; workerGroup = null; - Class socketClass = null; + Class socketClass; if (singleThreadPerConnection) { - bossGroup = new OioEventLoopGroup(Integer.MAX_VALUE, selectorThreadFactory); - workerGroup = new OioEventLoopGroup(Integer.MAX_VALUE, workerThreadFactory); - socketClass = OioServerSocketChannel.class; + socketClass = + startRedisServiceSingleThreadPerConnection(selectorThreadFactory, workerThreadFactory); } else { - bossGroup = new NioEventLoopGroup(this.numSelectorThreads, selectorThreadFactory); - workerGroup = new NioEventLoopGroup(this.numWorkerThreads, workerThreadFactory); + bossGroup = new NioEventLoopGroup(numSelectorThreads, selectorThreadFactory); + workerGroup = new NioEventLoopGroup(numWorkerThreads, workerThreadFactory); socketClass = NioServerSocketChannel.class; } InternalDistributedSystem system = (InternalDistributedSystem) cache.getDistributedSystem(); @@ -591,7 +587,7 @@ public Thread newThread(Runnable r) { b.group(bossGroup, workerGroup).channel(socketClass) .childHandler(new ChannelInitializer() { @Override - public void initChannel(SocketChannel ch) throws Exception { + public void initChannel(SocketChannel ch) { if (logger.fineEnabled()) { logger.fine("GeodeRedisServer-Connection established with " + ch.remoteAddress()); } @@ -609,17 +605,27 @@ public void initChannel(SocketChannel ch) throws Exception { // Bind and start to accept incoming connections. ChannelFuture f = b.bind(new InetSocketAddress(getBindAddress(), serverPort)).sync(); - if (this.logger.infoEnabled()) { + if (logger.infoEnabled()) { String logMessage = "GeodeRedisServer started {" + getBindAddress() + ":" + serverPort - + "}, Selector threads: " + this.numSelectorThreads; - if (this.singleThreadPerConnection) { + + "}, Selector threads: " + numSelectorThreads; + if (singleThreadPerConnection) { logMessage += ", One worker thread per connection"; } else { - logMessage += ", Worker threads: " + this.numWorkerThreads; + logMessage += ", Worker threads: " + numWorkerThreads; } - this.logger.info(logMessage); + logger.info(logMessage); } - this.serverChannel = f.channel(); + serverChannel = f.channel(); + } + + @SuppressWarnings("deprecation") + private Class startRedisServiceSingleThreadPerConnection( + ThreadFactory selectorThreadFactory, ThreadFactory workerThreadFactory) { + bossGroup = + new io.netty.channel.oio.OioEventLoopGroup(Integer.MAX_VALUE, selectorThreadFactory); + workerGroup = + new io.netty.channel.oio.OioEventLoopGroup(Integer.MAX_VALUE, workerThreadFactory); + return io.netty.channel.socket.oio.OioServerSocketChannel.class; } private void addSSLIfEnabled(SocketChannel ch, ChannelPipeline p) { @@ -662,12 +668,12 @@ private void addSSLIfEnabled(SocketChannel ch, ChannelPipeline p) { */ private void afterKeyCreate(EntryEvent event) { if (event.isOriginRemote()) { - final String key = (String) event.getKey(); + final String key = event.getKey(); final RedisDataType value = event.getNewValue(); if (value != RedisDataType.REDIS_STRING && value != RedisDataType.REDIS_HLL && value != RedisDataType.REDIS_PROTECTED) { try { - this.regionCache.createRemoteRegionReferenceLocally(Coder.stringToByteArrayWrapper(key), + regionCache.createRemoteRegionReferenceLocally(Coder.stringToByteArrayWrapper(key), value); } catch (RegionDestroyedException ignore) { // Region already destroyed, ignore } @@ -681,14 +687,14 @@ private void afterKeyCreate(EntryEvent event) { */ private void afterKeyDestroy(EntryEvent event) { if (event.isOriginRemote()) { - final String key = (String) event.getKey(); + final String key = event.getKey(); final RedisDataType value = event.getOldValue(); if (value != null && value != RedisDataType.REDIS_STRING && value != RedisDataType.REDIS_HLL && value != RedisDataType.REDIS_PROTECTED) { ByteArrayWrapper kW = Coder.stringToByteArrayWrapper(key); - Region r = this.regionCache.getRegion(kW); + Region r = regionCache.getRegion(kW); if (r != null) { - this.regionCache.removeRegionReferenceLocally(kW, value); + regionCache.removeRegionReferenceLocally(kW, value); } } } @@ -725,21 +731,21 @@ public synchronized void shutdown() { if (logger.infoEnabled()) { logger.info("GeodeRedisServer shutting down"); } - ChannelFuture closeFuture = this.serverChannel.closeFuture(); + ChannelFuture closeFuture = serverChannel.closeFuture(); Future c = workerGroup.shutdownGracefully(); Future c2 = bossGroup.shutdownGracefully(); - this.serverChannel.close(); + serverChannel.close(); c.syncUninterruptibly(); c2.syncUninterruptibly(); - this.regionCache.close(); + regionCache.close(); if (mainThread != null) { mainThread.interrupt(); } - for (ScheduledFuture f : this.expirationFutures.values()) { + for (ScheduledFuture f : expirationFutures.values()) { f.cancel(true); } - this.expirationFutures.clear(); - this.expirationExecutor.shutdownNow(); + expirationFutures.clear(); + expirationExecutor.shutdownNow(); closeFuture.syncUninterruptibly(); shutdown = true; } @@ -776,7 +782,7 @@ public static void main(String[] args) { Thread.sleep(Long.MAX_VALUE); } catch (InterruptedException e1) { break; - } catch (Exception e) { + } catch (Exception ignored) { } } } diff --git a/geode-redis/src/main/java/org/apache/geode/redis/internal/Pair.java b/geode-redis/src/main/java/org/apache/geode/redis/internal/Pair.java deleted file mode 100644 index aaff10eb757d..000000000000 --- a/geode-redis/src/main/java/org/apache/geode/redis/internal/Pair.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license - * agreements. See the NOTICE file distributed with this work for additional information regarding - * copyright ownership. The ASF licenses this file to You 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.apache.geode.redis.internal; - -public class Pair { - T fst; - U snd; - - Pair(Object fst, Object snd) { - this.fst = (T) fst; - this.snd = (U) snd; - } - - public static Pair of(Object fst, Object snd) { - return new Pair(fst, snd); - } -} diff --git a/geode-redis/src/main/java/org/apache/geode/redis/internal/PubSubImpl.java b/geode-redis/src/main/java/org/apache/geode/redis/internal/PubSubImpl.java index 3f0d8da246da..c5e52e55a0ca 100644 --- a/geode-redis/src/main/java/org/apache/geode/redis/internal/PubSubImpl.java +++ b/geode-redis/src/main/java/org/apache/geode/redis/internal/PubSubImpl.java @@ -45,6 +45,7 @@ public PubSubImpl(Subscriptions subscriptions) { @Override public long publish(String channel, byte[] message) { + @SuppressWarnings("unchecked") ResultCollector> subscriberCountCollector = FunctionService .onMembers() .setArguments(new Object[] {channel, message}) @@ -95,20 +96,20 @@ public void execute(FunctionContext context) { @Override public long unsubscribe(String channel, Client client) { - this.subscriptions.remove(channel, client); - return this.subscriptions.findSubscriptions(client).size(); + subscriptions.remove(channel, client); + return subscriptions.findSubscriptions(client).size(); } @Override public long punsubscribe(GlobPattern pattern, Client client) { - this.subscriptions.remove(pattern, client); - return this.subscriptions.findSubscriptions(client).size(); + subscriptions.remove(pattern, client); + return subscriptions.findSubscriptions(client).size(); } @VisibleForTesting long publishMessageToSubscribers(String channel, byte[] message) { - Map> results = this.subscriptions + Map> results = subscriptions .findSubscriptions(channel) .stream() .map(subscription -> subscription.publishMessage(channel, message)) diff --git a/geode-redis/src/main/java/org/apache/geode/redis/internal/RegionProvider.java b/geode-redis/src/main/java/org/apache/geode/redis/internal/RegionProvider.java index f68b296af2a3..c8d236f1df81 100644 --- a/geode-redis/src/main/java/org/apache/geode/redis/internal/RegionProvider.java +++ b/geode-redis/src/main/java/org/apache/geode/redis/internal/RegionProvider.java @@ -57,7 +57,7 @@ * synchronized, which is done away with and abstracted by this class. */ public class RegionProvider implements Closeable { - private final ConcurrentHashMap> regions; + private final ConcurrentHashMap> regions; /** * This is the Redis meta data {@link Region} that holds the {@link RedisDataType} information for @@ -92,6 +92,7 @@ public class RegionProvider implements Closeable { private static final CreateRegionCommand createRegionCmd = new CreateRegionCommand(); private final ConcurrentHashMap locks; + @SuppressWarnings("deprecation") public RegionProvider(Region stringsRegion, Region hLLRegion, KeyRegistrar redisMetaRegion, @@ -116,16 +117,16 @@ public RegionProvider(Region stringsRegion, } this.hashRegion = hashRegion; this.setRegion = setRegion; - this.regions = new ConcurrentHashMap<>(); + regions = new ConcurrentHashMap<>(); this.stringsRegion = stringsRegion; this.hLLRegion = hLLRegion; - this.keyRegistrar = redisMetaRegion; + keyRegistrar = redisMetaRegion; this.cache = cache; - this.queryService = cache.getQueryService(); + queryService = cache.getQueryService(); this.expirationsMap = expirationsMap; this.expirationExecutor = expirationExecutor; - this.defaultRegionType = defaultShortcut; - this.locks = new ConcurrentHashMap<>(); + defaultRegionType = defaultShortcut; + locks = new ConcurrentHashMap<>(); } public Region getRegion(ByteArrayWrapper key) { @@ -133,11 +134,11 @@ public RegionProvider(Region stringsRegion, return null; } - return this.regions.get(key); + return regions.get(key); } - public Region getRegionForType(RedisDataType redisDataType) { + public Region getRegionForType(RedisDataType redisDataType) { if (redisDataType == null) { return null; } @@ -166,7 +167,7 @@ public RegionProvider(Region stringsRegion, } public void removeRegionReferenceLocally(ByteArrayWrapper key, RedisDataType type) { - Lock lock = this.locks.get(key.toString()); + Lock lock = locks.get(key.toString()); boolean locked = false; try { if (lock != null) { @@ -186,7 +187,7 @@ public void removeRegionReferenceLocally(ByteArrayWrapper key, RedisDataType typ } public boolean removeKey(ByteArrayWrapper key) { - RedisDataType type = this.keyRegistrar.getType(key); + RedisDataType type = keyRegistrar.getType(key); return removeKey(key, type); } @@ -198,26 +199,26 @@ public boolean removeKey(ByteArrayWrapper key, RedisDataType type, boolean cance if (type == RedisDataType.REDIS_PROTECTED) { return false; } - Lock lock = this.locks.get(key.toString()); + Lock lock = locks.get(key.toString()); try { if (lock != null) {// Strings/hlls will not have locks lock.lock(); } - this.keyRegistrar.unregister(key); + keyRegistrar.unregister(key); try { if (type == RedisDataType.REDIS_STRING) { - return this.stringsRegion.remove(key) != null; + return stringsRegion.remove(key) != null; } else if (type == RedisDataType.REDIS_HLL) { - return this.hLLRegion.remove(key) != null; + return hLLRegion.remove(key) != null; } else if (type == RedisDataType.REDIS_LIST) { - return this.destroyRegion(key, type); + return destroyRegion(key, type); } else if (type == RedisDataType.REDIS_SET) { // remove the set - this.setRegion.remove(key); + setRegion.remove(key); return true; } else if (type == RedisDataType.REDIS_HASH) { // Check hash - this.hashRegion.remove(key); + hashRegion.remove(key); return true; } else { return false; @@ -231,7 +232,7 @@ public boolean removeKey(ByteArrayWrapper key, RedisDataType type, boolean cance removeKeyExpiration(key); } if (lock != null) { - this.locks.remove(key.toString()); + locks.remove(key.toString()); } } } finally { @@ -250,16 +251,16 @@ public void createRemoteRegionReferenceLocally(ByteArrayWrapper key, RedisDataTy if (type == null || type == RedisDataType.REDIS_STRING || type == RedisDataType.REDIS_HLL) { return; } - Region r = this.regions.get(key); + Region r = regions.get(key); if (r != null) { return; } - if (!this.regions.containsKey(key)) { + if (!regions.containsKey(key)) { String stringKey = key.toString(); - Lock lock = this.locks.get(stringKey); + Lock lock = locks.get(stringKey); if (lock == null) { - this.locks.putIfAbsent(stringKey, new ReentrantLock()); - lock = this.locks.get(stringKey); + locks.putIfAbsent(stringKey, new ReentrantLock()); + lock = locks.get(stringKey); } boolean locked = false; try { @@ -284,7 +285,7 @@ public void createRemoteRegionReferenceLocally(ByteArrayWrapper key, RedisDataTy // ignore } } - this.regions.put(key, r); + regions.put(key, r); } } finally { if (locked) { @@ -298,18 +299,18 @@ public void createRemoteRegionReferenceLocally(ByteArrayWrapper key, RedisDataTy ExecutionHandlerContext context, boolean addToMeta) { String regionName = key.toString(); - this.keyRegistrar.validate(key, type); - Region r = this.regions.get(key); + keyRegistrar.validate(key, type); + Region r = regions.get(key); if (r != null && r.isDestroyed()) { removeKey(key, type); r = null; } if (r == null) { String stringKey = key.toString(); - Lock lock = this.locks.get(stringKey); + Lock lock = locks.get(stringKey); if (lock == null) { - this.locks.putIfAbsent(stringKey, new ReentrantLock()); - lock = this.locks.get(stringKey); + locks.putIfAbsent(stringKey, new ReentrantLock()); + lock = locks.get(stringKey); } try { @@ -325,7 +326,7 @@ public void createRemoteRegionReferenceLocally(ByteArrayWrapper key, RedisDataTy txm = cache.getCacheTransactionManager(); transactionId = txm.suspend(); } - Exception concurrentCreateDestroyException = null; + Exception concurrentCreateDestroyException; do { concurrentCreateDestroyException = null; @@ -347,9 +348,9 @@ public void createRemoteRegionReferenceLocally(ByteArrayWrapper key, RedisDataTy } } } while (concurrentCreateDestroyException != null); - this.regions.put(key, r); + regions.put(key, r); if (addToMeta) { - this.keyRegistrar.register(key, type); + keyRegistrar.register(key, type); } } finally { if (hasTransaction) { @@ -372,7 +373,7 @@ public void createRemoteRegionReferenceLocally(ByteArrayWrapper key, RedisDataTy * @return Flag if destroyed */ private boolean destroyRegion(ByteArrayWrapper key, RedisDataType type) { - Region r = this.regions.get(key); + Region r = regions.get(key); if (r != null) { try { r.destroyRegion(); @@ -392,8 +393,8 @@ private boolean destroyRegion(ByteArrayWrapper key, RedisDataType type) { * @param type Type of key to remove all state */ private void removeRegionState(ByteArrayWrapper key, RedisDataType type) { - this.preparedQueries.remove(key); - this.regions.remove(key); + preparedQueries.remove(key); + regions.remove(key); } private void doInitializeSortedSet(ByteArrayWrapper key, Region r) @@ -409,24 +410,23 @@ private void doInitializeSortedSet(ByteArrayWrapper key, Region r) HashMap, Query> queryList = new HashMap<>(); for (SortedSetQuery lq : SortedSetQuery.values()) { String queryString = lq.getQueryString(fullpath); - Query query = this.queryService.newQuery(queryString); + Query query = queryService.newQuery(queryString); queryList.put(lq, query); } - this.preparedQueries.put(key, queryList); + preparedQueries.put(key, queryList); } - @SuppressWarnings("rawtypes") - private void doInitializeList(ByteArrayWrapper key, Region r) { + private void doInitializeList(ByteArrayWrapper key, Region r) { r.put("head", 0); r.put("tail", 0); String fullpath = r.getFullPath(); HashMap, Query> queryList = new HashMap<>(); for (ListQuery lq : ListQuery.values()) { String queryString = lq.getQueryString(fullpath); - Query query = this.queryService.newQuery(queryString); + Query query = queryService.newQuery(queryString); queryList.put(lq, query); } - this.preparedQueries.put(key, queryList); + preparedQueries.put(key, queryList); } /** @@ -436,8 +436,8 @@ private void doInitializeList(ByteArrayWrapper key, Region r) { * @param regionPath Name of Region to create * @return Region Region created globally */ - private Region createRegionGlobally(String regionPath) { - Region r = null; + private Region createRegionGlobally(String regionPath) { + Region r; r = cache.getRegion(regionPath); if (r != null) { return r; @@ -464,15 +464,15 @@ private void doInitializeList(ByteArrayWrapper key, Region r) { } public Query getQuery(ByteArrayWrapper key, Enum query) { - return this.preparedQueries.get(key).get(query); + return preparedQueries.get(key).get(query); } public boolean regionExists(ByteArrayWrapper key) { - return this.regions.containsKey(key); + return regions.containsKey(key); } public Region getStringsRegion() { - return this.stringsRegion; + return stringsRegion; } /** @@ -490,7 +490,7 @@ public Region> getSetRegion() { } public Region gethLLRegion() { - return this.hLLRegion; + return hLLRegion; } /** @@ -504,13 +504,13 @@ public Region gethLLRegion() { * @return True is expiration set, false otherwise */ public boolean setExpiration(ByteArrayWrapper key, long delay) { - RedisDataType type = this.keyRegistrar.getType(key); + RedisDataType type = keyRegistrar.getType(key); if (type == null) { return false; } - ScheduledFuture future = this.expirationExecutor + ScheduledFuture future = expirationExecutor .schedule(new ExpirationExecutor(key, type, this), delay, TimeUnit.MILLISECONDS); - this.expirationsMap.put(key, future); + expirationsMap.put(key, future); return true; } @@ -531,14 +531,14 @@ public boolean modifyExpiration(ByteArrayWrapper key, long delay) { return false; } - RedisDataType type = this.keyRegistrar.getType(key); + RedisDataType type = keyRegistrar.getType(key); if (type == null) { return false; } - ScheduledFuture future = this.expirationExecutor + ScheduledFuture future = expirationExecutor .schedule(new ExpirationExecutor(key, type, this), delay, TimeUnit.MILLISECONDS); - this.expirationsMap.put(key, future); + expirationsMap.put(key, future); return true; } @@ -567,7 +567,7 @@ private boolean removeKeyExpiration(ByteArrayWrapper key) { * @return True if key has expiration, false otherwise */ public boolean hasExpiration(ByteArrayWrapper key) { - return this.expirationsMap.containsKey(key); + return expirationsMap.containsKey(key); } /** @@ -577,19 +577,19 @@ public boolean hasExpiration(ByteArrayWrapper key) { * @return Remaining time in milliseconds or 0 if no delay or key doesn't exist */ public long getExpirationDelayMillis(ByteArrayWrapper key) { - ScheduledFuture future = this.expirationsMap.get(key); + ScheduledFuture future = expirationsMap.get(key); return future != null ? future.getDelay(TimeUnit.MILLISECONDS) : 0L; } @Override public void close() { - this.preparedQueries.clear(); + preparedQueries.clear(); } public String dumpRegionsCache() { StringBuilder builder = new StringBuilder(); - for (Entry> e : this.regions.entrySet()) { - builder.append(e.getKey() + " --> {" + e.getValue() + "}\n"); + for (Entry> e : regions.entrySet()) { + builder.append(e.getKey()).append(" --> {").append(e.getValue()).append("}\n"); } return builder.toString(); } diff --git a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/RenameExecutor.java b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/RenameExecutor.java index 0b5d00dacbb8..7458ba026c55 100644 --- a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/RenameExecutor.java +++ b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/RenameExecutor.java @@ -45,13 +45,16 @@ public void executeCommand(Command command, ExecutionHandlerContext context) { return; } - try (AutoCloseableLock lockForOldKey = context.getLockService().lock(key)) { - try (AutoCloseableLock lockForNewKey = context.getLockService().lock(newKey)) { + try (@SuppressWarnings("unused") + AutoCloseableLock lockForOldKey = context.getLockService().lock(key)) { + try (@SuppressWarnings("unused") + AutoCloseableLock lockForNewKey = context.getLockService().lock(newKey)) { RedisDataType redisDataType = context.getKeyRegistrar().getType(key); switch (redisDataType) { case REDIS_STRING: case REDIS_HASH: case REDIS_SET: + @SuppressWarnings("unchecked") Region region = (Region) context.getRegionProvider() .getRegionForType(redisDataType); diff --git a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LIndexExecutor.java b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LIndexExecutor.java index 004233c2f758..dddb024ac805 100755 --- a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LIndexExecutor.java +++ b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LIndexExecutor.java @@ -30,7 +30,7 @@ public class LIndexExecutor extends ListExecutor { - private final String ERROR_NOT_NUMERIC = "The index provided is not numeric"; + private static final String ERROR_NOT_NUMERIC = "The index provided is not numeric"; @Override public void executeCommand(Command command, ExecutionHandlerContext context) { @@ -45,7 +45,7 @@ public void executeCommand(Command command, ExecutionHandlerContext context) { byte[] indexArray = commandElems.get(2); checkDataType(key, RedisDataType.REDIS_LIST, context); - Region keyRegion = getRegion(context, key); + Region keyRegion = getRegion(context, key); if (keyRegion == null) { command.setResponse(Coder.getNilResponse(context.getByteBufAllocator())); @@ -107,7 +107,7 @@ private Struct getEntryAtIndex(ExecutionHandlerContext context, ByteArrayWrapper Query query = getQuery(key, ListQuery.LINDEX, context); - Object[] params = {Integer.valueOf(index + 1)}; + Object[] params = {index + 1}; SelectResults results = (SelectResults) query.execute(params); diff --git a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LLenExecutor.java b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LLenExecutor.java index 2992edb11aba..9dcc2d7cef3c 100755 --- a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LLenExecutor.java +++ b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LLenExecutor.java @@ -26,7 +26,7 @@ public class LLenExecutor extends ListExecutor { - private final int NOT_EXISTS = 0; + private static final int NOT_EXISTS = 0; @Override public void executeCommand(Command command, ExecutionHandlerContext context) { @@ -39,17 +39,15 @@ public void executeCommand(Command command, ExecutionHandlerContext context) { ByteArrayWrapper key = command.getKey(); - int listSize = 0; - checkDataType(key, RedisDataType.REDIS_LIST, context); - Region keyRegion = getRegion(context, key); + Region keyRegion = getRegion(context, key); if (keyRegion == null) { command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), NOT_EXISTS)); return; } - listSize = keyRegion.size() - LIST_EMPTY_SIZE; + int listSize = keyRegion.size() - LIST_EMPTY_SIZE; command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), listSize)); } diff --git a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LRangeExecutor.java b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LRangeExecutor.java index 9b00823cad44..18ddb3860ee6 100755 --- a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LRangeExecutor.java +++ b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LRangeExecutor.java @@ -30,7 +30,7 @@ public class LRangeExecutor extends ListExecutor { - private final String ERROR_NOT_NUMERIC = "The index provided is not numeric"; + private static final String ERROR_NOT_NUMERIC = "The index provided is not numeric"; @Override public void executeCommand(Command command, ExecutionHandlerContext context) { @@ -49,7 +49,7 @@ public void executeCommand(Command command, ExecutionHandlerContext context) { int redisStop; checkDataType(key, RedisDataType.REDIS_LIST, context); - Region keyRegion = getRegion(context, key); + Region keyRegion = getRegion(context, key); if (keyRegion == null) { command.setResponse(Coder.getEmptyArrayResponse(context.getByteBufAllocator())); @@ -81,7 +81,7 @@ public void executeCommand(Command command, ExecutionHandlerContext context) { List range; try { - range = getRange(context, key, redisStart, redisStop, keyRegion); + range = getRange(context, key, redisStart, redisStop); } catch (Exception e) { throw new RuntimeException(e); } @@ -95,14 +95,15 @@ public void executeCommand(Command command, ExecutionHandlerContext context) { } private List getRange(ExecutionHandlerContext context, ByteArrayWrapper key, int start, - int stop, Region r) throws Exception { + int stop) throws Exception { Query query = getQuery(key, ListQuery.LRANGE, context); - Object[] params = {Integer.valueOf(stop + 1)}; + Object[] params = {stop + 1}; + @SuppressWarnings("unchecked") SelectResults results = (SelectResults) query.execute(params); int size = results.size(); - if (results == null || size <= start) { + if (size <= start) { return null; } diff --git a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LRemExecutor.java b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LRemExecutor.java index f92287dcf7fa..4fe653c78f74 100755 --- a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LRemExecutor.java +++ b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LRemExecutor.java @@ -30,9 +30,9 @@ public class LRemExecutor extends ListExecutor { - private final String ERROR_NOT_NUMERIC = "The count provided is not numeric"; + private static final String ERROR_NOT_NUMERIC = "The count provided is not numeric"; - private final int NOT_EXISTS = 0; + private static final int NOT_EXISTS = 0; @Override public void executeCommand(Command command, ExecutionHandlerContext context) { @@ -50,7 +50,7 @@ public void executeCommand(Command command, ExecutionHandlerContext context) { int count; checkDataType(key, RedisDataType.REDIS_LIST, context); - Region keyRegion = getRegion(context, key); + Region keyRegion = getRegion(context, key); if (keyRegion == null) { command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), NOT_EXISTS)); @@ -94,15 +94,16 @@ private List getRemoveList(ExecutionHandlerContext context, ByteArrayWra Query query; if (count > 0) { query = getQuery(key, ListQuery.LREMG, context); - params = new Object[] {value, Integer.valueOf(count)}; + params = new Object[] {value, count}; } else if (count < 0) { query = getQuery(key, ListQuery.LREML, context); - params = new Object[] {value, Integer.valueOf(-count)}; + params = new Object[] {value, -count}; } else { query = getQuery(key, ListQuery.LREME, context); params = new Object[] {value}; } + @SuppressWarnings("unchecked") SelectResults results = (SelectResults) query.execute(params); if (results == null || results.isEmpty()) { diff --git a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LSetExecutor.java b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LSetExecutor.java index 8a9c0b798cd0..f78a599b644d 100755 --- a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LSetExecutor.java +++ b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LSetExecutor.java @@ -29,12 +29,12 @@ public class LSetExecutor extends ListExecutor { - private final String ERROR_NOT_NUMERIC = "The index provided is not numeric"; + private static final String ERROR_NOT_NUMERIC = "The index provided is not numeric"; - private final String ERROR_INDEX = + private static final String ERROR_INDEX = "The index provided is not within range of this list or the key does not exist"; - private final String SUCCESS = "OK"; + private static final String SUCCESS = "OK"; @Override public void executeCommand(Command command, ExecutionHandlerContext context) { @@ -52,7 +52,7 @@ public void executeCommand(Command command, ExecutionHandlerContext context) { int index; checkDataType(key, RedisDataType.REDIS_LIST, context); - Region keyRegion = getRegion(context, key); + Region keyRegion = getRegion(context, key); if (keyRegion == null) { command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ERROR_INDEX)); @@ -96,11 +96,12 @@ private Integer getIndexKey(ExecutionHandlerContext context, ByteArrayWrapper ke throws Exception { Query query = getQuery(key, ListQuery.LSET, context); - Object[] params = {Integer.valueOf(index + 1)}; + Object[] params = {index + 1}; + @SuppressWarnings("unchecked") SelectResults results = (SelectResults) query.execute(params); int size = results.size(); - if (results == null || size == 0) { + if (size == 0) { return null; } diff --git a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LTrimExecutor.java b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LTrimExecutor.java index bfdd783c1b2a..9dd9c19b55a3 100755 --- a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LTrimExecutor.java +++ b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/LTrimExecutor.java @@ -29,11 +29,11 @@ public class LTrimExecutor extends ListExecutor { - private final String ERROR_KEY_NOT_EXISTS = "The key does not exists on this server"; + private static final String ERROR_KEY_NOT_EXISTS = "The key does not exists on this server"; - private final String ERROR_NOT_NUMERIC = "The index provided is not numeric"; + private static final String ERROR_NOT_NUMERIC = "The index provided is not numeric"; - private final String SUCCESS = "OK"; + private static final String SUCCESS = "OK"; @Override public void executeCommand(Command command, ExecutionHandlerContext context) { @@ -52,7 +52,7 @@ public void executeCommand(Command command, ExecutionHandlerContext context) { int redisStop; checkDataType(key, RedisDataType.REDIS_LIST, context); - Region keyRegion = getRegion(context, key); + Region keyRegion = getRegion(context, key); if (keyRegion == null) { command @@ -90,7 +90,7 @@ public void executeCommand(Command command, ExecutionHandlerContext context) { List keepList; try { - keepList = getRange(context, key, redisStart, redisStop, keyRegion); + keepList = getRange(context, key, redisStart, redisStop); } catch (Exception e) { throw new RuntimeException(e); } @@ -108,11 +108,12 @@ public void executeCommand(Command command, ExecutionHandlerContext context) { } private List getRange(ExecutionHandlerContext context, ByteArrayWrapper key, int start, - int stop, Region r) throws Exception { + int stop) throws Exception { Query query = getQuery(key, ListQuery.LTRIM, context); - Object[] params = {Integer.valueOf(stop + 1)}; + Object[] params = {stop + 1}; + @SuppressWarnings("unchecked") SelectResults results = (SelectResults) query.execute(params); if (results == null || results.size() <= start) { return null; diff --git a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/ListExecutor.java b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/ListExecutor.java index 1ac6da8429bd..075d6d1a95f7 100755 --- a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/ListExecutor.java +++ b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/ListExecutor.java @@ -17,8 +17,6 @@ import java.util.List; import org.apache.geode.cache.Region; -import org.apache.geode.cache.query.QueryService; -import org.apache.geode.internal.cache.GemFireCacheImpl; import org.apache.geode.redis.internal.ByteArrayWrapper; import org.apache.geode.redis.internal.ExecutionHandlerContext; import org.apache.geode.redis.internal.RedisDataType; @@ -32,25 +30,19 @@ protected enum ListDirection { LEFT, RIGHT } - ; - - protected static QueryService getQueryService() { - return GemFireCacheImpl.getInstance().getQueryService(); - } - @SuppressWarnings("unchecked") @Override - protected Region getOrCreateRegion(ExecutionHandlerContext context, + protected Region getOrCreateRegion(ExecutionHandlerContext context, ByteArrayWrapper key, RedisDataType type) { - return (Region) context.getRegionProvider().getOrCreateRegion(key, + return (Region) context.getRegionProvider().getOrCreateRegion(key, type, context); } @SuppressWarnings("unchecked") - protected Region getRegion(ExecutionHandlerContext context, + protected Region getRegion(ExecutionHandlerContext context, ByteArrayWrapper key) { - return (Region) context.getRegionProvider().getRegion(key); + return (Region) context.getRegionProvider().getRegion(key); } /** @@ -68,7 +60,7 @@ protected Region getRegion(ExecutionHandlerContext co * @param context Context of this push */ protected void pushElements(ByteArrayWrapper key, List commandElems, int startIndex, - int endIndex, Region keyRegion, ListDirection pushType, + int endIndex, Region keyRegion, ListDirection pushType, ExecutionHandlerContext context) { String indexKey = pushType == ListDirection.LEFT ? "head" : "tail"; @@ -129,7 +121,7 @@ protected void pushElements(ByteArrayWrapper key, List commandElems, int * */ - boolean indexSet = false; + boolean indexSet; do { Integer existingIndex = (Integer) keyRegion.get(indexKey); if (index != null && ((pushType == ListDirection.RIGHT && existingIndex < index) diff --git a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/PopExecutor.java b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/PopExecutor.java index e12adb742396..5077d9902826 100755 --- a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/PopExecutor.java +++ b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/PopExecutor.java @@ -38,7 +38,7 @@ public void executeCommand(Command command, ExecutionHandlerContext context) { ByteArrayWrapper key = command.getKey(); checkDataType(key, RedisDataType.REDIS_LIST, context); - Region keyRegion = getRegion(context, key); + Region keyRegion = getRegion(context, key); if (keyRegion == null || keyRegion.size() == LIST_EMPTY_SIZE) { command.setResponse(Coder.getNilResponse(context.getByteBufAllocator())); @@ -50,7 +50,7 @@ public void executeCommand(Command command, ExecutionHandlerContext context) { Integer index = 0; int originalIndex = index; int incr = popType() == ListDirection.LEFT ? 1 : -1; - ByteArrayWrapper valueWrapper = null; + ByteArrayWrapper valueWrapper; /** * @@ -59,7 +59,7 @@ public void executeCommand(Command command, ExecutionHandlerContext context) { * */ - boolean indexChanged = false; + boolean indexChanged; do { index = (Integer) keyRegion.get(indexKey); Integer opp = (Integer) keyRegion.get(oppositeKey); @@ -124,7 +124,7 @@ public void executeCommand(Command command, ExecutionHandlerContext context) { index = metaIndex; } i++; - } while (!removed && keyRegion.size() != LIST_EMPTY_SIZE); + } while (keyRegion.size() != LIST_EMPTY_SIZE); respondBulkStrings(command, context, valueWrapper); } diff --git a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/PushExecutor.java b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/PushExecutor.java index fa1d5067d917..4d27e328ec75 100755 --- a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/PushExecutor.java +++ b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/PushExecutor.java @@ -26,7 +26,7 @@ public abstract class PushExecutor extends PushXExecutor implements Extendable { - private final int START_VALUES_INDEX = 2; + private static final int START_VALUES_INDEX = 2; @Override public void executeCommand(Command command, ExecutionHandlerContext context) { @@ -39,7 +39,7 @@ public void executeCommand(Command command, ExecutionHandlerContext context) { ByteArrayWrapper key = command.getKey(); - Region keyRegion = + Region keyRegion = getOrCreateRegion(context, key, RedisDataType.REDIS_LIST); pushElements(key, commandElems, START_VALUES_INDEX, commandElems.size(), keyRegion, pushType(), context); diff --git a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/PushXExecutor.java b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/PushXExecutor.java index 3123f92a4620..3f1ae4397e02 100755 --- a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/PushXExecutor.java +++ b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/list/PushXExecutor.java @@ -26,7 +26,7 @@ public abstract class PushXExecutor extends ListExecutor implements Extendable { - private final int NOT_EXISTS = 0; + private static final int NOT_EXISTS = 0; @Override public void executeCommand(Command command, ExecutionHandlerContext context) { @@ -39,7 +39,7 @@ public void executeCommand(Command command, ExecutionHandlerContext context) { ByteArrayWrapper key = command.getKey(); - Region keyRegion = getRegion(context, key); + Region keyRegion = getRegion(context, key); if (keyRegion == null) { command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), NOT_EXISTS)); return; diff --git a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/GeoSortedSetExecutor.java b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/GeoSortedSetExecutor.java index 70f0f408e6d2..52926c58dd95 100755 --- a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/GeoSortedSetExecutor.java +++ b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/GeoSortedSetExecutor.java @@ -59,6 +59,7 @@ protected List getGeoRadiusRange(ExecutionHandlerContext context, ByteArrayWrapper key, String hash) throws Exception { Query query = getQuery(key, SortedSetQuery.GEORADIUS, context); Object[] params = {hash + "%"}; + @SuppressWarnings("unchecked") SelectResults results = (SelectResults) query.execute(params); return results.asList(); } diff --git a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/ZRangeByLexExecutor.java b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/ZRangeByLexExecutor.java index 03864f5f6d81..0c48856c5818 100755 --- a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/ZRangeByLexExecutor.java +++ b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/ZRangeByLexExecutor.java @@ -40,12 +40,12 @@ public class ZRangeByLexExecutor extends SortedSetExecutor { - private final String ERROR_NOT_NUMERIC = "The index provided is not numeric"; + private static final String ERROR_NOT_NUMERIC = "The index provided is not numeric"; - private final String ERROR_ILLEGAL_SYNTAX = + private static final String ERROR_ILLEGAL_SYNTAX = "The min and max strings must either start with a (, [ or be - or +"; - private final String ERROR_LIMIT = "The offset and count cannot be negative"; + private static final String ERROR_LIMIT = "The offset and count cannot be negative"; @Override public void executeCommand(Command command, ExecutionHandlerContext context) { @@ -149,7 +149,7 @@ private List getRange(ByteArrayWrapper key, throws FunctionDomainException, TypeMismatchException, NameResolutionException, QueryInvocationTargetException { if (start.equals(minus) && stop.equals(plus)) { - List l = new ArrayList(keyRegion.keySet()); + List l = new ArrayList<>(keyRegion.keySet()); int size = l.size(); Collections.sort(l); if (limit == 0) { @@ -196,6 +196,7 @@ private List getRange(ByteArrayWrapper key, if (limit > 0) { params[params.length - 1] = (limit + offset); } + @SuppressWarnings("unchecked") SelectResults results = (SelectResults) query.execute(params); List list = results.asList(); diff --git a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/ZRangeByScoreExecutor.java b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/ZRangeByScoreExecutor.java index 23f4aab9b168..bb71fa15afed 100755 --- a/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/ZRangeByScoreExecutor.java +++ b/geode-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/ZRangeByScoreExecutor.java @@ -24,7 +24,6 @@ import org.apache.geode.cache.query.Query; import org.apache.geode.cache.query.QueryInvocationTargetException; import org.apache.geode.cache.query.SelectResults; -import org.apache.geode.cache.query.Struct; import org.apache.geode.cache.query.TypeMismatchException; import org.apache.geode.redis.internal.ByteArrayWrapper; import org.apache.geode.redis.internal.Coder; @@ -38,9 +37,9 @@ public class ZRangeByScoreExecutor extends SortedSetExecutor implements Extendable { - private final String ERROR_NOT_NUMERIC = "The number provided is not numeric"; + private static final String ERROR_NOT_NUMERIC = "The number provided is not numeric"; - private final String ERROR_LIMIT = "The offset and count cannot be negative"; + private static final String ERROR_LIMIT = "The offset and count cannot be negative"; @Override public void executeCommand(Command command, ExecutionHandlerContext context) { @@ -52,7 +51,7 @@ public void executeCommand(Command command, ExecutionHandlerContext context) { } boolean withScores = false; - byte[] elem4Array = null; + byte[] elem4Array; int offset = 0; int limit = -1; if (commandElems.size() >= 5) { @@ -162,7 +161,7 @@ private Collection getKeys(ByteArrayWrapper key, return null; } if (start == Double.NEGATIVE_INFINITY && stop == Double.POSITIVE_INFINITY) { - return new HashSet(keyRegion.entrySet()); + return new HashSet<>(keyRegion.entrySet()); } Query query; @@ -181,7 +180,6 @@ private Collection getKeys(ByteArrayWrapper key, query = getQuery(key, SortedSetQuery.ZREVRBS, context); } } - params = new Object[] {start, stop, INFINITY_LIMIT}; } else { if (startInclusive) { if (stopInclusive) { @@ -196,15 +194,15 @@ private Collection getKeys(ByteArrayWrapper key, query = getQuery(key, SortedSetQuery.ZRBS, context); } } - params = new Object[] {start, stop, INFINITY_LIMIT}; } + params = new Object[] {start, stop, INFINITY_LIMIT}; if (limit > 0) { params[params.length - 1] = (limit + offset); } SelectResults results = (SelectResults) query.execute(params); if (offset < results.size()) { - return (Collection) results.asList().subList(offset, results.size()); + return results.asList().subList(offset, results.size()); } else { return null; } diff --git a/geode-redis/src/test/java/org/apache/geode/redis/internal/ExecutionHandlerContextJUnitTest.java b/geode-redis/src/test/java/org/apache/geode/redis/internal/ExecutionHandlerContextJUnitTest.java index 324a99527a0c..90bacbacf96c 100644 --- a/geode-redis/src/test/java/org/apache/geode/redis/internal/ExecutionHandlerContextJUnitTest.java +++ b/geode-redis/src/test/java/org/apache/geode/redis/internal/ExecutionHandlerContextJUnitTest.java @@ -22,7 +22,6 @@ import org.junit.Test; import org.mockito.Mockito; -import org.apache.geode.LogWriter; import org.apache.geode.cache.Cache; import org.apache.geode.redis.GeodeRedisServer; @@ -42,7 +41,8 @@ public void testChannelReadChannelHandlerContextObject() throws Exception { ChannelPipeline channelPipeline = Mockito.mock(ChannelPipeline.class); EventExecutor eventExecutor = Mockito.mock(EventExecutor.class); ChannelHandlerContext channelHandlerContext = Mockito.mock(ChannelHandlerContext.class); - LogWriter logWriter = Mockito.mock(LogWriter.class); + @SuppressWarnings("deprecation") + org.apache.geode.LogWriter logWriter = Mockito.mock(org.apache.geode.LogWriter.class); Command msg = Mockito.mock(Command.class); RegionProvider regionProvider = Mockito.mock(RegionProvider.class); GeodeRedisServer server = Mockito.mock(GeodeRedisServer.class); diff --git a/geode-redis/src/test/java/org/apache/geode/redis/internal/executor/string/StringSetExecutorJUnitTest.java b/geode-redis/src/test/java/org/apache/geode/redis/internal/executor/string/StringSetExecutorJUnitTest.java index a89f8938505b..c05cf0061e61 100644 --- a/geode-redis/src/test/java/org/apache/geode/redis/internal/executor/string/StringSetExecutorJUnitTest.java +++ b/geode-redis/src/test/java/org/apache/geode/redis/internal/executor/string/StringSetExecutorJUnitTest.java @@ -50,10 +50,10 @@ public class StringSetExecutorJUnitTest { private ExecutionHandlerContext context; private SetExecutor executor; private ByteBuf buffer; - private KeyRegistrar keyRegistrar; private Region region; private RegionProvider regionProvider; + @SuppressWarnings("unchecked") @Before public void setup() { command = mock(Command.class); @@ -72,7 +72,7 @@ public void setup() { when(allocator.buffer(anyInt())).thenReturn(buffer); when(context.getByteBufAllocator()).thenReturn(allocator); - keyRegistrar = mock(KeyRegistrar.class); + KeyRegistrar keyRegistrar = mock(KeyRegistrar.class); when(context.getKeyRegistrar()).thenReturn(keyRegistrar); executor = spy(new SetExecutor());