Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id C3FF018805 for ; Fri, 27 Nov 2015 12:58:40 +0000 (UTC) Received: (qmail 39250 invoked by uid 500); 27 Nov 2015 12:58:40 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 39097 invoked by uid 500); 27 Nov 2015 12:58:40 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 39072 invoked by uid 99); 27 Nov 2015 12:58:40 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 27 Nov 2015 12:58:40 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 59E3EE041F; Fri, 27 Nov 2015 12:58:40 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: snazy@apache.org To: commits@cassandra.apache.org Date: Fri, 27 Nov 2015 12:58:41 -0000 Message-Id: In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [2/5] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2 Merge branch 'cassandra-2.1' into cassandra-2.2 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/432a8a48 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/432a8a48 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/432a8a48 Branch: refs/heads/trunk Commit: 432a8a484394f6774fc64486229939c07f8af9e2 Parents: e7116f2 7650fc1 Author: Robert Stupp Authored: Fri Nov 27 13:50:52 2015 +0100 Committer: Robert Stupp Committed: Fri Nov 27 13:50:52 2015 +0100 ---------------------------------------------------------------------- CHANGES.txt | 1 + NEWS.txt | 6 + src/java/org/apache/cassandra/gms/Gossiper.java | 28 +- .../locator/GossipingPropertyFileSnitch.java | 107 ++---- .../cassandra/locator/PropertyFileSnitch.java | 74 +++- .../cassandra/locator/SnitchProperties.java | 5 + .../cassandra/service/CassandraDaemon.java | 17 +- .../cassandra/service/MigrationManager.java | 4 +- .../apache/cassandra/service/StorageProxy.java | 10 +- .../cassandra/service/StorageService.java | 30 +- .../GossipingPropertyFileSnitchTest.java | 38 +-- .../locator/PropertyFileSnitchTest.java | 334 +++++++++++++++++++ 12 files changed, 496 insertions(+), 158 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/432a8a48/CHANGES.txt ---------------------------------------------------------------------- diff --cc CHANGES.txt index 7281bd3,a2f7b6e..ce82bd0 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,21 -1,5 +1,22 @@@ -2.1.12 +2.2.4 + * Serialize on disk the proper SSTable compression ratio (CASSANDRA-10775) + * Reject index queries while the index is building (CASSANDRA-8505) + * CQL.textile syntax incorrectly includes optional keyspace for aggregate SFUNC and FINALFUNC (CASSANDRA-10747) + * Fix JSON update with prepared statements (CASSANDRA-10631) + * Don't do anticompaction after subrange repair (CASSANDRA-10422) + * Fix SimpleDateType type compatibility (CASSANDRA-10027) + * (Hadoop) fix splits calculation (CASSANDRA-10640) + * (Hadoop) ensure that Cluster instances are always closed (CASSANDRA-10058) + * (cqlsh) show partial trace if incomplete after max_trace_wait (CASSANDRA-7645) + * Use most up-to-date version of schema for system tables (CASSANDRA-10652) + * Deprecate memory_allocator in cassandra.yaml (CASSANDRA-10581,10628) + * Expose phi values from failure detector via JMX and tweak debug + and trace logging (CASSANDRA-9526) + * Fix RangeNamesQueryPager (CASSANDRA-10509) + * Deprecate Pig support (CASSANDRA-10542) + * Reduce contention getting instances of CompositeType (CASSANDRA-10433) +Merged from 2.1: + * Warn or fail when changing cluster topology live (CASSANDRA-10243) * Status command in debian/ubuntu init script doesn't work (CASSANDRA-10213) * Some DROP ... IF EXISTS incorrectly result in exceptions on non-existing KS (CASSANDRA-10658) * DeletionTime.compareTo wrong in rare cases (CASSANDRA-10749) http://git-wip-us.apache.org/repos/asf/cassandra/blob/432a8a48/NEWS.txt ---------------------------------------------------------------------- diff --cc NEWS.txt index 4cf9c7b,cae8dfb..87e77f4 --- a/NEWS.txt +++ b/NEWS.txt @@@ -39,9 -24,22 +39,15 @@@ New feature - DTCS option max_sstable_age_days is now deprecated and defaults to 1000 days. - Native protocol server now allows both SSL and non-SSL connections on the same port. - - Switching racks is no longer an allowed operation on a node which has - data. Instead, the node will need to be decommissioned and rebootstrapped. - If moving from the SimpleSnitch, make sure the rack containing all current - nodes is named "rack1". To override this behavior when manually wiping - the node and bootstrapping, use -Dcassandra.ignore_rack=true. - - a new validate(key, cf) method is added to PerRowSecondaryIndex. A default - implementation is provided, so no changes are required to custom implementations. + Operations + ------------ + - Changing rack or dc of live nodes is no longer possible for PropertyFileSnitch + and YamlFileNetworkTopologySnitch. Reloading the configuration file of + GossipingPropertyFileSnitch has been disabled, CASSANDRA-10243. + -2.1.11 +2.2.3 ===== Upgrading http://git-wip-us.apache.org/repos/asf/cassandra/blob/432a8a48/src/java/org/apache/cassandra/gms/Gossiper.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/gms/Gossiper.java index 3d0d5fa,09851b2..fadaffc --- a/src/java/org/apache/cassandra/gms/Gossiper.java +++ b/src/java/org/apache/cassandra/gms/Gossiper.java @@@ -981,10 -984,11 +970,11 @@@ public class Gossiper implements IFailu for (IEndpointStateChangeSubscriber subscriber : subscribers) subscriber.onAlive(addr, localState); if (logger.isTraceEnabled()) - logger.trace("Notified " + subscribers); + logger.trace("Notified {}", subscribers); } - private void markDead(InetAddress addr, EndpointState localState) + @VisibleForTesting + public void markDead(InetAddress addr, EndpointState localState) { if (logger.isTraceEnabled()) logger.trace("marking as down {}", addr); http://git-wip-us.apache.org/repos/asf/cassandra/blob/432a8a48/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/locator/PropertyFileSnitch.java index 590117d,6115572..8843b77 --- a/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java +++ b/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java @@@ -133,11 -142,14 +143,12 @@@ public class PropertyFileSnitch extend public void reloadConfiguration(boolean isUpdate) throws ConfigurationException { - HashMap reloadedMap = new HashMap(); + HashMap reloadedMap = new HashMap<>(); + String[] reloadedDefaultDCRack = null; Properties properties = new Properties(); - InputStream stream = null; - try + try (InputStream stream = getClass().getClassLoader().getResourceAsStream(SNITCH_PROPERTIES_FILENAME)) { - stream = getClass().getClassLoader().getResourceAsStream(SNITCH_PROPERTIES_FILENAME); properties.load(stream); } catch (Exception e) @@@ -178,18 -194,24 +189,24 @@@ reloadedMap.put(host, token); } } - if (defaultDCRack == null && !reloadedMap.containsKey(FBUtilities.getBroadcastAddress())) - throw new ConfigurationException(String.format("Snitch definitions at %s do not define a location for this node's broadcast address %s, nor does it provides a default", + if (reloadedDefaultDCRack == null && !reloadedMap.containsKey(FBUtilities.getBroadcastAddress())) + throw new ConfigurationException(String.format("Snitch definitions at %s do not define a location for " + + "this node's broadcast address %s, nor does it provides a default", SNITCH_PROPERTIES_FILENAME, FBUtilities.getBroadcastAddress())); + if (isUpdate && !livenessCheck(reloadedMap, reloadedDefaultDCRack)) + return; + - if (logger.isDebugEnabled()) + if (logger.isTraceEnabled()) { StringBuilder sb = new StringBuilder(); for (Map.Entry entry : reloadedMap.entrySet()) - sb.append(entry.getKey()).append(":").append(Arrays.toString(entry.getValue())).append(", "); + sb.append(entry.getKey()).append(':').append(Arrays.toString(entry.getValue())).append(", "); - logger.debug("Loaded network topology from property file: {}", StringUtils.removeEnd(sb.toString(), ", ")); + logger.trace("Loaded network topology from property file: {}", StringUtils.removeEnd(sb.toString(), ", ")); } + + defaultDCRack = reloadedDefaultDCRack; endpointMap = reloadedMap; if (StorageService.instance != null) // null check tolerates circular dependency; see CASSANDRA-4145 { http://git-wip-us.apache.org/repos/asf/cassandra/blob/432a8a48/src/java/org/apache/cassandra/service/CassandraDaemon.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/service/CassandraDaemon.java index 45e1812,17553f3..59609a3 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@@ -231,19 -319,6 +231,22 @@@ public class CassandraDaemo } Keyspace.setInitialized(); + - String storedRack = SystemKeyspace.getRack(); - if (storedRack != null) ++ if (!Boolean.getBoolean("cassandra.ignore_rack")) + { - String currentRack = DatabaseDescriptor.getEndpointSnitch().getRack(FBUtilities.getBroadcastAddress()); - if (!storedRack.equals(currentRack)) ++ String storedRack = SystemKeyspace.getRack(); ++ if (storedRack != null) + { - logger.error("Cannot start node if snitch's rack differs from previous rack. " + - "Please fix the snitch or decommission and rebootstrap this node."); - System.exit(100); ++ String currentRack = DatabaseDescriptor.getEndpointSnitch().getRack(FBUtilities.getBroadcastAddress()); ++ if (!storedRack.equals(currentRack)) ++ { ++ logger.error("Cannot start node if snitch's rack differs from previous rack. " + ++ "Please fix the snitch or decommission and rebootstrap this node."); ++ System.exit(100); ++ } + } + } + // initialize keyspaces for (String keyspaceName : Schema.instance.getKeyspaces()) { http://git-wip-us.apache.org/repos/asf/cassandra/blob/432a8a48/src/java/org/apache/cassandra/service/MigrationManager.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/432a8a48/src/java/org/apache/cassandra/service/StorageProxy.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/432a8a48/src/java/org/apache/cassandra/service/StorageService.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/432a8a48/test/unit/org/apache/cassandra/locator/GossipingPropertyFileSnitchTest.java ---------------------------------------------------------------------- diff --cc test/unit/org/apache/cassandra/locator/GossipingPropertyFileSnitchTest.java index 16557b3,80d4559..20608ba --- a/test/unit/org/apache/cassandra/locator/GossipingPropertyFileSnitchTest.java +++ b/test/unit/org/apache/cassandra/locator/GossipingPropertyFileSnitchTest.java @@@ -17,55 -17,22 +17,33 @@@ */ package org.apache.cassandra.locator; +import java.net.InetAddress; - import java.nio.file.Files; - import java.nio.file.Path; - import java.nio.file.Paths; + +import com.google.common.net.InetAddresses; - import org.apache.cassandra.utils.FBUtilities; - import org.junit.Assert; import org.junit.Test; + import org.apache.cassandra.utils.FBUtilities; + ++import static org.junit.Assert.*; ++ /** * Unit tests for {@link GossipingPropertyFileSnitch}. */ public class GossipingPropertyFileSnitchTest { + public static void checkEndpoint(final AbstractNetworkTopologySnitch snitch, + final String endpointString, final String expectedDatacenter, + final String expectedRack) + { + final InetAddress endpoint = InetAddresses.forString(endpointString); - Assert.assertEquals(expectedDatacenter, snitch.getDatacenter(endpoint)); - Assert.assertEquals(expectedRack, snitch.getRack(endpoint)); ++ assertEquals(expectedDatacenter, snitch.getDatacenter(endpoint)); ++ assertEquals(expectedRack, snitch.getRack(endpoint)); + } + @Test - public void testAutoReloadConfig() throws Exception + public void testLoadConfig() throws Exception { - String confFile = FBUtilities.resourceToFile(SnitchProperties.RACKDC_PROPERTY_FILENAME); - - final GossipingPropertyFileSnitch snitch = new GossipingPropertyFileSnitch(/*refreshPeriodInSeconds*/1); + final GossipingPropertyFileSnitch snitch = new GossipingPropertyFileSnitch(); - YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, - FBUtilities.getBroadcastAddress().getHostAddress(), - "DC1", - "RAC1"); + checkEndpoint(snitch, FBUtilities.getBroadcastAddress().getHostAddress(), "DC1", "RAC1"); - - final Path effectiveFile = Paths.get(confFile); - final Path backupFile = Paths.get(confFile + ".bak"); - final Path modifiedFile = Paths.get(confFile + ".mod"); - - try - { - Files.copy(effectiveFile, backupFile); - Files.copy(modifiedFile, effectiveFile, java.nio.file.StandardCopyOption.REPLACE_EXISTING); - - Thread.sleep(1500); - - checkEndpoint(snitch, FBUtilities.getBroadcastAddress().getHostAddress(), "DC2", "RAC2"); - } - finally - { - Files.copy(backupFile, effectiveFile, java.nio.file.StandardCopyOption.REPLACE_EXISTING); - Files.delete(backupFile); - } } } http://git-wip-us.apache.org/repos/asf/cassandra/blob/432a8a48/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java ---------------------------------------------------------------------- diff --cc test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java index 0000000,24b8c77..e9a307b mode 000000,100644..100644 --- a/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java +++ b/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java @@@ -1,0 -1,321 +1,334 @@@ + /* + * 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.nio.charset.StandardCharsets; + import java.nio.file.Files; + import java.nio.file.Path; + import java.nio.file.Paths; + import java.nio.file.StandardOpenOption; + import java.util.ArrayList; + import java.util.Collections; + import java.util.HashMap; + import java.util.HashSet; + import java.util.List; + import java.util.Map; + import java.util.Set; + import java.util.UUID; + ++import com.google.common.net.InetAddresses; ++ + import org.apache.cassandra.dht.IPartitioner; + import org.apache.cassandra.dht.RandomPartitioner; + import org.apache.cassandra.dht.Token; + import org.apache.cassandra.exceptions.ConfigurationException; + import org.apache.cassandra.gms.ApplicationState; + import org.apache.cassandra.gms.Gossiper; + import org.apache.cassandra.gms.VersionedValue; + import org.apache.cassandra.service.StorageService; + import org.apache.cassandra.utils.FBUtilities; + + import org.junit.Before; + import org.junit.Test; + ++import static org.junit.Assert.*; ++ + /** + * Unit tests for {@link PropertyFileSnitch}. + */ + public class PropertyFileSnitchTest + { + private Path effectiveFile; + private Path backupFile; + + private VersionedValue.VersionedValueFactory valueFactory; + private Map> tokenMap; + + @Before + public void setup() throws ConfigurationException, IOException + { + String confFile = FBUtilities.resourceToFile(PropertyFileSnitch.SNITCH_PROPERTIES_FILENAME); + effectiveFile = Paths.get(confFile); + backupFile = Paths.get(confFile + ".bak"); + + restoreOrigConfigFile(); + + InetAddress[] hosts = { + InetAddress.getByName("127.0.0.1"), // this exists in the config file + InetAddress.getByName("127.0.0.2"), // this exists in the config file + InetAddress.getByName("127.0.0.9"), // this does not exist in the config file + }; + + IPartitioner partitioner = new RandomPartitioner(); + valueFactory = new VersionedValue.VersionedValueFactory(partitioner); + tokenMap = new HashMap<>(); + + for (InetAddress host : hosts) + { + Set tokens = Collections.singleton(partitioner.getRandomToken()); + Gossiper.instance.initializeNodeUnsafe(host, UUID.randomUUID(), 1); + Gossiper.instance.injectApplicationState(host, ApplicationState.TOKENS, valueFactory.tokens(tokens)); + + setNodeShutdown(host); + tokenMap.put(host, tokens); + } + } + + private void restoreOrigConfigFile() throws IOException + { + if (Files.exists(backupFile)) + { + Files.copy(backupFile, effectiveFile, java.nio.file.StandardCopyOption.REPLACE_EXISTING); + Files.delete(backupFile); + } + } + + private void replaceConfigFile(Map replacements) throws IOException + { + List lines = Files.readAllLines(effectiveFile, StandardCharsets.UTF_8); + List newLines = new ArrayList<>(lines.size()); + Set replaced = new HashSet<>(); + + for (String line : lines) + { + String[] info = line.split("="); + if (info.length == 2 && replacements.containsKey(info[0])) + { + String replacement = replacements.get(info[0]); + if (!replacement.isEmpty()) // empty means remove this line + newLines.add(info[0] + '=' + replacement); + + replaced.add(info[0]); + } + else + { + newLines.add(line); + } + } + + // add any new lines that were not replaced + for (Map.Entry replacement : replacements.entrySet()) + { + if (replaced.contains(replacement.getKey())) + continue; + + if (!replacement.getValue().isEmpty()) // empty means remove this line so do nothing here + newLines.add(replacement.getKey() + '=' + replacement.getValue()); + } + + Files.write(effectiveFile, newLines, StandardCharsets.UTF_8, StandardOpenOption.TRUNCATE_EXISTING); + } + + private void setNodeShutdown(InetAddress host) + { + StorageService.instance.getTokenMetadata().removeEndpoint(host); + Gossiper.instance.injectApplicationState(host, ApplicationState.STATUS, valueFactory.shutdown(true)); + Gossiper.instance.markDead(host, Gossiper.instance.getEndpointStateForEndpoint(host)); + } + + private void setNodeLive(InetAddress host) + { + Gossiper.instance.injectApplicationState(host, ApplicationState.STATUS, valueFactory.normal(tokenMap.get(host))); + Gossiper.instance.realMarkAlive(host, Gossiper.instance.getEndpointStateForEndpoint(host)); + StorageService.instance.getTokenMetadata().updateNormalTokens(tokenMap.get(host), host); + } + ++ private static void checkEndpoint(final AbstractNetworkTopologySnitch snitch, ++ final String endpointString, final String expectedDatacenter, ++ final String expectedRack) ++ { ++ final InetAddress endpoint = InetAddresses.forString(endpointString); ++ assertEquals(expectedDatacenter, snitch.getDatacenter(endpoint)); ++ assertEquals(expectedRack, snitch.getRack(endpoint)); ++ } ++ + /** + * Test that changing rack for a host in the configuration file is only effective if the host is not live. + * The original configuration file contains: 127.0.0.1=DC1:RAC1 + */ + @Test + public void testChangeHostRack() throws Exception + { + final InetAddress host = InetAddress.getByName("127.0.0.1"); + final PropertyFileSnitch snitch = new PropertyFileSnitch(/*refreshPeriodInSeconds*/1); - YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1"); ++ checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1"); + + try + { + setNodeLive(host); + + Files.copy(effectiveFile, backupFile); + replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "DC1:RAC2")); + + Thread.sleep(1500); - YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1"); ++ checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1"); + + setNodeShutdown(host); + replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "DC1:RAC2")); + + Thread.sleep(1500); - YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC2"); ++ checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC2"); + } + finally + { + restoreOrigConfigFile(); + setNodeShutdown(host); + } + } + + /** + * Test that changing dc for a host in the configuration file is only effective if the host is not live. + * The original configuration file contains: 127.0.0.1=DC1:RAC1 + */ + @Test + public void testChangeHostDc() throws Exception + { + final InetAddress host = InetAddress.getByName("127.0.0.1"); + final PropertyFileSnitch snitch = new PropertyFileSnitch(/*refreshPeriodInSeconds*/1); - YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1"); ++ checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1"); + + try + { + setNodeLive(host); + + Files.copy(effectiveFile, backupFile); + replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "DC2:RAC1")); + + Thread.sleep(1500); - YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1"); ++ checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1"); + + setNodeShutdown(host); + replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "DC2:RAC1")); + + Thread.sleep(1500); - YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC2", "RAC1"); ++ checkEndpoint(snitch, host.getHostAddress(), "DC2", "RAC1"); + } + finally + { + restoreOrigConfigFile(); + setNodeShutdown(host); + } + } + + /** + * Test that adding a host to the configuration file changes the host dc and rack only if the host + * is not live. The original configuration file does not contain 127.0.0.9 and so it should use + * the default default=DC1:r1. + */ + @Test + public void testAddHost() throws Exception + { + final InetAddress host = InetAddress.getByName("127.0.0.9"); + final PropertyFileSnitch snitch = new PropertyFileSnitch(/*refreshPeriodInSeconds*/1); - YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // default ++ checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // default + + try + { + setNodeLive(host); + + Files.copy(effectiveFile, backupFile); + replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "DC2:RAC2")); // add this line if not yet there + + Thread.sleep(1500); - YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // unchanged ++ checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // unchanged + + setNodeShutdown(host); + replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "DC2:RAC2")); // add this line if not yet there + + Thread.sleep(1500); - YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC2", "RAC2"); // changed ++ checkEndpoint(snitch, host.getHostAddress(), "DC2", "RAC2"); // changed + } + finally + { + restoreOrigConfigFile(); + setNodeShutdown(host); + } + } + + /** + * Test that removing a host from the configuration file changes the host rack only if the host + * is not live. The original configuration file contains 127.0.0.2=DC1:RAC2 and default=DC1:r1 so removing + * this host should result in a different rack if the host is not live. + */ + @Test + public void testRemoveHost() throws Exception + { + final InetAddress host = InetAddress.getByName("127.0.0.2"); + final PropertyFileSnitch snitch = new PropertyFileSnitch(/*refreshPeriodInSeconds*/1); - YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC2"); ++ checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC2"); + + try + { + setNodeLive(host); + + Files.copy(effectiveFile, backupFile); + replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "")); // removes line if found + + Thread.sleep(1500); - YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC2"); // unchanged ++ checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC2"); // unchanged + + setNodeShutdown(host); + replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "")); // removes line if found + + Thread.sleep(1500); - YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // default ++ checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // default + } + finally + { + restoreOrigConfigFile(); + setNodeShutdown(host); + } + } + + /** + * Test that we can change the default only if this does not result in any live node changing dc or rack. + * The configuration file contains default=DC1:r1 and we change it to default=DC2:r2. Let's use host 127.0.0.9 + * since it is not in the configuration file. + */ + @Test + public void testChangeDefault() throws Exception + { + final InetAddress host = InetAddress.getByName("127.0.0.9"); + final PropertyFileSnitch snitch = new PropertyFileSnitch(/*refreshPeriodInSeconds*/1); - YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // default ++ checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // default + + try + { + setNodeLive(host); + + Files.copy(effectiveFile, backupFile); + replaceConfigFile(Collections.singletonMap("default", "DC2:r2")); // change default + + Thread.sleep(1500); - YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // unchanged ++ checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // unchanged + + setNodeShutdown(host); + replaceConfigFile(Collections.singletonMap("default", "DC2:r2")); // change default again (refresh file update) + + Thread.sleep(1500); - YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC2", "r2"); // default updated ++ checkEndpoint(snitch, host.getHostAddress(), "DC2", "r2"); // default updated + } + finally + { + restoreOrigConfigFile(); + setNodeShutdown(host); + } + } + }