Skip to content

Commit

Permalink
Add replace_address_first_boot to yaml config
Browse files Browse the repository at this point in the history
It's more convenient to set it there in Astacus. Option semantics is
preserved, the value specified in Java properties has priority.
  • Loading branch information
dmitry-potepalov authored and meatlink committed Aug 2, 2023
1 parent ba33662 commit fbf6514
Show file tree
Hide file tree
Showing 3 changed files with 9 additions and 1 deletion.
1 change: 1 addition & 0 deletions src/java/org/apache/cassandra/config/Config.java
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,7 @@ public class Config
public Integer allocate_tokens_for_local_replication_factor = null;

public boolean skip_bootstrap_streaming = false;
public String replace_address_first_boot = null;

public long native_transport_idle_timeout_in_ms = 0L;

Expand Down
7 changes: 7 additions & 0 deletions src/java/org/apache/cassandra/config/DatabaseDescriptor.java
Original file line number Diff line number Diff line change
Expand Up @@ -1571,6 +1571,8 @@ public static InetAddressAndPort getReplaceAddress()
return InetAddressAndPort.getByName(System.getProperty(Config.PROPERTY_PREFIX + "replace_address", null));
else if (System.getProperty(Config.PROPERTY_PREFIX + "replace_address_first_boot", null) != null)
return InetAddressAndPort.getByName(System.getProperty(Config.PROPERTY_PREFIX + "replace_address_first_boot", null));
else if (conf.replace_address_first_boot != null)
return InetAddressAndPort.getByName(conf.replace_address_first_boot);
return null;
}
catch (UnknownHostException e)
Expand All @@ -1584,6 +1586,11 @@ public static boolean skipBootstrapStreaming()
return conf.skip_bootstrap_streaming;
}

public static boolean replaceOnFirstBootRequested()
{
return System.getProperty("cassandra.replace_address_first_boot", null) != null || conf.replace_address_first_boot != null;
}

public static Collection<String> getReplaceTokens()
{
return tokensFromString(System.getProperty(Config.PROPERTY_PREFIX + "replace_token", null));
Expand Down
2 changes: 1 addition & 1 deletion src/java/org/apache/cassandra/service/StorageService.java
Original file line number Diff line number Diff line change
Expand Up @@ -931,7 +931,7 @@ private void populatePeerTokenMetadata()

private boolean isReplacing()
{
if (System.getProperty("cassandra.replace_address_first_boot", null) != null && SystemKeyspace.bootstrapComplete())
if (DatabaseDescriptor.replaceOnFirstBootRequested() && SystemKeyspace.bootstrapComplete())
{
logger.info("Replace address on first boot requested; this node is already bootstrapped");
return false;
Expand Down

0 comments on commit fbf6514

Please sign in to comment.