blob: bbfdd3b208aea2418bce0ef38b9f6ac4ad231cf1 [file] [log] [blame]
/*
* 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.cassandra.locator;
import java.io.IOException;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import com.google.common.collect.HashMultimap;
import com.google.common.collect.Multimap;
import org.junit.Assert;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.dht.OrderPreservingPartitioner.StringToken;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.exceptions.ConfigurationException;
public class NetworkTopologyStrategyTest
{
private String keyspaceName = "Keyspace1";
private static final Logger logger = LoggerFactory.getLogger(NetworkTopologyStrategyTest.class);
@Test
public void testProperties() throws IOException, ConfigurationException
{
IEndpointSnitch snitch = new PropertyFileSnitch();
DatabaseDescriptor.setEndpointSnitch(snitch);
TokenMetadata metadata = new TokenMetadata();
createDummyTokens(metadata, true);
Map<String, String> configOptions = new HashMap<String, String>();
configOptions.put("DC1", "3");
configOptions.put("DC2", "2");
configOptions.put("DC3", "1");
// Set the localhost to the tokenmetadata. Embedded cassandra way?
NetworkTopologyStrategy strategy = new NetworkTopologyStrategy(keyspaceName, metadata, snitch, configOptions);
assert strategy.getReplicationFactor("DC1") == 3;
assert strategy.getReplicationFactor("DC2") == 2;
assert strategy.getReplicationFactor("DC3") == 1;
// Query for the natural hosts
ArrayList<InetAddress> endpoints = strategy.getNaturalEndpoints(new StringToken("123"));
assert 6 == endpoints.size();
assert 6 == new HashSet<InetAddress>(endpoints).size(); // ensure uniqueness
}
@Test
public void testPropertiesWithEmptyDC() throws IOException, ConfigurationException
{
IEndpointSnitch snitch = new PropertyFileSnitch();
DatabaseDescriptor.setEndpointSnitch(snitch);
TokenMetadata metadata = new TokenMetadata();
createDummyTokens(metadata, false);
Map<String, String> configOptions = new HashMap<String, String>();
configOptions.put("DC1", "3");
configOptions.put("DC2", "3");
configOptions.put("DC3", "0");
// Set the localhost to the tokenmetadata. Embedded cassandra way?
NetworkTopologyStrategy strategy = new NetworkTopologyStrategy(keyspaceName, metadata, snitch, configOptions);
assert strategy.getReplicationFactor("DC1") == 3;
assert strategy.getReplicationFactor("DC2") == 3;
assert strategy.getReplicationFactor("DC3") == 0;
// Query for the natural hosts
ArrayList<InetAddress> endpoints = strategy.getNaturalEndpoints(new StringToken("123"));
assert 6 == endpoints.size();
assert 6 == new HashSet<InetAddress>(endpoints).size(); // ensure uniqueness
}
@Test
public void testLargeCluster() throws UnknownHostException, ConfigurationException
{
int[] dcRacks = new int[]{2, 4, 8};
int[] dcEndpoints = new int[]{128, 256, 512};
int[] dcReplication = new int[]{2, 6, 6};
IEndpointSnitch snitch = new RackInferringSnitch();
DatabaseDescriptor.setEndpointSnitch(snitch);
TokenMetadata metadata = new TokenMetadata();
Map<String, String> configOptions = new HashMap<String, String>();
Multimap<InetAddress, Token> tokens = HashMultimap.create();
int totalRF = 0;
for (int dc = 0; dc < dcRacks.length; ++dc)
{
totalRF += dcReplication[dc];
configOptions.put(Integer.toString(dc), Integer.toString(dcReplication[dc]));
for (int rack = 0; rack < dcRacks[dc]; ++rack)
{
for (int ep = 1; ep <= dcEndpoints[dc]/dcRacks[dc]; ++ep)
{
byte[] ipBytes = new byte[]{10, (byte)dc, (byte)rack, (byte)ep};
InetAddress address = InetAddress.getByAddress(ipBytes);
StringToken token = new StringToken(String.format("%02x%02x%02x", ep, rack, dc));
logger.debug("adding node {} at {}", address, token);
tokens.put(address, token);
}
}
}
metadata.updateNormalTokens(tokens);
NetworkTopologyStrategy strategy = new NetworkTopologyStrategy(keyspaceName, metadata, snitch, configOptions);
for (String testToken : new String[]{"123456", "200000", "000402", "ffffff", "400200"})
{
List<InetAddress> endpoints = strategy.calculateNaturalEndpoints(new StringToken(testToken), metadata);
Set<InetAddress> epSet = new HashSet<InetAddress>(endpoints);
Assert.assertEquals(totalRF, endpoints.size());
Assert.assertEquals(totalRF, epSet.size());
logger.debug("{}: {}", testToken, endpoints);
}
}
public void createDummyTokens(TokenMetadata metadata, boolean populateDC3) throws UnknownHostException
{
// DC 1
tokenFactory(metadata, "123", new byte[]{ 10, 0, 0, 10 });
tokenFactory(metadata, "234", new byte[]{ 10, 0, 0, 11 });
tokenFactory(metadata, "345", new byte[]{ 10, 0, 0, 12 });
// Tokens for DC 2
tokenFactory(metadata, "789", new byte[]{ 10, 20, 114, 10 });
tokenFactory(metadata, "890", new byte[]{ 10, 20, 114, 11 });
//tokens for DC3
if (populateDC3)
{
tokenFactory(metadata, "456", new byte[]{ 10, 21, 119, 13 });
tokenFactory(metadata, "567", new byte[]{ 10, 21, 119, 10 });
}
// Extra Tokens
tokenFactory(metadata, "90A", new byte[]{ 10, 0, 0, 13 });
if (populateDC3)
tokenFactory(metadata, "0AB", new byte[]{ 10, 21, 119, 14 });
tokenFactory(metadata, "ABC", new byte[]{ 10, 20, 114, 15 });
}
public void tokenFactory(TokenMetadata metadata, String token, byte[] bytes) throws UnknownHostException
{
Token token1 = new StringToken(token);
InetAddress add1 = InetAddress.getByAddress(bytes);
metadata.updateNormalToken(token1, add1);
}
}