diff --git a/src/main/java/io/lettuce/core/cluster/models/partitions/RedisClusterNode.java b/src/main/java/io/lettuce/core/cluster/models/partitions/RedisClusterNode.java index dc78f2eda7..e6e05d331b 100644 --- a/src/main/java/io/lettuce/core/cluster/models/partitions/RedisClusterNode.java +++ b/src/main/java/io/lettuce/core/cluster/models/partitions/RedisClusterNode.java @@ -16,26 +16,28 @@ package io.lettuce.core.cluster.models.partitions; import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; -import java.util.Set; +import java.util.*; import io.lettuce.core.RedisURI; +import io.lettuce.core.cluster.SlotHash; import io.lettuce.core.internal.LettuceAssert; -import io.lettuce.core.internal.LettuceSets; import io.lettuce.core.models.role.RedisNodeDescription; /** - * Representation of a Redis Cluster node. A {@link RedisClusterNode} is identified by its {@code nodeId}. A - * {@link RedisClusterNode} can be a {@link #getRole() responsible master} for zero to - * {@link io.lettuce.core.cluster.SlotHash#SLOT_COUNT 16384} slots, a slave of one {@link #getSlaveOf() master} of carry - * different {@link io.lettuce.core.cluster.models.partitions.RedisClusterNode.NodeFlag flags}. + * Representation of a Redis Cluster node. A {@link RedisClusterNode} is identified by its {@code nodeId}. + *

+ * A {@link RedisClusterNode} can be a {@link #getRole() responsible master} or slave. Masters can be responsible for zero to + * {@link io.lettuce.core.cluster.SlotHash#SLOT_COUNT 16384} slots. Each slave refers to exactly one {@link #getSlaveOf() + * master}. Nodes can have different {@link io.lettuce.core.cluster.models.partitions.RedisClusterNode.NodeFlag flags} assigned. + *

+ * This class is mutable and not thread-safe if mutated by multiple threads concurrently. * * @author Mark Paluch * @since 3.0 */ @SuppressWarnings("serial") public class RedisClusterNode implements Serializable, RedisNodeDescription { + private RedisURI uri; private String nodeId; @@ -45,15 +47,15 @@ public class RedisClusterNode implements Serializable, RedisNodeDescription { private long pongReceivedTimestamp; private long configEpoch; - private List slots; - private Set flags; + private BitSet slots; + private final Set flags = EnumSet.noneOf(NodeFlag.class); public RedisClusterNode() { - } public RedisClusterNode(RedisURI uri, String nodeId, boolean connected, String slaveOf, long pingSentTimestamp, long pongReceivedTimestamp, long configEpoch, List slots, Set flags) { + this.uri = uri; this.nodeId = nodeId; this.connected = connected; @@ -61,11 +63,15 @@ public RedisClusterNode(RedisURI uri, String nodeId, boolean connected, String s this.pingSentTimestamp = pingSentTimestamp; this.pongReceivedTimestamp = pongReceivedTimestamp; this.configEpoch = configEpoch; - this.slots = slots; - this.flags = flags; + + setSlotBits(slots); + setFlags(flags); } public RedisClusterNode(RedisClusterNode redisClusterNode) { + + LettuceAssert.notNull(redisClusterNode, "RedisClusterNode must not be null"); + this.uri = redisClusterNode.uri; this.nodeId = redisClusterNode.nodeId; this.connected = redisClusterNode.connected; @@ -73,8 +79,13 @@ public RedisClusterNode(RedisClusterNode redisClusterNode) { this.pingSentTimestamp = redisClusterNode.pingSentTimestamp; this.pongReceivedTimestamp = redisClusterNode.pongReceivedTimestamp; this.configEpoch = redisClusterNode.configEpoch; - this.slots = new ArrayList<>(redisClusterNode.slots); - this.flags = LettuceSets.newHashSet(redisClusterNode.flags); + + if (redisClusterNode.slots != null && !redisClusterNode.slots.isEmpty()) { + this.slots = new BitSet(SlotHash.SLOT_COUNT); + this.slots.or(redisClusterNode.slots); + } + + setFlags(redisClusterNode.flags); } /** @@ -84,8 +95,12 @@ public RedisClusterNode(RedisClusterNode redisClusterNode) { * @return a new instance of {@link RedisClusterNode} */ public static RedisClusterNode of(String nodeId) { + + LettuceAssert.notNull(nodeId, "NodeId must not be null"); + RedisClusterNode redisClusterNode = new RedisClusterNode(); redisClusterNode.setNodeId(nodeId); + return redisClusterNode; } @@ -94,11 +109,12 @@ public RedisURI getUri() { } /** - * Sets thhe connection point details. Usually the host/ip/port where a particular Redis Cluster node server is running. + * Sets the connection point details. Usually the host/ip/port where a particular Redis Cluster node server is running. * * @param uri the {@link RedisURI}, must not be {@literal null} */ public void setUri(RedisURI uri) { + LettuceAssert.notNull(uri, "RedisURI must not be null"); this.uri = uri; } @@ -184,20 +200,52 @@ public void setConfigEpoch(long configEpoch) { } public List getSlots() { + + if (slots == null || slots.isEmpty()) { + return Collections.emptyList(); + } + + List slots = new ArrayList<>(); + + for (int i = 0; i < SlotHash.SLOT_COUNT; i++) { + + if (this.slots.get(i)) { + slots.add(i); + } + } + return slots; } /** * Sets the list of slots for which this {@link RedisClusterNode} is the - * {@link io.lettuce.core.cluster.models.partitions.RedisClusterNode.NodeFlag#MASTER}. The list is empty if this node - * is not a master or the node is not responsible for any slots at all. + * {@link io.lettuce.core.cluster.models.partitions.RedisClusterNode.NodeFlag#MASTER}. The list is empty if this node is not + * a master or the node is not responsible for any slots at all. * * @param slots list of slots, must not be {@literal null} but may be empty */ public void setSlots(List slots) { + LettuceAssert.notNull(slots, "Slots must not be null"); - this.slots = slots; + setSlotBits(slots); + } + + private void setSlotBits(List slots) { + + if (slots.isEmpty() && this.slots == null) { + return; + } + + if (this.slots == null) { + this.slots = new BitSet(SlotHash.SLOT_COUNT); + } + + this.slots.clear(); + + for (Integer slot : slots) { + this.slots.set(slot); + } } public Set getFlags() { @@ -210,7 +258,35 @@ public Set getFlags() { * @param flags the set of node flags. */ public void setFlags(Set flags) { - this.flags = flags; + + this.flags.clear(); + this.flags.addAll(flags); + } + + /** + * @param nodeFlag the node flag + * @return true if the {@linkplain NodeFlag} is contained within the flags. + */ + public boolean is(NodeFlag nodeFlag) { + return getFlags().contains(nodeFlag); + } + + /** + * @param slot the slot hash + * @return true if the slot is contained within the handled slots. + */ + public boolean hasSlot(int slot) { + return slot <= SlotHash.SLOT_COUNT && this.slots != null && this.slots.get(slot); + } + + /** + * Returns the {@link Role} of the Redis Cluster node based on the {@link #getFlags() flags}. + * + * @return the Redis Cluster node role + */ + @Override + public Role getRole() { + return is(NodeFlag.MASTER) ? Role.MASTER : Role.SLAVE; } @Override @@ -233,13 +309,12 @@ public boolean equals(Object o) { @Override public int hashCode() { - int result = 31 * (nodeId != null ? nodeId.hashCode() : 0); - return result; + return 31 * (nodeId != null ? nodeId.hashCode() : 0); } @Override public String toString() { - final StringBuilder sb = new StringBuilder(); + StringBuilder sb = new StringBuilder(); sb.append(getClass().getSimpleName()); sb.append(" [uri=").append(uri); sb.append(", nodeId='").append(nodeId).append('\''); @@ -250,46 +325,16 @@ public String toString() { sb.append(", configEpoch=").append(configEpoch); sb.append(", flags=").append(flags); if (slots != null) { - sb.append(", slot count=").append(slots.size()); + sb.append(", slot count=").append(slots.cardinality()); } sb.append(']'); return sb.toString(); } - /** - * - * @param nodeFlag the node flag - * @return true if the {@linkplain NodeFlag} is contained within the flags. - */ - public boolean is(NodeFlag nodeFlag) { - return getFlags().contains(nodeFlag); - } - - /** - * - * @param slot the slot hash - * @return true if the slot is contained within the handled slots. - */ - public boolean hasSlot(int slot) { - return getSlots().contains(slot); - } - - /** - * Returns the {@link io.lettuce.core.models.role.RedisInstance.Role} of the Redis Cluster node based on the - * {@link #getFlags() flags}. - * - * @return the Redis Cluster node role - */ - @Override - public Role getRole() { - return is(NodeFlag.MASTER) ? Role.MASTER : Role.SLAVE; - } - /** * Redis Cluster node flags. */ public enum NodeFlag { NOFLAGS, MYSELF, SLAVE, MASTER, EVENTUAL_FAIL, FAIL, HANDSHAKE, NOADDR; } - } diff --git a/src/test/java/io/lettuce/core/cluster/RedisClusterClientTest.java b/src/test/java/io/lettuce/core/cluster/RedisClusterClientTest.java index 1d3cd2c342..f78e41d799 100644 --- a/src/test/java/io/lettuce/core/cluster/RedisClusterClientTest.java +++ b/src/test/java/io/lettuce/core/cluster/RedisClusterClientTest.java @@ -24,6 +24,8 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import org.junit.*; import org.junit.runners.MethodSorters; @@ -292,13 +294,9 @@ public void testClusterRedirection() throws Exception { Partitions partitions = clusterClient.getPartitions(); for (RedisClusterNode partition : partitions) { - partition.setSlots(new ArrayList<>()); + partition.setSlots(Collections.emptyList()); if (partition.getFlags().contains(RedisClusterNode.NodeFlag.MYSELF)) { - - int[] slots = createSlots(0, 16384); - for (int i = 0; i < slots.length; i++) { - partition.getSlots().add(i); - } + partition.setSlots(IntStream.range(0, SlotHash.SLOT_COUNT).boxed().collect(Collectors.toList())); } } partitions.updateCache(); @@ -335,13 +333,9 @@ public void testClusterRedirectionLimit() throws Exception { for (RedisClusterNode partition : partitions) { if (partition.getSlots().contains(15495)) { - partition.setSlots(new ArrayList<>()); + partition.setSlots(Collections.emptyList()); } else { - partition.setSlots(new ArrayList<>()); - int[] slots = createSlots(0, 16384); - for (int i = 0; i < slots.length; i++) { - partition.getSlots().add(i); - } + partition.setSlots(IntStream.range(0, SlotHash.SLOT_COUNT).boxed().collect(Collectors.toList())); } } diff --git a/src/test/java/io/lettuce/core/cluster/models/partitions/RedisClusterNodeTest.java b/src/test/java/io/lettuce/core/cluster/models/partitions/RedisClusterNodeTest.java index e395072cd8..05f121b589 100644 --- a/src/test/java/io/lettuce/core/cluster/models/partitions/RedisClusterNodeTest.java +++ b/src/test/java/io/lettuce/core/cluster/models/partitions/RedisClusterNodeTest.java @@ -17,13 +17,34 @@ import static org.assertj.core.api.Assertions.assertThat; +import java.util.Arrays; + import org.junit.Test; import io.lettuce.core.RedisURI; +import io.lettuce.core.cluster.SlotHash; +/** + * @author Mark Paluch + */ public class RedisClusterNodeTest { + @Test - public void testEquality() throws Exception { + public void shouldCopyNode() { + + RedisClusterNode node = new RedisClusterNode(); + node.setSlots(Arrays.asList(1, 2, 3, SlotHash.SLOT_COUNT - 1)); + + RedisClusterNode copy = new RedisClusterNode(node); + + assertThat(copy.getSlots()).containsExactly(1, 2, 3, SlotHash.SLOT_COUNT - 1); + assertThat(copy.hasSlot(1)).isTrue(); + assertThat(copy.hasSlot(SlotHash.SLOT_COUNT - 1)).isTrue(); + } + + @Test + public void testEquality() { + RedisClusterNode node = new RedisClusterNode(); assertThat(node).isEqualTo(new RedisClusterNode()); @@ -31,11 +52,11 @@ public void testEquality() throws Exception { node.setUri(new RedisURI()); assertThat(node.hashCode()).isNotEqualTo(new RedisClusterNode()); - } @Test - public void testToString() throws Exception { + public void testToString() { + RedisClusterNode node = new RedisClusterNode(); assertThat(node.toString()).contains(RedisClusterNode.class.getSimpleName()); diff --git a/src/test/java/io/lettuce/core/cluster/models/slots/ClusterSlotsParserTest.java b/src/test/java/io/lettuce/core/cluster/models/slots/ClusterSlotsParserTest.java index 43ab839c5d..cdb949ed8e 100644 --- a/src/test/java/io/lettuce/core/cluster/models/slots/ClusterSlotsParserTest.java +++ b/src/test/java/io/lettuce/core/cluster/models/slots/ClusterSlotsParserTest.java @@ -30,26 +30,26 @@ public class ClusterSlotsParserTest { @Test - public void testEmpty() throws Exception { + public void testEmpty() { List result = ClusterSlotsParser.parse(new ArrayList<>()); assertThat(result).isNotNull().isEmpty(); } @Test - public void testOneString() throws Exception { + public void testOneString() { List result = ClusterSlotsParser.parse(LettuceLists.newList("")); assertThat(result).isNotNull().isEmpty(); } @Test - public void testOneStringInList() throws Exception { + public void testOneStringInList() { List list = Arrays.asList(LettuceLists.newList("0")); List result = ClusterSlotsParser.parse(list); assertThat(result).isNotNull().isEmpty(); } @Test - public void testParse() throws Exception { + public void testParse() { List list = Arrays.asList(LettuceLists.newList("0", "1", LettuceLists.newList("1", "2"))); List result = ClusterSlotsParser.parse(list); assertThat(result).hasSize(1); @@ -58,7 +58,7 @@ public void testParse() throws Exception { } @Test - public void testParseWithSlave() throws Exception { + public void testParseWithSlave() { List list = Arrays.asList(LettuceLists.newList("100", "200", LettuceLists.newList("1", "2", "nodeId1"), LettuceLists.newList("1", 2, "nodeId2"))); List result = ClusterSlotsParser.parse(list); @@ -85,7 +85,7 @@ public void testParseWithSlave() throws Exception { } @Test - public void testSameNode() throws Exception { + public void testSameNode() { List list = Arrays.asList( LettuceLists.newList("100", "200", LettuceLists.newList("1", "2", "nodeId1"), LettuceLists.newList("1", 2, "nodeId2")), @@ -109,19 +109,19 @@ public void testSameNode() throws Exception { } @Test(expected = IllegalArgumentException.class) - public void testParseInvalidMaster() throws Exception { + public void testParseInvalidMaster() { List list = Arrays.asList(LettuceLists.newList("0", "1", LettuceLists.newList("1"))); ClusterSlotsParser.parse(list); } @Test(expected = IllegalArgumentException.class) - public void testParseInvalidMaster2() throws Exception { + public void testParseInvalidMaster2() { List list = Arrays.asList(LettuceLists.newList("0", "1", "")); ClusterSlotsParser.parse(list); } @Test - public void testModel() throws Exception { + public void testModel() { ClusterSlotRange range = new ClusterSlotRange(); range.setFrom(1); diff --git a/src/test/java/io/lettuce/core/cluster/topology/TopologyRefreshTest.java b/src/test/java/io/lettuce/core/cluster/topology/TopologyRefreshTest.java index 04be54e1f9..e26c11583a 100644 --- a/src/test/java/io/lettuce/core/cluster/topology/TopologyRefreshTest.java +++ b/src/test/java/io/lettuce/core/cluster/topology/TopologyRefreshTest.java @@ -17,6 +17,8 @@ import static org.assertj.core.api.Assertions.assertThat; +import java.util.Collections; +import java.util.List; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -62,22 +64,22 @@ public class TopologyRefreshTest extends AbstractTest { private RedisCommands redis2; @Before - public void openConnection() throws Exception { - clusterClient = RedisClusterClient.create(client.getResources(), - RedisURI.Builder.redis(host, AbstractClusterTest.port1).build()); + public void openConnection() { + clusterClient = RedisClusterClient.create(client.getResources(), RedisURI.Builder + .redis(host, AbstractClusterTest.port1).build()); redis1 = client.connect(RedisURI.Builder.redis(AbstractClusterTest.host, AbstractClusterTest.port1).build()).sync(); redis2 = client.connect(RedisURI.Builder.redis(AbstractClusterTest.host, AbstractClusterTest.port2).build()).sync(); } @After - public void closeConnection() throws Exception { + public void closeConnection() { redis1.getStatefulConnection().close(); redis2.getStatefulConnection().close(); FastShutdown.shutdown(clusterClient); } @Test - public void shouldUnsubscribeTopologyRefresh() throws Exception { + public void shouldUnsubscribeTopologyRefresh() { ClusterTopologyRefreshOptions topologyRefreshOptions = ClusterTopologyRefreshOptions.builder() .enablePeriodicRefresh(true) // @@ -89,8 +91,8 @@ public void shouldUnsubscribeTopologyRefresh() throws Exception { AtomicBoolean clusterTopologyRefreshActivated = (AtomicBoolean) ReflectionTestUtils.getField(clusterClient, "clusterTopologyRefreshActivated"); - AtomicReference> clusterTopologyRefreshFuture = (AtomicReference) ReflectionTestUtils - .getField(clusterClient, "clusterTopologyRefreshFuture"); + AtomicReference> clusterTopologyRefreshFuture = (AtomicReference) ReflectionTestUtils.getField( + clusterClient, "clusterTopologyRefreshFuture"); assertThat(clusterTopologyRefreshActivated.get()).isTrue(); assertThat((Future) clusterTopologyRefreshFuture.get()).isNotNull(); @@ -107,7 +109,7 @@ public void shouldUnsubscribeTopologyRefresh() throws Exception { } @Test - public void changeTopologyWhileOperations() throws Exception { + public void changeTopologyWhileOperations() { ClusterTopologyRefreshOptions topologyRefreshOptions = ClusterTopologyRefreshOptions.builder() .enablePeriodicRefresh(true)// @@ -126,7 +128,7 @@ public void changeTopologyWhileOperations() throws Exception { } @Test - public void dynamicSourcesProvidesClientCountForAllNodes() throws Exception { + public void dynamicSourcesProvidesClientCountForAllNodes() { ClusterTopologyRefreshOptions topologyRefreshOptions = ClusterTopologyRefreshOptions.create(); clusterClient.setOptions(ClusterClientOptions.builder().topologyRefreshOptions(topologyRefreshOptions).build()); @@ -143,7 +145,7 @@ public void dynamicSourcesProvidesClientCountForAllNodes() throws Exception { } @Test - public void staticSourcesProvidesClientCountForSeedNodes() throws Exception { + public void staticSourcesProvidesClientCountForSeedNodes() { ClusterTopologyRefreshOptions topologyRefreshOptions = ClusterTopologyRefreshOptions.builder() .dynamicRefreshSources(false).build(); @@ -161,7 +163,7 @@ public void staticSourcesProvidesClientCountForSeedNodes() throws Exception { } @Test - public void adaptiveTopologyUpdateOnDisconnectNodeIdConnection() throws Exception { + public void adaptiveTopologyUpdateOnDisconnectNodeIdConnection() { runReconnectTest((clusterConnection, node) -> { RedisClusterAsyncCommands connection = clusterConnection.getConnection(node.getUri().getHost(), @@ -172,7 +174,7 @@ public void adaptiveTopologyUpdateOnDisconnectNodeIdConnection() throws Exceptio } @Test - public void adaptiveTopologyUpdateOnDisconnectHostAndPortConnection() throws Exception { + public void adaptiveTopologyUpdateOnDisconnectHostAndPortConnection() { runReconnectTest((clusterConnection, node) -> { RedisClusterAsyncCommands connection = clusterConnection.getConnection(node.getUri().getHost(), @@ -183,7 +185,7 @@ public void adaptiveTopologyUpdateOnDisconnectHostAndPortConnection() throws Exc } @Test - public void adaptiveTopologyUpdateOnDisconnectDefaultConnection() throws Exception { + public void adaptiveTopologyUpdateOnDisconnectDefaultConnection() { runReconnectTest((clusterConnection, node) -> { return clusterConnection; @@ -265,7 +267,7 @@ public void adaptiveTriggerDoesNotFireOnSingleReconnect() throws Exception { } @Test - public void adaptiveTriggerOnMoveRedirection() throws Exception { + public void adaptiveTriggerOnMoveRedirection() { ClusterTopologyRefreshOptions topologyRefreshOptions = ClusterTopologyRefreshOptions.builder()// .enableAdaptiveRefreshTrigger(ClusterTopologyRefreshOptions.RefreshTrigger.MOVED_REDIRECT)// @@ -279,8 +281,10 @@ public void adaptiveTriggerOnMoveRedirection() throws Exception { RedisClusterNode node1 = partitions.getPartitionBySlot(0); RedisClusterNode node2 = partitions.getPartitionBySlot(12000); - node2.getSlots().addAll(node1.getSlots()); - node1.getSlots().clear(); + List slots = node2.getSlots(); + slots.addAll(node1.getSlots()); + node2.setSlots(slots); + node1.setSlots(Collections.emptyList()); partitions.updateCache(); assertThat(clusterClient.getPartitions().getPartitionByNodeId(node1.getNodeId()).getSlots()).hasSize(0); @@ -290,7 +294,7 @@ public void adaptiveTriggerOnMoveRedirection() throws Exception { Wait.untilEquals(12000, new Wait.Supplier() { @Override - public Integer get() throws Exception { + public Integer get() { return clusterClient.getPartitions().getPartitionByNodeId(node1.getNodeId()).getSlots().size(); } }).waitOrTimeout(); @@ -301,8 +305,7 @@ public Integer get() throws Exception { } private void runReconnectTest( - BiFunction, RedisClusterNode, BaseRedisAsyncCommands> function) - throws Exception { + BiFunction, RedisClusterNode, BaseRedisAsyncCommands> function) { ClusterTopologyRefreshOptions topologyRefreshOptions = ClusterTopologyRefreshOptions.builder()// .refreshTriggersReconnectAttempts(0)// diff --git a/src/test/jmh/io/lettuce/core/cluster/models/partitions/JmhMain.java b/src/test/jmh/io/lettuce/core/cluster/models/partitions/JmhMain.java new file mode 100644 index 0000000000..336a82b78e --- /dev/null +++ b/src/test/jmh/io/lettuce/core/cluster/models/partitions/JmhMain.java @@ -0,0 +1,54 @@ +/* + * Copyright 2018 the original author or authors. + * + * 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 io.lettuce.core.cluster.models.partitions; + +import java.util.concurrent.TimeUnit; + +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.ChainedOptionsBuilder; +import org.openjdk.jmh.runner.options.OptionsBuilder; +import org.openjdk.jmh.runner.options.TimeValue; + +/** + * Manual JMH Test Launcher. + * + * @author Mark Paluch + */ +public class JmhMain { + + public static void main(String... args) throws Exception { + runClusterNodeBenchmark(); + } + + private static void runClusterNodeBenchmark() throws RunnerException { + + new Runner(prepareOptions().mode(Mode.AverageTime) // + .timeUnit(TimeUnit.NANOSECONDS) // + .include(".*RedisClusterNodeBenchmark.*") // + .build()).run(); + } + + private static ChainedOptionsBuilder prepareOptions() { + return new OptionsBuilder()// + .forks(1) // + .warmupIterations(5)// + .threads(1) // + .measurementIterations(5) // + .timeout(TimeValue.seconds(2)); + } +} diff --git a/src/test/jmh/io/lettuce/core/cluster/models/partitions/RedisClusterNodeBenchmark.java b/src/test/jmh/io/lettuce/core/cluster/models/partitions/RedisClusterNodeBenchmark.java new file mode 100644 index 0000000000..67c94f6dad --- /dev/null +++ b/src/test/jmh/io/lettuce/core/cluster/models/partitions/RedisClusterNodeBenchmark.java @@ -0,0 +1,60 @@ +/* + * Copyright 2018 the original author or authors. + * + * 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 io.lettuce.core.cluster.models.partitions; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; + +import io.lettuce.core.cluster.SlotHash; + +/** + * @author Mark Paluch + */ +@State(Scope.Benchmark) +public class RedisClusterNodeBenchmark { + + final static List ALL_SLOTS = IntStream.range(0, SlotHash.SLOT_COUNT).boxed().collect(Collectors.toList()); + final static List LOWER_SLOTS = IntStream.range(0, 8192).boxed().collect(Collectors.toList()); + + final static RedisClusterNode NODE = new RedisClusterNode(null, null, true, null, 0, 0, 0, ALL_SLOTS, + Collections.emptySet()); + + @Benchmark + public RedisClusterNode createClusterNodeAllSlots() { + return new RedisClusterNode(null, null, true, null, 0, 0, 0, ALL_SLOTS, Collections.emptySet()); + } + + @Benchmark + public RedisClusterNode createClusterNodeLowerSlots() { + return new RedisClusterNode(null, null, true, null, 0, 0, 0, LOWER_SLOTS, Collections.emptySet()); + } + + @Benchmark + public void querySlotStatusPresent() { + NODE.hasSlot(1234); + } + + @Benchmark + public void querySlotStatusAbsent() { + NODE.hasSlot(8193); + } +}