You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by aw...@apache.org on 2022/12/09 20:01:39 UTC

[cassandra] 01/05: first batch of fixes after David's review

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

aweisberg pushed a commit to branch cas-accord-v2
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit 5acac4881225735b3e354ca376c15a49518feea6
Author: Caleb Rackliffe <ca...@gmail.com>
AuthorDate: Mon Dec 5 17:14:19 2022 -0600

    first batch of fixes after David's review
---
 src/antlr/Parser.g                                 |  2 +-
 .../cassandra/audit/AuditLogEntryCategory.java     |  2 +-
 .../apache/cassandra/audit/AuditLogEntryType.java  |  4 +---
 .../org/apache/cassandra/audit/AuditLogFilter.java |  2 +-
 src/java/org/apache/cassandra/config/Config.java   |  3 +--
 .../cassandra/config/DatabaseDescriptor.java       | 25 ++++++++++++++++------
 .../apache/cassandra/service/StorageService.java   | 11 ++++++++++
 .../cassandra/service/StorageServiceMBean.java     |  5 +++++
 .../cassandra/service/accord/AccordService.java    |  4 ++--
 .../cassandra/service/accord/txn/TxnDataName.java  | 16 ++++++++------
 .../org/apache/cassandra/utils/ByteBufferUtil.java | 11 ++++++++--
 .../distributed/test/accord/AccordTestBase.java    | 25 ++++++++++++----------
 .../cassandra/config/DatabaseDescriptorTest.java   |  2 ++
 .../org/apache/cassandra/utils/Generators.java     | 18 +++++++++++-----
 14 files changed, 89 insertions(+), 41 deletions(-)

diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g
index c91329e115..b17ffc6f1a 100644
--- a/src/antlr/Parser.g
+++ b/src/antlr/Parser.g
@@ -83,7 +83,7 @@ options {
     public RowDataReference.Raw newRowDataReference(Selectable.RawIdentifier tuple, Selectable.Raw selectable)
     {
         if (!isParsingTxn)
-            throw new IllegalStateException();
+            throw new SyntaxException("Cannot create a row data reference unless parsing a transaction");
 
         if (references == null)
             references = new ArrayList<>();
diff --git a/src/java/org/apache/cassandra/audit/AuditLogEntryCategory.java b/src/java/org/apache/cassandra/audit/AuditLogEntryCategory.java
index 9db4ce05e9..ae5efc6677 100644
--- a/src/java/org/apache/cassandra/audit/AuditLogEntryCategory.java
+++ b/src/java/org/apache/cassandra/audit/AuditLogEntryCategory.java
@@ -23,5 +23,5 @@ package org.apache.cassandra.audit;
  */
 public enum AuditLogEntryCategory
 {
-    QUERY, DML, DDL, DCL, OTHER, AUTH, ERROR, PREPARE
+    QUERY, DML, DDL, DCL, OTHER, AUTH, ERROR, PREPARE, TRANSACTION
 }
diff --git a/src/java/org/apache/cassandra/audit/AuditLogEntryType.java b/src/java/org/apache/cassandra/audit/AuditLogEntryType.java
index ec3fe4d54c..9285429a5f 100644
--- a/src/java/org/apache/cassandra/audit/AuditLogEntryType.java
+++ b/src/java/org/apache/cassandra/audit/AuditLogEntryType.java
@@ -60,9 +60,7 @@ public enum AuditLogEntryType
     CREATE_ROLE(AuditLogEntryCategory.DCL),
     USE_KEYSPACE(AuditLogEntryCategory.OTHER),
     DESCRIBE(AuditLogEntryCategory.OTHER),
-    
-    // TODO: Is DML the most appropriate classification, given a transaction can read, write, or both?
-    TRANSACTION(AuditLogEntryCategory.DML),
+    TRANSACTION(AuditLogEntryCategory.TRANSACTION),
 
     /*
      * Common Audit Log Entry Types
diff --git a/src/java/org/apache/cassandra/audit/AuditLogFilter.java b/src/java/org/apache/cassandra/audit/AuditLogFilter.java
index d240e78c83..d75e544752 100644
--- a/src/java/org/apache/cassandra/audit/AuditLogFilter.java
+++ b/src/java/org/apache/cassandra/audit/AuditLogFilter.java
@@ -28,7 +28,7 @@ final class AuditLogFilter
 {
     private static final Logger logger = LoggerFactory.getLogger(AuditLogFilter.class);
 
-    private static ImmutableSet<String> EMPTY_FILTERS = ImmutableSet.of();
+    private static final ImmutableSet<String> EMPTY_FILTERS = ImmutableSet.of();
 
     final ImmutableSet<String> excludedKeyspaces;
     final ImmutableSet<String> includedKeyspaces;
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index 8a7dcacf44..db1ed12c8c 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -1173,11 +1173,10 @@ public class Config
             String value;
             try
             {
-                // don't use exceptions for normal control flow!
                 Object obj = field.get(config);
                 value = obj != null ? obj.toString() : "null";
             }
-            catch (NullPointerException | IllegalAccessException npe)
+            catch (IllegalAccessException npe)
             {
                 value = "null";
             }
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 57c7db266b..2955635694 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -1192,46 +1192,52 @@ public class DatabaseDescriptor
     @VisibleForTesting
     static void checkForLowestAcceptedTimeouts(Config conf)
     {
-        if(conf.read_request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
+        if (conf.read_request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
         {
             logInfo("read_request_timeout", conf.read_request_timeout, LOWEST_ACCEPTED_TIMEOUT);
             conf.read_request_timeout = new DurationSpec.LongMillisecondsBound("10ms");
         }
 
-        if(conf.range_request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
+        if (conf.range_request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
         {
             logInfo("range_request_timeout", conf.range_request_timeout, LOWEST_ACCEPTED_TIMEOUT);
             conf.range_request_timeout = new DurationSpec.LongMillisecondsBound("10ms");
         }
 
-        if(conf.request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
+        if (conf.request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
         {
             logInfo("request_timeout", conf.request_timeout, LOWEST_ACCEPTED_TIMEOUT);
             conf.request_timeout = new DurationSpec.LongMillisecondsBound("10ms");
         }
 
-        if(conf.write_request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
+        if (conf.write_request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
         {
             logInfo("write_request_timeout", conf.write_request_timeout, LOWEST_ACCEPTED_TIMEOUT);
             conf.write_request_timeout = new DurationSpec.LongMillisecondsBound("10ms");
         }
 
-        if(conf.cas_contention_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
+        if (conf.cas_contention_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
         {
             logInfo("cas_contention_timeout", conf.cas_contention_timeout, LOWEST_ACCEPTED_TIMEOUT);
             conf.cas_contention_timeout = new DurationSpec.LongMillisecondsBound("10ms");
         }
 
-        if(conf.counter_write_request_timeout.toMilliseconds()< LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
+        if (conf.counter_write_request_timeout.toMilliseconds()< LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
         {
             logInfo("counter_write_request_timeout", conf.counter_write_request_timeout, LOWEST_ACCEPTED_TIMEOUT);
             conf.counter_write_request_timeout = new DurationSpec.LongMillisecondsBound("10ms");
         }
-        if(conf.truncate_request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
+        if (conf.truncate_request_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
         {
             logInfo("truncate_request_timeout", conf.truncate_request_timeout, LOWEST_ACCEPTED_TIMEOUT);
             conf.truncate_request_timeout = LOWEST_ACCEPTED_TIMEOUT;
         }
+
+        if (conf.transaction_timeout.toMilliseconds() < LOWEST_ACCEPTED_TIMEOUT.toMilliseconds())
+        {
+            logInfo("transaction_timeout", conf.transaction_timeout, LOWEST_ACCEPTED_TIMEOUT);
+            conf.transaction_timeout = LOWEST_ACCEPTED_TIMEOUT;
+        }
     }
 
     private static void logInfo(String property, DurationSpec.LongMillisecondsBound actualValue, DurationSpec.LongMillisecondsBound lowestAcceptedValue)
@@ -1860,6 +1866,11 @@ public class DatabaseDescriptor
         return conf.transaction_timeout.to(unit);
     }
 
+    public static void setTransactionTimeout(long timeOutInMillis)
+    {
+        conf.transaction_timeout = new DurationSpec.LongMillisecondsBound(timeOutInMillis);
+    }
+
     public static void setCasContentionTimeout(long timeOutInMillis)
     {
         conf.cas_contention_timeout = new DurationSpec.LongMillisecondsBound(timeOutInMillis);
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index f20167569a..70f4308154 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -1587,6 +1587,17 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return DatabaseDescriptor.getTruncateRpcTimeout(MILLISECONDS);
     }
 
+    public void setTransactionTimeout(long value)
+    {
+        DatabaseDescriptor.setTransactionTimeout(value);
+        logger.info("set transaction timeout to {} ms", value);
+    }
+
+    public long getTransactionTimeout()
+    {
+        return DatabaseDescriptor.getTransactionTimeout(MILLISECONDS);
+    }
+
     @Deprecated
     public void setStreamThroughputMbPerSec(int value)
     {
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index c4e143410e..c036bf15bc 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -26,6 +26,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import javax.annotation.Nullable;
 import javax.management.NotificationEmitter;
@@ -33,6 +34,7 @@ import javax.management.openmbean.CompositeData;
 import javax.management.openmbean.OpenDataException;
 import javax.management.openmbean.TabularData;
 
+import org.apache.cassandra.config.DurationSpec;
 import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.utils.BreaksJMX;
@@ -630,6 +632,9 @@ public interface StorageServiceMBean extends NotificationEmitter
     public void setTruncateRpcTimeout(long value);
     public long getTruncateRpcTimeout();
 
+    public void setTransactionTimeout(long value);
+    public long getTransactionTimeout();
+
     public void setStreamThroughputMbitPerSec(int value);
     /**
      * @return stream_throughput_outbound in megabits
diff --git a/src/java/org/apache/cassandra/service/accord/AccordService.java b/src/java/org/apache/cassandra/service/accord/AccordService.java
index 344f7829d0..6d21f652de 100644
--- a/src/java/org/apache/cassandra/service/accord/AccordService.java
+++ b/src/java/org/apache/cassandra/service/accord/AccordService.java
@@ -115,7 +115,7 @@ public class AccordService implements Shutdownable
         try
         {
             Future<Result> future = node.coordinate(txn);
-            Result result = future.get(DatabaseDescriptor.getTransactionTimeout(TimeUnit.SECONDS), TimeUnit.SECONDS);
+            Result result = future.get(DatabaseDescriptor.getTransactionTimeout(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS);
             return (TxnData) result;
         }
         catch (ExecutionException e)
@@ -123,7 +123,7 @@ public class AccordService implements Shutdownable
             Throwable cause = e.getCause();
             if (cause instanceof Timeout)
                 throw throwTimeout(txn);
-            throw new RuntimeException(e);
+            throw new RuntimeException(cause);
         }
         catch (InterruptedException e)
         {
diff --git a/src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java b/src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java
index e073ee5d01..e4526beabd 100644
--- a/src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java
+++ b/src/java/org/apache/cassandra/service/accord/txn/TxnDataName.java
@@ -32,10 +32,15 @@ import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.ObjectSizes;
+
+import static com.google.common.primitives.Ints.checkedCast;
+import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
 
 public class TxnDataName implements Comparable<TxnDataName>
 {
     private static final TxnDataName RETURNING = new TxnDataName(Kind.RETURNING);
+    private static final long EMPTY_SIZE = ObjectSizes.measure(RETURNING);
 
     public enum Kind
     {
@@ -114,8 +119,7 @@ public class TxnDataName implements Comparable<TxnDataName>
     {
         if (kind != Kind.AUTO_READ)
             return false;
-        return metadata.keyspace.equals(parts[0])
-               && metadata.name.equals(parts[1]);
+        return metadata.keyspace.equals(parts[0]) && metadata.name.equals(parts[1]);
     }
 
     public DecoratedKey getDecoratedKey(TableMetadata metadata)
@@ -139,7 +143,7 @@ public class TxnDataName implements Comparable<TxnDataName>
 
     public long estimatedSizeOnHeap()
     {
-        long size = 0;
+        long size = EMPTY_SIZE;
         for (String part : parts)
             size += part.length();
         return size;
@@ -197,7 +201,7 @@ public class TxnDataName implements Comparable<TxnDataName>
         public void serialize(TxnDataName t, DataOutputPlus out, int version) throws IOException
         {
             out.writeByte(t.kind.value);
-            out.writeInt(t.parts.length);
+            out.writeUnsignedVInt(t.parts.length);
             for (String part : t.parts)
                 out.writeUTF(part);
         }
@@ -206,7 +210,7 @@ public class TxnDataName implements Comparable<TxnDataName>
         public TxnDataName deserialize(DataInputPlus in, int version) throws IOException
         {
             Kind kind = Kind.from(in.readByte());
-            int length = in.readInt();
+            int length = checkedCast(in.readUnsignedVInt());
             String[] parts = new String[length];
             for (int i = 0; i < length; i++)
                 parts[i] = in.readUTF();
@@ -216,7 +220,7 @@ public class TxnDataName implements Comparable<TxnDataName>
         @Override
         public long serializedSize(TxnDataName t, int version)
         {
-            int size = Byte.BYTES + Integer.BYTES;
+            int size = Byte.BYTES + sizeofUnsignedVInt(t.parts.length);
             for (String part : t.parts)
                 size += TypeSizes.sizeof(part);
             return size;
diff --git a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
index 9310741580..062d65f6e1 100644
--- a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
+++ b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
@@ -565,7 +565,12 @@ public class ByteBufferUtil
             // convert subtypes to BB
             Map<ByteBuffer, ByteBuffer> bbs = new LinkedHashMap<>();
             for (Map.Entry<?, ?> e : map.entrySet())
-                bbs.put(objectToBytes(e.getKey()), objectToBytes(e.getValue()));
+            {
+                Object key = e.getKey();
+                ByteBuffer previousValue = bbs.put(objectToBytes(key), objectToBytes(e.getValue()));
+                if (previousValue != null)
+                    throw new IllegalStateException("Key " + key + " already maps to value " + previousValue);
+            }
             // decompose/serializer doesn't use the isMultiCell, so safe to do this
             return MapType.getInstance(BytesType.instance, BytesType.instance, false).decompose(bbs);
         }
@@ -574,7 +579,9 @@ public class ByteBufferUtil
             Set<?> set = (Set<?>) obj;
             // convert subtypes to BB
             Set<ByteBuffer> bbs = new LinkedHashSet<>();
-            set.forEach(o -> bbs.add(objectToBytes(o)));
+            for (Object o : set)
+                if (!bbs.add(objectToBytes(o)))
+                    throw new IllegalStateException("Object " + o + " maps to a buffer that already exists in the set");
             // decompose/serializer doesn't use the isMultiCell, so safe to do this
             return SetType.getInstance(BytesType.instance, false).decompose(bbs);
         }
diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java
index c0e40707f5..60b583b40a 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java
@@ -23,7 +23,6 @@ import java.util.Map;
 import java.util.concurrent.Callable;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import com.google.common.base.Throwables;
 import net.bytebuddy.ByteBuddy;
 import net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
 import net.bytebuddy.implementation.MethodDelegation;
@@ -44,6 +43,7 @@ import org.apache.cassandra.distributed.api.Feature;
 import org.apache.cassandra.distributed.api.SimpleQueryResult;
 import org.apache.cassandra.distributed.test.TestBaseImpl;
 import org.apache.cassandra.service.accord.AccordService;
+import org.apache.cassandra.utils.AssertionUtils;
 import org.apache.cassandra.utils.FailingConsumer;
 
 import static net.bytebuddy.matcher.ElementMatchers.named;
@@ -90,10 +90,14 @@ public abstract class AccordTestBase extends TestBaseImpl
         // Evict commands from the cache immediately to expose problems loading from disk.
         sharedCluster.forEach(node -> node.runOnInstance(() -> AccordService.instance().setCacheSize(0)));
 
-        fn.accept(sharedCluster);
-
-        // Reset any messaging filters.
-        sharedCluster.filters().reset();
+        try
+        {
+            fn.accept(sharedCluster);
+        }
+        finally
+        {
+            sharedCluster.filters().reset();
+        }
     }
 
     protected void test(FailingConsumer<Cluster> fn) throws Exception
@@ -108,7 +112,7 @@ public abstract class AccordTestBase extends TestBaseImpl
         return init(Cluster.build(2)
                            .withoutVNodes()
                            .withConfig(c -> c.with(Feature.NETWORK).set("write_request_timeout", "10s").set("transaction_timeout", "15s"))
-                           .withInstanceInitializer(ByteBuddyHelper::install)
+                           .withInstanceInitializer(EnforceUpdateDoesNotPerformRead::install)
                            .start());
     }
 
@@ -128,20 +132,20 @@ public abstract class AccordTestBase extends TestBaseImpl
         }
         catch (Throwable t)
         {
-            if (Throwables.getRootCause(t).toString().contains(Preempted.class.getName()))
+            if (AssertionUtils.rootCauseIs(Preempted.class).matches(t))
                 return executeWithRetry(cluster, check, boundValues);
 
             throw t;
         }
     }
 
-    public static class ByteBuddyHelper
+    public static class EnforceUpdateDoesNotPerformRead
     {
         public static void install(ClassLoader classLoader, Integer num)
         {
             new ByteBuddy().rebase(ModificationStatement.class)
                            .method(named("readRequiredLists"))
-                           .intercept(MethodDelegation.to(ByteBuddyHelper.class))
+                           .intercept(MethodDelegation.to(EnforceUpdateDoesNotPerformRead.class))
                            .make()
                            .load(classLoader, ClassLoadingStrategy.Default.INJECTION);
         }
@@ -152,8 +156,7 @@ public abstract class AccordTestBase extends TestBaseImpl
             Map<?, ?> map = fn.call();
             if (map != null)
             {
-                // if the call tree has a TransactionStatement, then fail as this violates
-                // the query
+                // if the call tree has a TransactionStatement, then fail as this violates the query
                 for (StackTraceElement e : Thread.currentThread().getStackTrace())
                     if (TransactionStatement.class.getCanonicalName().equals(e.getClassName()))
                         throw new IllegalStateException("Attempted to load required partition!");
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index 2b72cd733b..39f9636a43 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -383,6 +383,7 @@ public class DatabaseDescriptorTest
         testConfig.cas_contention_timeout = lowerThanLowestTimeout;
         testConfig.counter_write_request_timeout = lowerThanLowestTimeout;
         testConfig.request_timeout = lowerThanLowestTimeout;
+        testConfig.transaction_timeout = lowerThanLowestTimeout;
 
         DatabaseDescriptor.checkForLowestAcceptedTimeouts(testConfig);
 
@@ -393,6 +394,7 @@ public class DatabaseDescriptorTest
         assertEquals(testConfig.cas_contention_timeout, DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
         assertEquals(testConfig.counter_write_request_timeout, DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
         assertEquals(testConfig.request_timeout, DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
+        assertEquals(testConfig.transaction_timeout, DatabaseDescriptor.LOWEST_ACCEPTED_TIMEOUT);
     }
 
     @Test
diff --git a/test/unit/org/apache/cassandra/utils/Generators.java b/test/unit/org/apache/cassandra/utils/Generators.java
index 51b583e749..d16b7bcfee 100644
--- a/test/unit/org/apache/cassandra/utils/Generators.java
+++ b/test/unit/org/apache/cassandra/utils/Generators.java
@@ -50,9 +50,12 @@ public final class Generators
 
     private static final Constraint DNS_DOMAIN_PARTS_CONSTRAINT = Constraint.between(1, 127);
 
+    private static final char CHAR_UNDERSCORE = 95;
+
     private static final char[] LETTER_DOMAIN = createLetterDomain();
     private static final Constraint LETTER_CONSTRAINT = Constraint.between(0, LETTER_DOMAIN.length - 1).withNoShrinkPoint();
     private static final char[] LETTER_OR_DIGIT_DOMAIN = createLetterOrDigitDomain();
+    private static final char[] LETTER_OR_DIGIT_DOMAIN_WITH_UNDERSCORE = createLetterOrDigitDomainWithUnderscore();
     private static final Constraint LETTER_OR_DIGIT_CONSTRAINT = Constraint.between(0, LETTER_OR_DIGIT_DOMAIN.length - 1).withNoShrinkPoint();
     private static final char[] REGEX_WORD_DOMAIN = createRegexWordDomain();
     private static final Constraint REGEX_WORD_CONSTRAINT = Constraint.between(0, REGEX_WORD_DOMAIN.length - 1).withNoShrinkPoint();
@@ -69,13 +72,10 @@ public final class Generators
         // see CASSANDRA-17919
         return !("P".equals(value) || "PT".equals(value));
     }
-    private static final char CHAR_UNDERSCORE = 95;
+
     public static Gen<String> symbolGen(Gen<Integer> size)
     {
-        char[] domain = new char[LETTER_OR_DIGIT_DOMAIN.length + 1];
-        System.arraycopy(LETTER_OR_DIGIT_DOMAIN, 0, domain, 0, LETTER_OR_DIGIT_DOMAIN.length);
-        domain[domain.length - 1] = CHAR_UNDERSCORE;
-        return string(size, domain, (index, c) -> !(index == 0 && !Character.isLetter(c)));
+        return string(size, LETTER_OR_DIGIT_DOMAIN_WITH_UNDERSCORE, (index, c) -> !(index == 0 && !Character.isLetter(c)));
     }
 
     public static final Gen<UUID> UUID_RANDOM_GEN = rnd -> {
@@ -324,6 +324,14 @@ public final class Generators
         return domain;
     }
 
+    private static char[] createLetterOrDigitDomainWithUnderscore()
+    {
+        char[] domain = new char[LETTER_OR_DIGIT_DOMAIN.length + 1];
+        System.arraycopy(LETTER_OR_DIGIT_DOMAIN, 0, domain, 0, LETTER_OR_DIGIT_DOMAIN.length);
+        domain[domain.length - 1] = CHAR_UNDERSCORE;
+        return domain;
+    }
+
     private static char[] createRegexWordDomain()
     {
         // \w == [a-zA-Z_0-9] the only difference with letterOrDigit is the addition of _


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org