This is an automated email from the ASF dual-hosted git repository. samt pushed a commit to branch cassandra-5.0 in repository https://gitbox.apache.org/repos/asf/cassandra.git
commit 332341b9b60c078914a1ca99d951f5127f43068f Merge: 184887069f 6f90e962f5 Author: Sam Tunnicliffe <s...@apache.org> AuthorDate: Mon Jan 13 09:50:38 2025 +0000 Merge branch 'cassandra-4.1' into cassandra-5.0 CHANGES.txt | 1 + src/java/org/apache/cassandra/config/Config.java | 1 + .../cassandra/config/DatabaseDescriptor.java | 52 +++++++ .../exceptions/OversizedCQLMessageException.java | 27 ++++ .../cassandra/net/AbstractMessageHandler.java | 4 +- .../cassandra/transport/CQLMessageHandler.java | 112 +++++++++++++- .../cassandra/transport/ExceptionHandlers.java | 5 + .../transport/InitialConnectionHandler.java | 3 +- .../cassandra/transport/PipelineConfigurator.java | 2 + .../apache/cassandra/transport/SimpleClient.java | 1 + .../transport/AuthMessageSizeLimitTest.java | 104 +++++++++++++ .../transport/ClientResourceLimitsTest.java | 105 ++++--------- .../cassandra/transport/MessageSizeLimitTest.java | 124 ++++++++++++++++ .../transport/NativeProtocolLimitsTestBase.java | 165 +++++++++++++++++++++ .../cassandra/transport/RateLimitingTest.java | 69 +++------ 15 files changed, 635 insertions(+), 140 deletions(-) diff --cc CHANGES.txt index 0d11571d81,45ef4fe379..46ab465be2 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,16 -1,9 +1,17 @@@ -4.1.8 +5.0.3 + * Remove v30 and v3X from 5.x in-JVM upgrade tests (CASSANDRA-20103) + * Avoid memory allocation in offheap_object's NativeCell.valueSize() and NativeClustering.dataSize() (CASSANDRA-20162) + * Add flag to avoid invalidating key cache on sstable deletions (CASSANDRA-20068) + * Interpret inet, bigint, varint, and decimal as non-reversed types for query construction and post-filtering (CASSANDRA-20100) + * Fix delayed gossip shutdown messages clobbering startup states that leave restarted nodes appearing down (CASSANDRA-20033) + * Streamline the serialized format for index status gossip messages (CASSANDRA-20058) + * Batch clusterings into single SAI partition post-filtering reads (CASSANDRA-19497) + * Ban the usage of "var" instead of full types in the production code (CASSANDRA-20038) + * Suppress CVE-2024-45772 from lucene-core-9.7.0.jar (CASSANDRA-20024) +Merged from 4.1: + * Enforce CQL message size limit on multiframe messages (CASSANDRA-20052) * Add nodetool checktokenmetadata command that checks TokenMetadata is insync with Gossip endpointState (CASSANDRA-18758) - * Backport Java 11 support for Simulator (CASSANDRA-17178/CASSANDRA-19935) * Equality check for Paxos.Electorate should not depend on collection types (CASSANDRA-19935) - * Fix race condition in DecayingEstimatedHistogramReservoir during rescale (CASSANDRA-19365) Merged from 4.0: * IndexOutOfBoundsException when accessing partition where the column was deleted (CASSANDRA-20108) * Enhance CQLSSTableWriter to notify clients on sstable production (CASSANDRA-19800) diff --cc src/java/org/apache/cassandra/config/Config.java index 8f6283ac1d,b64ff079de..6b4acc8f10 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@@ -282,8 -269,9 +282,9 @@@ public class Confi public int native_transport_max_threads = 128; @Replaces(oldName = "native_transport_max_frame_size_in_mb", converter = Converters.MEBIBYTES_DATA_STORAGE_INT, deprecated = true) public DataStorageSpec.IntMebibytesBound native_transport_max_frame_size = new DataStorageSpec.IntMebibytesBound("16MiB"); + /** do bcrypt hashing in a limited pool to prevent cpu load spikes; note: any value < 1 will be set to 1 on init **/ + public int native_transport_max_auth_threads = 4; + public volatile DataStorageSpec.LongBytesBound native_transport_max_message_size = null; - /** do bcrypt hashing in a limited pool to prevent cpu load spikes; 0 means that all requests will go to default request executor**/ - public int native_transport_max_auth_threads = 0; public volatile long native_transport_max_concurrent_connections = -1L; public volatile long native_transport_max_concurrent_connections_per_ip = -1L; public boolean native_transport_flush_in_batches_legacy = false; diff --cc src/java/org/apache/cassandra/transport/CQLMessageHandler.java index a2527a0cc8,792f6bf7b6..ae014eb185 --- a/src/java/org/apache/cassandra/transport/CQLMessageHandler.java +++ b/src/java/org/apache/cassandra/transport/CQLMessageHandler.java @@@ -752,10 -782,48 +791,48 @@@ public class CQLMessageHandler<M extend this.backpressure = backpressure; } + private void markTooBig() + { + this.tooBig = true; + } + + @Override + protected void onIntactFrame(IntactFrame frame) + { + if (tooBig || overload != Overload.NONE) + // we do not want to add the frame to buffers (to not consume a lot of memory and throw it away later + // we also do not want to release capacity because we haven't accuired it + frame.consume(); + else + super.onIntactFrame(frame); + } + + @Override + protected void onCorruptFrame() + { + if (!isExpired && !isCorrupt && !tooBig) + { + releaseBuffers(); // release resources once we transition from normal state to corrupt + if (overload != Overload.BYTES_IN_FLIGHT) + releaseCapacity(size); + } + isCorrupt = true; + isExpired |= approxTime.isAfter(expiresAtNanos); + } + + + @Override protected void onComplete() { - if (overload != Overload.NONE) - handleErrorAndRelease(buildOverloadedException(endpointReserveCapacity, globalReserveCapacity, requestRateLimiter, overload), header); + if (tooBig) + // we haven't accuired a capacity for too big messages to release it + handleError(buildOversizedCQLMessageException(header.bodySizeInBytes), header); + else if (overload != Overload.NONE) + if (overload == Overload.BYTES_IN_FLIGHT) + // we haven't accuired a capacity successfully to release it - handleError(buildOverloadedException(endpointReserveCapacity, globalReserveCapacity, overload), header); ++ handleError(buildOverloadedException(endpointReserveCapacity, globalReserveCapacity, requestRateLimiter, overload), header); + else - handleErrorAndRelease(buildOverloadedException(endpointReserveCapacity, globalReserveCapacity, overload), header); ++ handleErrorAndRelease(buildOverloadedException(endpointReserveCapacity, globalReserveCapacity, requestRateLimiter, overload), header); else if (!isCorrupt) processRequest(assembleFrame(), backpressure); } --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org