This is an automated email from the ASF dual-hosted git repository.

mck pushed a commit to branch cassandra-3.11
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit d9e1af8824f6914a6eff99a6345575b93188c51a
Merge: 50d8245 8ef5a88
Author: Mick Semb Wever <[email protected]>
AuthorDate: Tue Nov 17 22:42:06 2020 +0100

    Merge branch 'cassandra-3.0' into cassandra-3.11

 CHANGES.txt                                        |  1 +
 NEWS.txt                                           |  8 +++
 src/java/org/apache/cassandra/config/Config.java   |  2 +-
 .../cassandra/config/DatabaseDescriptor.java       | 27 ++++++--
 .../config/DatabaseDescriptorRefTest.java          |  2 +-
 .../cassandra/config/DatabaseDescriptorTest.java   | 78 ++++++++++++++++++++++
 6 files changed, 112 insertions(+), 6 deletions(-)

diff --cc CHANGES.txt
index 1083096,7aecefa..80b1532
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,7 -1,8 +1,8 @@@
 -3.0.24:
 +3.11.10
 +Merged from 3.0:
+  * Improved check of num_tokens against the length of initial_token 
(CASSANDRA-14477)
   * Fix a race condition on ColumnFamilyStore and TableMetrics 
(CASSANDRA-16228)
   * Remove the SEPExecutor blocking behavior (CASSANDRA-16186)
 - * Wait for schema agreement when bootstrapping (CASSANDRA-15158)
   * Fix invalid cell value skipping when reading from disk (CASSANDRA-16223)
   * Prevent invoking enable/disable gossip when not in NORMAL (CASSANDRA-16146)
  
diff --cc NEWS.txt
index b3246b5,42fbf63..d16bcce
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -42,15 -42,16 +42,23 @@@ restore snapshots created with the prev
  'sstableloader' tool. You can upgrade the file format of your snapshots
  using the provided 'sstableupgrade' tool.
  
 -3.0.24
 -======
 -
++3.11.10
++=====
+ Upgrading
+ ---------
+     - In cassandra.yaml, num_tokens must be defined if initial_token is 
defined.
+       If it is not defined, or not equal to the numbers of tokens defined in 
initial_tokens,
+       the node will not start. See CASSANDRA-14477 for details.
+ 
 -3.0.20
 +3.11.9
 +======
 +Upgrading
 +---------
 +   - Custom compaction strategies must handle getting sstables added/removed 
notifications for
 +     sstables already added/removed - see CASSANDRA-14103 for details. This 
has been a requirement
 +     for correct operation since 3.11.0 due to an issue in 
CompactionStrategyManager.
 +
 +3.11.7
  ======
  
  Upgrading
diff --cc src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index fe7291b,04293fb..cbf42b9
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@@ -314,34 -193,136 +314,34 @@@ public class DatabaseDescripto
          }
      }
  
 -    @VisibleForTesting
 -    public static void applyAddressConfig(Config config) throws 
