Skip to content

Commit

Permalink
Merge branch 'cassandra-3.11' into cassandra-4.0
Browse files Browse the repository at this point in the history
  • Loading branch information
smiklosovic committed May 17, 2023
2 parents b791644 + f5c42f4 commit 2bb634a
Show file tree
Hide file tree
Showing 3 changed files with 84 additions and 7 deletions.
1 change: 1 addition & 0 deletions CHANGES.txt
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@
Merged from 3.11:
* Fix the capital P usage in the CQL parser (CASSANDRA-17919)
Merged from 3.0:
* Validate the existence of a datacenter in nodetool rebuild (CASSANDRA-14319)
* Suppress CVE-2023-2251 (CASSANDRA-18497)
* Do not remove SSTables when cause of FSReadError is OutOfMemoryError while using best_effort disk failure policy (CASSANDRA-18336)
* Do not remove truncated_at entry in system.local while dropping an index (CASSANDRA-18105)
Expand Down
33 changes: 26 additions & 7 deletions src/java/org/apache/cassandra/service/StorageService.java
Original file line number Diff line number Diff line change
Expand Up @@ -1327,16 +1327,35 @@ public void rebuild(String sourceDc)

public void rebuild(String sourceDc, String keyspace, String tokens, String specificSources)
{
// check ongoing rebuild
if (!isRebuilding.compareAndSet(false, true))
try
{
throw new IllegalStateException("Node is still rebuilding. Check nodetool netstats.");
}
// check ongoing rebuild
if (!isRebuilding.compareAndSet(false, true))
{
throw new IllegalStateException("Node is still rebuilding. Check nodetool netstats.");
}

if (sourceDc != null)
{
TokenMetadata.Topology topology = getTokenMetadata().cloneOnlyTokenMap().getTopology();
Set<String> availableDCs = topology.getDatacenterEndpoints().keySet();
if (!availableDCs.contains(sourceDc))
{
throw new IllegalArgumentException(String.format("Provided datacenter '%s' is not a valid datacenter, available datacenters are: %s",
sourceDc, String.join(",", availableDCs)));
}
}

// check the arguments
if (keyspace == null && tokens != null)
// check the arguments
if (keyspace == null && tokens != null)
{
throw new IllegalArgumentException("Cannot specify tokens without keyspace.");
}
}
catch (Throwable ex)
{
throw new IllegalArgumentException("Cannot specify tokens without keyspace.");
isRebuilding.set(false);
throw ex;
}

logger.info("rebuild from dc: {}, {}, {}", sourceDc == null ? "(any dc)" : sourceDc,
Expand Down
57 changes: 57 additions & 0 deletions test/unit/org/apache/cassandra/service/StorageServiceTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@

import org.apache.cassandra.locator.EndpointsByReplica;
import org.apache.cassandra.locator.ReplicaCollection;

import com.google.common.collect.ImmutableMultimap;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
Expand All @@ -34,8 +36,10 @@
import org.apache.cassandra.locator.InetAddressAndPort;
import org.apache.cassandra.locator.Replica;
import org.apache.cassandra.locator.ReplicaMultimap;
import org.apache.cassandra.locator.SimpleSnitch;
import org.apache.cassandra.locator.SimpleStrategy;
import org.apache.cassandra.locator.TokenMetadata;
import org.mockito.Mockito;

import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
Expand Down Expand Up @@ -157,4 +161,57 @@ public void testGetChangedReplicasForLeaving() throws Exception
expectedResult.put(new Replica(aAddress, dRange, false), new Replica(bAddress, dRange, false));
assertMultimapEqualsIgnoreOrder(result, expectedResult.build());
}

@Test
public void testRebuildFailOnNonExistingDatacenter()
{
String nonExistentDC = "NON_EXISTENT_DC";

try
{
getStorageService().rebuild(nonExistentDC, "StorageServiceTest", null, null);
fail();
}
catch (IllegalArgumentException ex)
{
assertEquals(String.format("Provided datacenter '%s' is not a valid datacenter, available datacenters are: %s",
nonExistentDC,
SimpleSnitch.DATA_CENTER_NAME),
ex.getMessage());
}
}

@Test
public void testRebuildingWithTokensWithoutKeyspace() throws Exception
{
try
{
getStorageService().rebuild("datacenter1", null, "123", null);
fail();
}
catch (IllegalArgumentException ex)
{
assertEquals("Cannot specify tokens without keyspace.", ex.getMessage());
}
}

private StorageService getStorageService()
{
ImmutableMultimap.Builder<String, InetAddressAndPort> builder = ImmutableMultimap.builder();
builder.put(SimpleSnitch.DATA_CENTER_NAME, aAddress);

TokenMetadata.Topology tokenMetadataTopology = Mockito.mock(TokenMetadata.Topology.class);
Mockito.when(tokenMetadataTopology.getDatacenterEndpoints()).thenReturn(builder.build());

TokenMetadata metadata = new TokenMetadata(new SimpleSnitch());
TokenMetadata spiedMetadata = Mockito.spy(metadata);

Mockito.when(spiedMetadata.getTopology()).thenReturn(tokenMetadataTopology);

StorageService spiedStorageService = Mockito.spy(StorageService.instance);
Mockito.when(spiedStorageService.getTokenMetadata()).thenReturn(spiedMetadata);
Mockito.when(spiedMetadata.cloneOnlyTokenMap()).thenReturn(spiedMetadata);

return spiedStorageService;
}
}

0 comments on commit 2bb634a

Please sign in to comment.