ConfigurationException
 +    private static void setConfig(Config config)
      {
 -        listenAddress = null;
 -        rpcAddress = null;
 -        broadcastAddress = null;
 -        broadcastRpcAddress = null;
 -
 -        /* Local IP, hostname or interface to bind services to */
 -        if (config.listen_address != null && config.listen_interface != null)
 -        {
 -            throw new ConfigurationException("Set listen_address OR 
listen_interface, not both", false);
 -        }
 -        else if (config.listen_address != null)
 -        {
 -            try
 -            {
 -                listenAddress = InetAddress.getByName(config.listen_address);
 -            }
 -            catch (UnknownHostException e)
 -            {
 -                throw new ConfigurationException("Unknown listen_address '" + 
config.listen_address + "'", false);
 -            }
 -
 -            if (listenAddress.isAnyLocalAddress())
 -                throw new ConfigurationException("listen_address cannot be a 
wildcard address (" + config.listen_address + ")!", false);
 -        }
 -        else if (config.listen_interface != null)
 -        {
 -            listenAddress = 
getNetworkInterfaceAddress(config.listen_interface, "listen_interface", 
config.listen_interface_prefer_ipv6);
 -        }
 +        conf = config;
 +    }
  
 -        /* Gossip Address to broadcast */
 -        if (config.broadcast_address != null)
 -        {
 -            try
 -            {
 -                broadcastAddress = 
InetAddress.getByName(config.broadcast_address);
 -            }
 -            catch (UnknownHostException e)
 -            {
 -                throw new ConfigurationException("Unknown broadcast_address 
'" + config.broadcast_address + "'", false);
 -            }
 +    private static void applyAll() throws ConfigurationException
 +    {
 +        applySimpleConfig();
  
 -            if (broadcastAddress.isAnyLocalAddress())
 -                throw new ConfigurationException("broadcast_address cannot be 
a wildcard address (" + config.broadcast_address + ")!", false);
 -        }
 +        applyPartitioner();
  
 -        /* Local IP, hostname or interface to bind RPC server to */
 -        if (config.rpc_address != null && config.rpc_interface != null)
 -        {
 -            throw new ConfigurationException("Set rpc_address OR 
rpc_interface, not both", false);
 -        }
 -        else if (config.rpc_address != null)
 -        {
 -            try
 -            {
 -                rpcAddress = InetAddress.getByName(config.rpc_address);
 -            }
 -            catch (UnknownHostException e)
 -            {
 -                throw new ConfigurationException("Unknown host in rpc_address 
" + config.rpc_address, false);
 -            }
 -        }
 -        else if (config.rpc_interface != null)
 -        {
 -            rpcAddress = getNetworkInterfaceAddress(config.rpc_interface, 
"rpc_interface", config.rpc_interface_prefer_ipv6);
 -        }
 -        else
 -        {
 -            rpcAddress = FBUtilities.getLocalAddress();
 -        }
 +        applyAddressConfig();
  
 -        /* RPC address to broadcast */
 -        if (config.broadcast_rpc_address != null)
 -        {
 -            try
 -            {
 -                broadcastRpcAddress = 
InetAddress.getByName(config.broadcast_rpc_address);
 -            }
 -            catch (UnknownHostException e)
 -            {
 -                throw new ConfigurationException("Unknown 
broadcast_rpc_address '" + config.broadcast_rpc_address + "'", false);
 -            }
 +        applyThriftHSHA();
  
 -            if (broadcastRpcAddress.isAnyLocalAddress())
 -                throw new ConfigurationException("broadcast_rpc_address 
cannot be a wildcard address (" + config.broadcast_rpc_address + ")!", false);
 -        }
 -        else
 -        {
 -            if (rpcAddress.isAnyLocalAddress())
 -                throw new ConfigurationException("If rpc_address is set to a 
wildcard address (" + config.rpc_address + "), then " +
 -                                                 "you must set 
broadcast_rpc_address to a value other than " + config.rpc_address, false);
 -        }
 -    }
 +        applySnitch();
  
 -    @VisibleForTesting
 -    static void applyTokensConfig(Config config) throws ConfigurationException
 -    {
 -        if (config.num_tokens != null && config.num_tokens > MAX_NUM_TOKENS)
 -            throw new ConfigurationException(String.format("A maximum number 
of %d tokens per node is supported", MAX_NUM_TOKENS), false);
 +        applyRequestScheduler();
  
-         applyInitialTokens();
 -        if (config.initial_token != null)
 -        {
 -            Collection<String> tokens = 
tokensFromString(config.initial_token);
 -            if (config.num_tokens == null)
 -            {
 -                throw new ConfigurationException("initial_token was set but 
num_tokens is not!", false);
 -            }
++        applyTokensConfig();
  
 -            if (tokens.size() != config.num_tokens)
 -            {
 -                throw new ConfigurationException(String.format("The number of 
initial tokens (by initial_token) specified (%s) is different from num_tokens 
value (%s)",
 -                                                               tokens.size(),
 -                                                               
config.num_tokens),
 -                                                 false);
 -            }
 +        applySeedProvider();
  
 -            for (String token : tokens)
 -                partitioner.getTokenFactory().validate(token);
 -        }
 -        else if (config.num_tokens == null)
 -        {
 -            config.num_tokens = 1;
 -        }
 +        applyEncryptionContext();
      }
  
 -    public static void applyConfig(Config config) throws 
ConfigurationException
 +    private static void applySimpleConfig()
      {
 -        conf = config;
  
          if (conf.commitlog_sync == null)
          {
@@@ -616,40 -686,7 +616,40 @@@
          if (conf.concurrent_compactors <= 0)
              throw new ConfigurationException("concurrent_compactors should be 
strictly greater than 0, but was " + conf.concurrent_compactors, false);
  
-         if (conf.num_tokens > MAX_NUM_TOKENS)
 -        applyTokensConfig(config);
++        if (conf.num_tokens != null && conf.num_tokens > MAX_NUM_TOKENS)
 +            throw new ConfigurationException(String.format("A maximum number 
of %d tokens per node is supported", MAX_NUM_TOKENS), false);
 +
 +        try
 +        {
 +            // if prepared_statements_cache_size_mb option was set to "auto" 
then size of the cache should be "max(1/256 of Heap (in MB), 10MB)"
 +            preparedStatementsCacheSizeInMB = 
(conf.prepared_statements_cache_size_mb == null)
 +                                              ? Math.max(10, (int) 
(Runtime.getRuntime().maxMemory() / 1024 / 1024 / 256))
 +                                              : 
conf.prepared_statements_cache_size_mb;
 +
 +            if (preparedStatementsCacheSizeInMB <= 0)
 +                throw new NumberFormatException(); // to escape duplicating 
error message
 +        }
 +        catch (NumberFormatException e)
 +        {
 +            throw new 
ConfigurationException("prepared_statements_cache_size_mb option was set 
incorrectly to '"
 +                                             + 
conf.prepared_statements_cache_size_mb + "', supported values are <integer> >= 
0.", false);
 +        }
 +
 +        try
 +        {
 +            // if thrift_prepared_statements_cache_size_mb option was set to 
"auto" then size of the cache should be "max(1/256 of Heap (in MB), 10MB)"
 +            thriftPreparedStatementsCacheSizeInMB = 
(conf.thrift_prepared_statements_cache_size_mb == null)
 +                                                    ? Math.max(10, (int) 
(Runtime.getRuntime().maxMemory() / 1024 / 1024 / 256))
 +                                                    : 
conf.thrift_prepared_statements_cache_size_mb;
 +
 +            if (thriftPreparedStatementsCacheSizeInMB <= 0)
 +                throw new NumberFormatException(); // to escape duplicating 
error message
 +        }
 +        catch (NumberFormatException e)
 +        {
 +            throw new 
ConfigurationException("thrift_prepared_statements_cache_size_mb option was set 
incorrectly to '"
 +                                             + 
conf.thrift_prepared_statements_cache_size_mb + "', supported values are 
<integer> >= 0.", false);
 +        }
  
          try
          {
@@@ -940,107 -753,60 +940,126 @@@
          }
          if (seedProvider.getSeeds().size() == 0)
              throw new ConfigurationException("The seed provider lists no 
seeds.", false);
 +    }
  
-     public static void applyInitialTokens()
 -        if (conf.user_defined_function_fail_timeout < 0)
 -            throw new 
ConfigurationException("user_defined_function_fail_timeout must not be 
negative", false);
 -        if (conf.user_defined_function_warn_timeout < 0)
 -            throw new 
ConfigurationException("user_defined_function_warn_timeout must not be 
negative", false);
++    public static void applyTokensConfig()
++    {
++        applyTokensConfig(conf);
++    }
+ 
 -        if (conf.user_defined_function_fail_timeout < 
conf.user_defined_function_warn_timeout)
 -            throw new 
ConfigurationException("user_defined_function_warn_timeout must less than 
user_defined_function_fail_timeout", false);
++    static void applyTokensConfig(Config conf)
 +    {
 +        if (conf.initial_token != null)
 +        {
 +            Collection<String> tokens = tokensFromString(conf.initial_token);
++            if (conf.num_tokens == null)
++            {
++                throw new ConfigurationException("initial_token was set but 
num_tokens is not!", false);
++            }
+ 
 -        if (conf.commitlog_segment_size_in_mb <= 0)
 -            throw new ConfigurationException("commitlog_segment_size_in_mb 
must be positive, but was "
 -                    + conf.commitlog_segment_size_in_mb, false);
 -        else if (conf.commitlog_segment_size_in_mb >= 2048)
 -            throw new ConfigurationException("commitlog_segment_size_in_mb 
must be smaller than 2048, but was "
 -                    + conf.commitlog_segment_size_in_mb, false);
 +            if (tokens.size() != conf.num_tokens)
-                 throw new ConfigurationException("The number of initial 
tokens (by initial_token) specified is different from num_tokens value", false);
++            {
++                throw new ConfigurationException(String.format("The number of 
initial tokens (by initial_token) specified (%s) is different from num_tokens 
value (%s)",
++                                                               tokens.size(),
++                                                               
conf.num_tokens),
++                                                 false);
++            }
 +
 +            for (String token : tokens)
 +                partitioner.getTokenFactory().validate(token);
 +        }
++        else if (conf.num_tokens == null)
++        {
++            conf.num_tokens = 1;
++        }
 +    }
 +
 +    // Maybe safe for clients + tools
 +    public static void applyRequestScheduler()
 +    {
 +        /* Request Scheduler setup */
 +        requestSchedulerOptions = conf.request_scheduler_options;
 +        if (conf.request_scheduler != null)
 +        {
 +            try
 +            {
 +                if (requestSchedulerOptions == null)
 +                {
 +                    requestSchedulerOptions = new RequestSchedulerOptions();
 +                }
 +                Class<?> cls = Class.forName(conf.request_scheduler);
 +                requestScheduler = (IRequestScheduler) 
cls.getConstructor(RequestSchedulerOptions.class).newInstance(requestSchedulerOptions);
 +            }
 +            catch (ClassNotFoundException e)
 +            {
 +                throw new ConfigurationException("Invalid Request Scheduler 
class " + conf.request_scheduler, false);
 +            }
 +            catch (Exception e)
 +            {
 +                throw new ConfigurationException("Unable to instantiate 
request scheduler", e);
 +            }
 +        }
 +        else
 +        {
 +            requestScheduler = new NoScheduler();
 +        }
  
 -        if (conf.max_mutation_size_in_kb == null)
 -            conf.max_mutation_size_in_kb = conf.commitlog_segment_size_in_mb 
* 1024 / 2;
 -        else if (conf.commitlog_segment_size_in_mb * 1024 < 2 * 
conf.max_mutation_size_in_kb)
 -            throw new ConfigurationException("commitlog_segment_size_in_mb 
must be at least twice the size of max_mutation_size_in_kb / 1024", false);
 +        if (conf.request_scheduler_id == RequestSchedulerId.keyspace)
 +        {
 +            requestSchedulerId = conf.request_scheduler_id;
 +        }
 +        else
 +        {
 +            // Default to Keyspace
 +            requestSchedulerId = RequestSchedulerId.keyspace;
 +        }
 +    }
  
 -        // native transport encryption options
 -        if (conf.native_transport_port_ssl != null
 -            && conf.native_transport_port_ssl.intValue() != 
conf.native_transport_port.intValue()
 -            && !conf.client_encryption_options.enabled)
 +    // definitely not safe for tools + clients - implicitly instantiates 
StorageService
 +    public static void applySnitch()
 +    {
 +        /* end point snitch */
 +        if (conf.endpoint_snitch == null)
          {
 -            throw new ConfigurationException("Encryption must be enabled in 
client_encryption_options for native_transport_port_ssl", false);
 +            throw new ConfigurationException("Missing endpoint_snitch 
directive", false);
          }
 +        snitch = createEndpointSnitch(conf.dynamic_snitch, 
conf.endpoint_snitch);
 +        EndpointSnitchInfo.create();
  
 -        // If max protocol version has been set, just validate it's within an 
acceptable range
 -        if (conf.native_transport_max_negotiable_protocol_version != 
Integer.MIN_VALUE)
 +        localDC = snitch.getDatacenter(FBUtilities.getBroadcastAddress());
 +        localComparator = new Comparator<InetAddress>()
          {
 -            if (conf.native_transport_max_negotiable_protocol_version < 
Server.MIN_SUPPORTED_VERSION
 -                || conf.native_transport_max_negotiable_protocol_version > 
Server.CURRENT_VERSION)
 +            public int compare(InetAddress endpoint1, InetAddress endpoint2)
              {
 -                throw new ConfigurationException(String.format("Invalid 
setting for native_transport_max_negotiable_version (%d); " +
 -                                                               "Values 
between %s and %s are supported",
 -                                                               
conf.native_transport_max_negotiable_protocol_version,
 -                                                               
Server.MIN_SUPPORTED_VERSION,
 -                                                               
Server.CURRENT_VERSION));
 +                boolean local1 = 
localDC.equals(snitch.getDatacenter(endpoint1));
 +                boolean local2 = 
localDC.equals(snitch.getDatacenter(endpoint2));
 +                if (local1 && !local2)
 +                    return -1;
 +                if (local2 && !local1)
 +                    return 1;
 +                return 0;
              }
 -        }
 -
 -        if (conf.max_value_size_in_mb == null || conf.max_value_size_in_mb <= 
0)
 -            throw new ConfigurationException("max_value_size_in_mb must be 
positive", false);
 -        else if (conf.max_value_size_in_mb >= 2048)
 -            throw new ConfigurationException("max_value_size_in_mb must be 
smaller than 2048, but was "
 -                    + conf.max_value_size_in_mb, false);
 +        };
 +    }
  
 -        if (conf.otc_coalescing_enough_coalesced_messages > 128)
 -            throw new 
ConfigurationException("otc_coalescing_enough_coalesced_messages must be 
smaller than 128", false);
 +    // definitely not safe for tools + clients - implicitly instantiates 
schema
 +    public static void applyPartitioner()
 +    {
 +        /* Hashing strategy */
 +        if (conf.partitioner == null)
 +        {
 +            throw new ConfigurationException("Missing directive: 
partitioner", false);
 +        }
 +        try
 +        {
 +            partitioner = 
FBUtilities.newPartitioner(System.getProperty(Config.PROPERTY_PREFIX + 
"partitioner", conf.partitioner));
 +        }
 +        catch (Exception e)
 +        {
 +            throw new ConfigurationException("Invalid partitioner class " + 
conf.partitioner, false);
 +        }
  
 -        if (conf.otc_coalescing_enough_coalesced_messages <= 0)
 -            throw new 
ConfigurationException("otc_coalescing_enough_coalesced_messages must be 
positive", false);
 +        paritionerName = partitioner.getClass().getCanonicalName();
      }
  
      /**
diff --cc test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java
index 88c965b,0000000..6e865ae
mode 100644,000000..100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorRefTest.java
@@@ -1,268 -1,0 +1,268 @@@
 +/*
 + * 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.config;
 +
 +import java.io.ByteArrayOutputStream;
 +import java.io.IOException;
 +import java.io.InputStream;
 +import java.io.PrintStream;
 +import java.lang.management.ManagementFactory;
 +import java.lang.management.ThreadInfo;
 +import java.lang.management.ThreadMXBean;
 +import java.lang.reflect.Method;
 +import java.net.URL;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +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 org.junit.Test;
 +
 +import org.apache.cassandra.utils.Pair;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.fail;
 +
 +/**
 + * Verifies that {@link DatabaseDescriptor#clientInitialization()} } and a 
couple of <i>apply</i> methods
 + * do not somehow lazily initialize any unwanted part of Cassandra like 
schema, commit log or start
 + * unexpected threads.
 + *
 + * {@link DatabaseDescriptor#toolInitialization()} is tested via unit tests 
extending
 + * {@link org.apache.cassandra.tools.ToolsTester}.
 + */
 +public class DatabaseDescriptorRefTest
 +{
 +    static final String[] validClasses = {
 +    "org.apache.cassandra.auth.IInternodeAuthenticator",
 +    "org.apache.cassandra.auth.IAuthenticator",
 +    "org.apache.cassandra.auth.IAuthorizer",
 +    "org.apache.cassandra.auth.IRoleManager",
 +    "org.apache.cassandra.config.DatabaseDescriptor",
 +    "org.apache.cassandra.config.ConfigurationLoader",
 +    "org.apache.cassandra.config.Config",
 +    "org.apache.cassandra.config.Config$1",
 +    "org.apache.cassandra.config.Config$RequestSchedulerId",
 +    "org.apache.cassandra.config.Config$CommitLogSync",
 +    "org.apache.cassandra.config.Config$DiskAccessMode",
 +    "org.apache.cassandra.config.Config$DiskFailurePolicy",
 +    "org.apache.cassandra.config.Config$CommitFailurePolicy",
 +    "org.apache.cassandra.config.Config$DiskOptimizationStrategy",
 +    "org.apache.cassandra.config.Config$InternodeCompression",
 +    "org.apache.cassandra.config.Config$MemtableAllocationType",
 +    "org.apache.cassandra.config.Config$UserFunctionTimeoutPolicy",
 +    "org.apache.cassandra.config.RequestSchedulerOptions",
 +    "org.apache.cassandra.config.ParameterizedClass",
 +    "org.apache.cassandra.config.EncryptionOptions",
 +    "org.apache.cassandra.config.EncryptionOptions$ClientEncryptionOptions",
 +    "org.apache.cassandra.config.EncryptionOptions$ServerEncryptionOptions",
 +    
"org.apache.cassandra.config.EncryptionOptions$ServerEncryptionOptions$InternodeEncryption",
 +    "org.apache.cassandra.config.ReplicaFilteringProtectionOptions",
 +    "org.apache.cassandra.config.YamlConfigurationLoader",
 +    "org.apache.cassandra.config.YamlConfigurationLoader$PropertiesChecker",
 +    "org.apache.cassandra.config.YamlConfigurationLoader$PropertiesChecker$1",
 +    "org.apache.cassandra.config.YamlConfigurationLoader$CustomConstructor",
 +    "org.apache.cassandra.config.TransparentDataEncryptionOptions",
 +    "org.apache.cassandra.dht.IPartitioner",
 +    "org.apache.cassandra.distributed.impl.InstanceClassLoader",
 +    "org.apache.cassandra.distributed.impl.InstanceConfig",
 +    "org.apache.cassandra.distributed.impl.InvokableInstance",
 +    
"org.apache.cassandra.distributed.impl.InvokableInstance$CallableNoExcept",
 +    
"org.apache.cassandra.distributed.impl.InvokableInstance$InstanceFunction",
 +    
"org.apache.cassandra.distributed.impl.InvokableInstance$SerializableBiConsumer",
 +    
"org.apache.cassandra.distributed.impl.InvokableInstance$SerializableBiFunction",
 +    
"org.apache.cassandra.distributed.impl.InvokableInstance$SerializableCallable",
 +    
"org.apache.cassandra.distributed.impl.InvokableInstance$SerializableConsumer",
 +    
"org.apache.cassandra.distributed.impl.InvokableInstance$SerializableFunction",
 +    
"org.apache.cassandra.distributed.impl.InvokableInstance$SerializableRunnable",
 +    
"org.apache.cassandra.distributed.impl.InvokableInstance$SerializableTriFunction",
 +    "org.apache.cassandra.distributed.impl.InvokableInstance$TriFunction",
 +    "org.apache.cassandra.distributed.impl.Message",
 +    "org.apache.cassandra.exceptions.ConfigurationException",
 +    "org.apache.cassandra.exceptions.RequestValidationException",
 +    "org.apache.cassandra.exceptions.CassandraException",
 +    "org.apache.cassandra.exceptions.TransportException",
 +    "org.apache.cassandra.locator.IEndpointSnitch",
 +    "org.apache.cassandra.io.FSWriteError",
 +    "org.apache.cassandra.io.FSError",
 +    "org.apache.cassandra.io.compress.ICompressor",
 +    "org.apache.cassandra.io.compress.LZ4Compressor",
 +    "org.apache.cassandra.io.sstable.metadata.MetadataType",
 +    "org.apache.cassandra.io.util.BufferedDataOutputStreamPlus",
 +    "org.apache.cassandra.io.util.DataOutputBuffer",
 +    "org.apache.cassandra.io.util.DataOutputBufferFixed",
 +    "org.apache.cassandra.io.util.DataOutputStreamPlus",
 +    "org.apache.cassandra.io.util.DataOutputPlus",
 +    "org.apache.cassandra.io.util.DiskOptimizationStrategy",
 +    "org.apache.cassandra.io.util.SpinningDiskOptimizationStrategy",
 +    "org.apache.cassandra.locator.SimpleSeedProvider",
 +    "org.apache.cassandra.locator.SeedProvider",
 +    "org.apache.cassandra.net.BackPressureStrategy",
 +    "org.apache.cassandra.scheduler.IRequestScheduler",
 +    "org.apache.cassandra.security.EncryptionContext",
 +    "org.apache.cassandra.service.CacheService$CacheType",
 +    "org.apache.cassandra.utils.FBUtilities",
 +    "org.apache.cassandra.utils.FBUtilities$1",
 +    "org.apache.cassandra.utils.CloseableIterator",
 +    "org.apache.cassandra.utils.Pair",
 +    "org.apache.cassandra.OffsetAwareConfigurationLoader",
 +    "org.apache.cassandra.ConsoleAppender",
 +    "org.apache.cassandra.ConsoleAppender$1",
 +    "org.apache.cassandra.LogbackStatusListener",
 +    "org.apache.cassandra.LogbackStatusListener$ToLoggerOutputStream",
 +    "org.apache.cassandra.LogbackStatusListener$WrappedPrintStream",
 +    "org.apache.cassandra.TeeingAppender",
 +    // generated classes
 +    "org.apache.cassandra.config.ConfigBeanInfo",
 +    "org.apache.cassandra.config.ConfigCustomizer",
 +    "org.apache.cassandra.config.EncryptionOptionsBeanInfo",
 +    "org.apache.cassandra.config.EncryptionOptionsCustomizer",
 +    
"org.apache.cassandra.config.EncryptionOptions$ServerEncryptionOptionsBeanInfo",
 +    
"org.apache.cassandra.config.EncryptionOptions$ServerEncryptionOptionsCustomizer",
 +    "org.apache.cassandra.ConsoleAppenderBeanInfo",
 +    "org.apache.cassandra.ConsoleAppenderCustomizer",
 +    };
 +
 +    static final Set<String> checkedClasses = new 
HashSet<>(Arrays.asList(validClasses));
 +
 +    @Test
 +    public void testDatabaseDescriptorRef() throws Throwable
 +    {
 +        PrintStream out = System.out;
 +        PrintStream err = System.err;
 +
 +        ThreadMXBean threads = ManagementFactory.getThreadMXBean();
 +        int threadCount = threads.getThreadCount();
 +
 +        ClassLoader delegate = Thread.currentThread().getContextClassLoader();
 +
 +        List<Pair<String, Exception>> violations = 
Collections.synchronizedList(new ArrayList<>());
 +
 +        ClassLoader cl = new ClassLoader(null)
 +        {
 +            final Map<String, Class<?>> classMap = new HashMap<>();
 +
 +            public URL getResource(String name)
 +            {
 +                return delegate.getResource(name);
 +            }
 +
 +            public InputStream getResourceAsStream(String name)
 +            {
 +                return delegate.getResourceAsStream(name);
 +            }
 +
 +            protected Class<?> findClass(String name) throws 
ClassNotFoundException
 +            {
 +                Class<?> cls = classMap.get(name);
 +                if (cls != null)
 +                    return cls;
 +
 +                if (name.startsWith("org.apache.cassandra."))
 +                {
 +                    // out.println(name);
 +
 +                    if (!checkedClasses.contains(name))
 +                        violations.add(Pair.create(name, new Exception()));
 +                }
 +
 +                URL url = delegate.getResource(name.replace('.', '/') + 
".class");
 +                if (url == null)
 +                    throw new ClassNotFoundException(name);
 +                try (InputStream in = url.openConnection().getInputStream())
 +                {
 +                    ByteArrayOutputStream os = new ByteArrayOutputStream();
 +                    int c;
 +                    while ((c = in.read()) != -1)
 +                        os.write(c);
 +                    byte[] data = os.toByteArray();
 +                    cls = defineClass(name, data, 0, data.length);
 +                    classMap.put(name, cls);
 +                    return cls;
 +                }
 +                catch (IOException e)
 +                {
 +                    throw new ClassNotFoundException(name, e);
 +                }
 +            }
 +        };
 +
 +        Thread.currentThread().setContextClassLoader(cl);
 +
 +        assertEquals("thread started", threadCount, threads.getThreadCount());
 +
 +        Class cDatabaseDescriptor = 
Class.forName("org.apache.cassandra.config.DatabaseDescriptor", true, cl);
 +
 +        for (String methodName : new String[]{
 +            "clientInitialization",
 +            "applyAddressConfig",
 +            "applyThriftHSHA",
-             "applyInitialTokens",
++            "applyTokensConfig",
 +            // no seed provider in default configuration for clients
 +            // "applySeedProvider",
 +            // definitely not safe for clients - implicitly instantiates 
schema
 +            // "applyPartitioner",
 +            // definitely not safe for clients - implicitly instantiates 
StorageService
 +            // "applySnitch",
 +            "applyEncryptionContext",
 +            // starts "REQUEST-SCHEDULER" thread via RoundRobinScheduler
 +            // "applyRequestScheduler",
 +        })
 +        {
 +            Method method = cDatabaseDescriptor.getDeclaredMethod(methodName);
 +            method.invoke(null);
 +
 +            if ("clientInitialization".equals(methodName) &&
 +                threadCount + 1 == threads.getThreadCount())
 +            {
 +                // ignore the "AsyncAppender-Worker-ASYNC" thread
 +                threadCount++;
 +            }
 +
 +            if (threadCount != threads.getThreadCount())
 +            {
 +                for (ThreadInfo threadInfo : 
threads.getThreadInfo(threads.getAllThreadIds()))
 +                    out.println("Thread #" + threadInfo.getThreadId() + ": " 
+ threadInfo.getThreadName());
 +                assertEquals("thread started in " + methodName, threadCount, 
ManagementFactory.getThreadMXBean().getThreadCount());
 +            }
 +
 +            checkViolations(err, violations);
 +        }
 +    }
 +
 +    private void checkViolations(PrintStream err, List<Pair<String, 
Exception>> violations)
 +    {
 +        if (!violations.isEmpty())
 +        {
 +            for (Pair<String, Exception> violation : new 
ArrayList<>(violations))
 +            {
 +                err.println();
 +                err.println();
 +                err.println("VIOLATION: " + violation.left);
 +                violation.right.printStackTrace(err);
 +            }
 +
 +            fail();
 +        }
 +    }
 +}
diff --cc test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index 4788289,7614e02..4c6b3d7
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@@ -321,4 -313,131 +321,82 @@@ public class DatabaseDescriptorTes
              DatabaseDescriptor.setRepairSessionMaxTreeDepth(previousDepth);
          }
      }
+ 
+     @Test
 -    public void 
testApplyInitialTokensInitialTokensSetNumTokensSetAndDoesMatch() throws 
Exception
++    public void 
testApplyInitialTokensInitialTokensSetNumTokensSetAndDoesMatch()
+     {
+         Config config = DatabaseDescriptor.loadConfig();
+         config.initial_token = "0,256,1024";
+         config.num_tokens = 3;
+ 
 -        unregisterSnitchesForTokenConfigTest();
 -
+         try
+         {
+             DatabaseDescriptor.applyTokensConfig(config);
 -
+             Assert.assertEquals(Integer.valueOf(3), config.num_tokens);
+             Assert.assertEquals(3, 
DatabaseDescriptor.tokensFromString(config.initial_token).size());
+         }
 -        finally
++        catch (ConfigurationException e)
+         {
 -            unregisterSnitchesForTokenConfigTest();
++            Assert.fail("number of tokens in initial_token=0,256,1024 does 
not match num_tokens = 3");
+         }
+     }
+ 
+     @Test
 -    public void 
testApplyInitialTokensInitialTokensSetNumTokensSetAndDoesntMatch() throws 
Exception
++    public void 
testApplyInitialTokensInitialTokensSetNumTokensSetAndDoesntMatch()
+     {
+         Config config = DatabaseDescriptor.loadConfig();
+         config.initial_token = "0,256,1024";
+         config.num_tokens = 10;
+ 
 -        unregisterSnitchesForTokenConfigTest();
 -
+         try
+         {
+             DatabaseDescriptor.applyTokensConfig(config);
+ 
+             Assert.fail("initial_token = 0,256,1024 and num_tokens = 10 but 
applyInitialTokens() did not fail!");
+         }
+         catch (ConfigurationException ex)
+         {
+             Assert.assertEquals("The number of initial tokens (by 
initial_token) specified (3) is different from num_tokens value (10)",
+                                 ex.getMessage());
+         }
 -        finally
 -        {
 -            unregisterSnitchesForTokenConfigTest();
 -        }
+     }
+ 
+     @Test
 -    public void testApplyInitialTokensInitialTokensSetNumTokensNotSet() 
throws Exception
++    public void testApplyInitialTokensInitialTokensSetNumTokensNotSet()
+     {
+         Config config = DatabaseDescriptor.loadConfig();
 -
 -        unregisterSnitchesForTokenConfigTest();
++        config.initial_token = "0,256,1024";
+ 
+         try
+         {
 -            config.initial_token = "0,256,1024";
 -            config.num_tokens = null;
+             DatabaseDescriptor.applyTokensConfig(config);
+             Assert.fail("setting initial_token and not setting num_tokens is 
invalid");
+         }
+         catch (ConfigurationException ex)
+         {
+             Assert.assertEquals("initial_token was set but num_tokens is 
not!", ex.getMessage());
+         }
 -        finally
 -        {
 -            unregisterSnitchesForTokenConfigTest();
 -        }
+     }
+ 
+     @Test
 -    public void testApplyInitialTokensInitialTokensNotSetNumTokensSet() 
throws Exception
++    public void testApplyInitialTokensInitialTokensNotSetNumTokensSet()
+     {
+         Config config = DatabaseDescriptor.loadConfig();
+         config.num_tokens = 3;
+ 
 -        unregisterSnitchesForTokenConfigTest();
 -
 -        try
 -        {
 -            DatabaseDescriptor.applyTokensConfig(config);
 -        }
 -        finally
 -        {
 -            unregisterSnitchesForTokenConfigTest();
 -        }
++        DatabaseDescriptor.applyTokensConfig(config);
+ 
+         Assert.assertEquals(Integer.valueOf(3), config.num_tokens);
+         
Assert.assertTrue(DatabaseDescriptor.tokensFromString(config.initial_token).isEmpty());
+     }
+ 
+     @Test
 -    public void testApplyInitialTokensInitialTokensNotSetNumTokensNotSet() 
throws Exception
++    public void testApplyInitialTokensInitialTokensNotSetNumTokensNotSet()
+     {
+         Config config = DatabaseDescriptor.loadConfig();
 -
 -        unregisterSnitchesForTokenConfigTest();
 -
 -        try
 -        {
 -            DatabaseDescriptor.applyTokensConfig(config);
 -        }
 -        finally
 -        {
 -            unregisterSnitchesForTokenConfigTest();
 -        }
++        DatabaseDescriptor.applyTokensConfig(config);
+ 
+         Assert.assertEquals(Integer.valueOf(1), config.num_tokens);
+         
Assert.assertTrue(DatabaseDescriptor.tokensFromString(config.initial_token).isEmpty());
+     }
 -
 -    private void unregisterSnitchesForTokenConfigTest() throws Exception
 -    {
 -        try
 -        {
 -            MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
 -            mbs.unregisterMBean(new 
ObjectName("org.apache.cassandra.db:type=DynamicEndpointSnitch"));
 -            mbs.unregisterMBean(new 
ObjectName("org.apache.cassandra.db:type=EndpointSnitchInfo"));
 -        }
 -        catch (InstanceNotFoundException ex)
 -        {
 -            // ok
 -        }
 -    }
 -}
 +}


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to