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

konstantinov pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 00bf5380fe Reduce memory allocation during transformation of 
BatchStatement to Mutation
00bf5380fe is described below

commit 00bf5380fe83dd8919e8f286edd2e6a65786ca6e
Author: Dmitry Konstantinov <[email protected]>
AuthorDate: Wed Jan 28 12:34:20 2026 +0000

    Reduce memory allocation during transformation of BatchStatement to Mutation
    
    avoid ClusteringIndexSliceFilter allocation if a write does not required a 
read (plain usual write), avoid iterator allocation, use array instead of 
ArrayList for perStatementOptions which does not grow dynamically
    BatchStatement: check if many similar rows for the same table are written 
unconditionally, in this case we can avoid columns info merging and builders 
allocation
    add fast path in nonTokenRestrictionValues logic, when we specify a single 
partition key (a single row) to modify, optimize also the case if a partition 
or clustering key is a single column
    add fast path in valuesAsClustering logic, when we specify a single 
clustering key (a single row) to modify
    UpdateParameters: allocate DeletionTime on demand (it is not needed if we 
do insert/updates)
    serializedRowBodySize: avoid capturing lamda allocation per cell by moving 
capturing arguments to SerializationHelper (same optimization as it was done in 
serializeRowBody)
    force hash3_x64_128 inlining to help JIT with escape analysis and long[] 
heap allocation elimination
    
    patch by Dmitry Konstantinov; reviewed by Chris Lohfink for CASSANDRA-21141
---
 CHANGES.txt                                        |  1 +
 .../apache/cassandra/cql3/BatchQueryOptions.java   | 13 ++---
 .../apache/cassandra/cql3/UpdateParameters.java    | 15 +++---
 .../restrictions/ClusteringColumnRestrictions.java |  8 ++++
 .../restrictions/PartitionKeyRestrictions.java     | 56 ++++++++++++++++++++++
 .../cql3/restrictions/RestrictionSet.java          |  7 ++-
 .../cql3/restrictions/SimpleRestriction.java       | 19 ++++++++
 .../cassandra/cql3/statements/BatchStatement.java  | 55 +++++++++++++++++----
 .../cql3/statements/ModificationStatement.java     | 29 ++++++-----
 .../org/apache/cassandra/cql3/terms/Marker.java    | 27 +++++++++++
 .../org/apache/cassandra/cql3/terms/Terms.java     | 24 ++++++++++
 .../org/apache/cassandra/db/MultiCBuilder.java     |  5 ++
 .../cassandra/db/rows/SerializationHelper.java     |  3 +-
 .../cassandra/db/rows/UnfilteredSerializer.java    | 24 ++++++----
 .../apache/cassandra/dht/Murmur3Partitioner.java   |  4 ++
 .../org/apache/cassandra/utils/MurmurHash.java     |  6 +++
 16 files changed, 251 insertions(+), 45 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 3233ffe7bf..016aa739f8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 5.1
+ * Reduce memory allocation during transformation of BatchStatement to 
Mutation (CASSANDRA-21141)
  * Direct I/O support for compaction reads (CASSANDRA-19987)
  * Support custom StartupCheck implementations via SPI (CASSANDRA-21093)
  * Make sstableexpiredblockers support human-readable output with SSTable 
sizes (CASSANDRA-20448)
diff --git a/src/java/org/apache/cassandra/cql3/BatchQueryOptions.java 
b/src/java/org/apache/cassandra/cql3/BatchQueryOptions.java
index 13143453bb..ef767aa396 100644
--- a/src/java/org/apache/cassandra/cql3/BatchQueryOptions.java
+++ b/src/java/org/apache/cassandra/cql3/BatchQueryOptions.java
@@ -152,19 +152,20 @@ public abstract class BatchQueryOptions
 
     private static class WithPerStatementVariables extends BatchQueryOptions
     {
-        private final List<QueryOptions> perStatementOptions;
+        private final QueryOptions[] perStatementOptions;
 
         private WithPerStatementVariables(QueryOptions wrapped, List<byte[][]> 
variables, List<Object> queryOrIdList)
         {
             super(wrapped, queryOrIdList);
-            this.perStatementOptions = new ArrayList<>(variables.size());
+            this.perStatementOptions = new QueryOptions[variables.size()];
+            int i = 0;
             for (final byte[][] vars : variables)
-                perStatementOptions.add(new BatchQueryOptionsWrapper(wrapped, 
vars));
+                perStatementOptions[i++] = new 
BatchQueryOptionsWrapper(wrapped, vars);
         }
 
         public QueryOptions forStatement(int i)
         {
-            return perStatementOptions.get(i);
+            return perStatementOptions[i];
         }
 
         @Override
@@ -172,10 +173,10 @@ public abstract class BatchQueryOptions
         {
             if (isPreparedStatement(i))
             {
-                QueryOptions options = perStatementOptions.get(i);
+                QueryOptions options = perStatementOptions[i];
                 options.prepare(boundNames);
                 options = QueryOptions.addColumnSpecifications(options, 
boundNames);
-                perStatementOptions.set(i, options);
+                perStatementOptions[i] = options;
             }
             else
             {
diff --git a/src/java/org/apache/cassandra/cql3/UpdateParameters.java 
b/src/java/org/apache/cassandra/cql3/UpdateParameters.java
index a0a51a3829..f7a5cf1e37 100644
--- a/src/java/org/apache/cassandra/cql3/UpdateParameters.java
+++ b/src/java/org/apache/cassandra/cql3/UpdateParameters.java
@@ -58,7 +58,7 @@ public class UpdateParameters
     protected final long timestamp;
     private final int ttl;
 
-    private final DeletionTime deletionTime;
+    private DeletionTime deletionTime;
 
     // Holds data for operations that require a read-before-write. Will be 
null otherwise.
     private final Map<DecoratedKey, Partition> prefetchedRows;
@@ -82,8 +82,6 @@ public class UpdateParameters
         this.timestamp = timestamp;
         this.ttl = ttl;
 
-        this.deletionTime = DeletionTime.build(timestamp, nowInSec);
-
         this.prefetchedRows = prefetchedRows;
 
         // We use MIN_VALUE internally to mean the absence of of timestamp (in 
Selection, in sstable stats, ...), so exclude
@@ -128,7 +126,7 @@ public class UpdateParameters
 
     public void addRowDeletion()
     {
-        addRowDeletion(Row.Deletion.regular(deletionTime));
+        addRowDeletion(Row.Deletion.regular(deletionTime()));
     }
 
     private void addRowDeletion(Row.Deletion deletion)
@@ -266,11 +264,12 @@ public class UpdateParameters
 
     public void setComplexDeletionTime(ColumnMetadata column)
     {
-        builder.addComplexDeletion(column, deletionTime);
+        builder.addComplexDeletion(column, deletionTime());
     }
 
     public void setComplexDeletionTimeForOverwrite(ColumnMetadata column)
     {
+        DeletionTime deletionTime = deletionTime();
         builder.addComplexDeletion(column, 
DeletionTime.build(deletionTime.markedForDeleteAt() - 1, 
deletionTime.localDeletionTime()));
     }
 
@@ -283,7 +282,9 @@ public class UpdateParameters
 
     public DeletionTime deletionTime()
     {
-        return deletionTime;
+         if (deletionTime == null)
+             deletionTime = DeletionTime.build(timestamp, nowInSec);
+         return deletionTime;
     }
 
     public RangeTombstone makeRangeTombstone(ClusteringComparator comparator, 
Clustering<?> clustering)
@@ -293,7 +294,7 @@ public class UpdateParameters
 
     public RangeTombstone makeRangeTombstone(Slice slice)
     {
-        return new RangeTombstone(slice, deletionTime);
+        return new RangeTombstone(slice, deletionTime());
     }
 
     public byte[] nextTimeUUIDAsBytes()
diff --git 
a/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java
 
b/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java
index 93b93a7c81..4d12be8158 100644
--- 
a/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java
+++ 
b/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java
@@ -104,6 +104,14 @@ final class ClusteringColumnRestrictions extends 
RestrictionSetWrapper
 
     public NavigableSet<Clustering<?>> valuesAsClustering(QueryOptions 
options, ClientState state) throws InvalidRequestException
     {
+        // fast path, a typical case when a single full restriction is used
+        // for example, when we specify a single clustering key (a single row) 
to insert/update
+        if (restrictions.size() == 1 && !restrictions.hasIN())
+        {
+            SingleRestriction r = restrictions.lastRestriction();
+            List<ClusteringElements> values = r.values(options);
+            return MultiCBuilder.build(comparator, values);
+        }
         MultiCBuilder builder = new MultiCBuilder(comparator);
         for (SingleRestriction r : restrictions)
         {
diff --git 
a/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeyRestrictions.java 
b/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeyRestrictions.java
index c2b565196d..a19e91f03c 100644
--- 
a/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeyRestrictions.java
+++ 
b/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeyRestrictions.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.cql3.restrictions;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 import java.util.SortedSet;
@@ -35,6 +36,8 @@ import org.apache.cassandra.db.ClusteringPrefix;
 import org.apache.cassandra.db.MultiCBuilder;
 import org.apache.cassandra.db.PartitionPosition;
 import org.apache.cassandra.db.guardrails.Guardrails;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
+import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.Bounds;
 import org.apache.cassandra.dht.IPartitioner;
@@ -43,6 +46,7 @@ import org.apache.cassandra.dht.Token.TokenFactory;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.utils.FBUtilities;
 
 /**
  * A set of restrictions on the partition key.
@@ -191,6 +195,22 @@ final class PartitionKeyRestrictions extends 
RestrictionSetWrapper
      */
     private List<ByteBuffer> nonTokenRestrictionValues(QueryOptions options, 
ClientState state)
     {
+        // fast path, a typical case when a single full restriction is used
+        // for example, when we specify a single partition key to modify
+        if (restrictions.size() == 1 && !restrictions.hasIN())
+        {
+            SingleRestriction r = restrictions.lastRestriction();
+            List<ClusteringElements> values = r.values(options);
+            if (values.size() == 1)
+            {
+                ClusteringElements elements = values.get(0);
+                validatePartitionKey(elements);
+                ByteBuffer pk = serializeAsPartitionKey(elements);
+                return Collections.singletonList(pk);
+            }
+            return toByteBuffers(MultiCBuilder.build(comparator, values));
+        }
+
         MultiCBuilder builder = new MultiCBuilder(comparator);
         for (SingleRestriction r : restrictions)
         {
@@ -205,8 +225,44 @@ final class PartitionKeyRestrictions extends 
RestrictionSetWrapper
         return toByteBuffers(builder.build());
     }
 
+    private ByteBuffer serializeAsPartitionKey(ClusteringElements elements)
+    {
+        // Single-column partition key: just return the value directly
+        if (elements.size() == 1)
+            return elements.get(0);
+
+        // Composite partition key: need to build composite
+        return CompositeType.build(ByteBufferAccessor.instance, 
elements.toArray(new ByteBuffer[elements.size()]));
+    }
+
+    // repeats the logic of ClusteringPrefix.validate()
+    private void validatePartitionKey(ClusteringElements partitionKey)
+    {
+        int sum = 0;
+        for (ByteBuffer columnValue : partitionKey)
+        {
+            int size = columnValue != null ? columnValue.remaining() : 0;
+            if (size > FBUtilities.MAX_UNSIGNED_SHORT)
+                throw new InvalidRequestException(String.format("Key length of 
%d is longer than maximum of %d",
+                                                                size,
+                                                                
FBUtilities.MAX_UNSIGNED_SHORT));
+            sum += size;
+        }
+        if (sum > FBUtilities.MAX_UNSIGNED_SHORT)
+            throw new InvalidRequestException(String.format("Key length of %d 
is longer than maximum of %d",
+                                                            sum,
+                                                            
FBUtilities.MAX_UNSIGNED_SHORT));
+    }
+
     private List<ByteBuffer> toByteBuffers(SortedSet<? extends 
ClusteringPrefix<?>> clusterings)
     {
+        if (clusterings.size() == 1)
+        {
+            ClusteringPrefix<?> clustering = clusterings.first();
+            clustering.validate();
+            return 
Collections.singletonList(clustering.serializeAsPartitionKey());
+        }
+
         List<ByteBuffer> l = new ArrayList<>(clusterings.size());
         for (ClusteringPrefix<?> clustering : clusterings)
         {
diff --git 
a/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java 
b/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java
index 9bd46f2afd..d7b463b762 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java
@@ -70,6 +70,9 @@ final class RestrictionSet implements Restrictions, 
Iterable<SingleRestriction>
      */
     private final NavigableMap<ColumnMetadata, SingleRestriction> restrictions;
 
+    private final SingleRestriction lastRestriction;
+
+
     /**
      * {@code true} if it contains multi-column restrictions, {@code false} 
otherwise.
      */
@@ -100,6 +103,8 @@ final class RestrictionSet implements Restrictions, 
Iterable<SingleRestriction>
                            boolean needsFilteringOrIndexing)
     {
         this.restrictions = restrictions;
+        // Map.lastEntry allocates an object, so we cache the value to avoid 
it, restrictions is immutable
+        this.lastRestriction = restrictions.isEmpty() ? null : 
restrictions.lastEntry().getValue();
         this.hasMultiColumnRestrictions = hasMultiColumnRestrictions;
         this.hasIn = hasIn;
         this.hasSlice = hasSlice;
@@ -312,7 +317,7 @@ final class RestrictionSet implements Restrictions, 
Iterable<SingleRestriction>
      */
     SingleRestriction lastRestriction()
     {
-        return restrictions.lastEntry().getValue();
+        return lastRestriction;
     }
 
     /**
diff --git 
a/src/java/org/apache/cassandra/cql3/restrictions/SimpleRestriction.java 
b/src/java/org/apache/cassandra/cql3/restrictions/SimpleRestriction.java
index 16141b8b8c..c5e05560a2 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/SimpleRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/SimpleRestriction.java
@@ -258,10 +258,22 @@ public final class SimpleRestriction implements 
SingleRestriction
 
     private List<ClusteringElements> 
bindAndGetSingleTermClusteringElements(QueryOptions options)
     {
+        if (values.isSingleTerm(options))
+        {
+            ByteBuffer value = bindAndGetSingle(options);
+            return 
Collections.singletonList(ClusteringElements.of(columnsExpression.columnSpecification(),
 value, isOnToken()));
+        }
+
         List<ByteBuffer> values = bindAndGet(options);
         if (values.isEmpty())
             return Collections.emptyList();
 
+        if (values.size() == 1)
+        {
+            ClusteringElements value = 
ClusteringElements.of(columnsExpression.columnSpecification(), values.get(0), 
isOnToken());
+            return Collections.singletonList(value);
+        }
+
         List<ClusteringElements> elements = new ArrayList<>(values.size());
         for (int i = 0; i < values.size(); i++)
             
elements.add(ClusteringElements.of(columnsExpression.columnSpecification(), 
values.get(i), isOnToken()));
@@ -288,6 +300,13 @@ public final class SimpleRestriction implements 
SingleRestriction
         return buffers;
     }
 
+    private ByteBuffer bindAndGetSingle(QueryOptions options)
+    {
+        ByteBuffer buffer = values.bindAndGetSingleTermValue(options);
+        validate(buffer);
+        return buffer;
+    }
+
     private List<List<ByteBuffer>> bindAndGetElements(QueryOptions options)
     {
         List<List<ByteBuffer>> elementsList = 
values.bindAndGetElements(options);
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java 
b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index 39d36aa4a5..8a38387d88 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -142,27 +142,62 @@ public class BatchStatement implements 
CQLStatement.CompositeCQLStatement
         this.attrs = attrs;
 
         boolean hasConditions = false;
-        MultiTableColumnsBuilder regularBuilder = new 
MultiTableColumnsBuilder();
-        RegularAndStaticColumns.Builder conditionBuilder = 
RegularAndStaticColumns.builder();
         boolean updateRegular = false;
         boolean updateStatic = false;
         boolean updatesVirtualTables = false;
 
+        boolean sameTableAndColumnsNoConditions = true;
+        TableMetadata tableMetadata = null;
+        RegularAndStaticColumns regularAndStaticColumns = null;
+        // we check initially if it is a typical scenario:
+        // when many similar rows for the same table are written 
unconditionally
+        // in this case we can avoid columns info merging and builders 
allocation
         for (ModificationStatement stmt : statements)
         {
-            regularBuilder.addAll(stmt.metadata(), stmt.updatedColumns());
+            if (tableMetadata == null)
+                tableMetadata = stmt.metadata();
+            if (regularAndStaticColumns == null)
+                regularAndStaticColumns = stmt.updatedColumns();
+
+            if (tableMetadata != stmt.metadata()
+                || regularAndStaticColumns != stmt.updatedColumns()
+                || stmt.hasConditions())
+            {
+                sameTableAndColumnsNoConditions = false;
+                break;
+            }
+
             updateRegular |= stmt.updatesRegularRows();
             updatesVirtualTables |= stmt.isVirtual();
-            if (stmt.hasConditions())
+            updateStatic |= stmt.updatesStaticRow();
+        }
+
+        if (sameTableAndColumnsNoConditions && tableMetadata != null)
+        {
+            this.updatedColumns = Collections.singletonMap(tableMetadata.id(), 
regularAndStaticColumns);
+            this.conditionColumns = RegularAndStaticColumns.NONE;
+        }
+        else
+        {
+
+            MultiTableColumnsBuilder regularBuilder = new 
MultiTableColumnsBuilder();
+            RegularAndStaticColumns.Builder conditionBuilder = 
RegularAndStaticColumns.builder();
+            for (ModificationStatement stmt : statements)
             {
-                hasConditions = true;
-                conditionBuilder.addAll(stmt.conditionColumns());
-                updateStatic |= stmt.updatesStaticRow();
+                regularBuilder.addAll(stmt.metadata(), stmt.updatedColumns());
+                updateRegular |= stmt.updatesRegularRows();
+                updatesVirtualTables |= stmt.isVirtual();
+                if (stmt.hasConditions())
+                {
+                    hasConditions = true;
+                    conditionBuilder.addAll(stmt.conditionColumns());
+                    updateStatic |= stmt.updatesStaticRow();
+                }
             }
-        }
 
-        this.updatedColumns = regularBuilder.build();
-        this.conditionColumns = conditionBuilder.build();
+            this.updatedColumns = regularBuilder.build();
+            this.conditionColumns = conditionBuilder.build();
+        }
         this.updatesRegularRows = updateRegular;
         this.updatesStaticRow = updateStatic;
         this.hasConditions = hasConditions;
diff --git 
a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java 
b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 118f2c1fa4..fd853004be 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -521,8 +521,8 @@ public abstract class ModificationStatement implements 
CQLStatement.SingleKeyspa
     throws InvalidRequestException
     {
         List<ByteBuffer> partitionKeys = 
restrictions.getPartitionKeys(options, state);
-        for (ByteBuffer key : partitionKeys)
-            QueryProcessor.validateKey(key);
+        for (int i = 0; i < partitionKeys.size(); i++)
+            QueryProcessor.validateKey(partitionKeys.get(i));
 
         return partitionKeys;
     }
@@ -567,8 +567,9 @@ public abstract class ModificationStatement implements 
CQLStatement.SingleKeyspa
         return isReadRequired;
     }
 
-    private Map<DecoratedKey, Partition> 
readRequiredLists(Collection<ByteBuffer> partitionKeys,
-                                                           
ClusteringIndexFilter filter,
+    private <F> Map<DecoratedKey, Partition> 
readRequiredLists(Collection<ByteBuffer> partitionKeys,
+                                                           
java.util.function.Function<F, ClusteringIndexFilter> filterBuilder,
+                                                           F filterArg,
                                                            DataLimits limits,
                                                            boolean local,
                                                            ConsistencyLevel cl,
@@ -595,7 +596,7 @@ public abstract class ModificationStatement implements 
CQLStatement.SingleKeyspa
                                                            RowFilter.none(),
                                                            limits,
                                                            
metadata().partitioner.decorateKey(key),
-                                                           filter));
+                                                           
filterBuilder.apply(filterArg)));
 
         SinglePartitionReadCommand.Group group = 
SinglePartitionReadCommand.Group.create(commands, DataLimits.NONE);
 
@@ -1024,7 +1025,8 @@ public abstract class ModificationStatement implements 
CQLStatement.SingleKeyspa
                 return;
 
             UpdateParameters params = makeUpdateParameters(keys,
-                                                           new 
ClusteringIndexSliceFilter(slices, false),
+                                                           (slicesToFilter) -> 
new ClusteringIndexSliceFilter(slicesToFilter, false),
+                                                           slices,
                                                            state,
                                                            options,
                                                            DataLimits.NONE,
@@ -1117,7 +1119,8 @@ public abstract class ModificationStatement implements 
CQLStatement.SingleKeyspa
     {
         if (clusterings.contains(Clustering.STATIC_CLUSTERING))
             return makeUpdateParameters(keys,
-                                        new 
ClusteringIndexSliceFilter(Slices.ALL, false),
+                                        (clusteringsToFilter) -> new 
ClusteringIndexSliceFilter(Slices.ALL, false),
+                                        clusterings,
                                         state,
                                         options,
                                         DataLimits.cqlLimits(1),
@@ -1128,7 +1131,8 @@ public abstract class ModificationStatement implements 
CQLStatement.SingleKeyspa
             );
 
         return makeUpdateParameters(keys,
-                                    new 
ClusteringIndexNamesFilter(clusterings, false),
+                                    (clusteringsToFilter) -> new 
ClusteringIndexNamesFilter(clusteringsToFilter, false),
+                                    clusterings,
                                     state,
                                     options,
                                     DataLimits.NONE,
@@ -1139,8 +1143,10 @@ public abstract class ModificationStatement implements 
CQLStatement.SingleKeyspa
         );
     }
 
-    private UpdateParameters makeUpdateParameters(Collection<ByteBuffer> keys,
-                                                  ClusteringIndexFilter filter,
+    private <F> UpdateParameters makeUpdateParameters(Collection<ByteBuffer> 
keys,
+                                                  // filter is needed rarely, 
so we allocate it on demand
+                                                  
java.util.function.Function<F, ClusteringIndexFilter> filterBuilder,
+                                                  F filterArg,
                                                   ClientState state,
                                                   QueryOptions options,
                                                   DataLimits limits,
@@ -1152,7 +1158,8 @@ public abstract class ModificationStatement implements 
CQLStatement.SingleKeyspa
         // Some lists operation requires reading
         Map<DecoratedKey, Partition> lists =
             readRequiredLists(keys,
-                              filter,
+                              filterBuilder,
+                              filterArg,
                               limits,
                               local,
                               options.getConsistency(),
diff --git a/src/java/org/apache/cassandra/cql3/terms/Marker.java 
b/src/java/org/apache/cassandra/cql3/terms/Marker.java
index 26ff845987..d36e81bdfd 100644
--- a/src/java/org/apache/cassandra/cql3/terms/Marker.java
+++ b/src/java/org/apache/cassandra/cql3/terms/Marker.java
@@ -95,6 +95,33 @@ public final class Marker extends Term.NonTerminal
         }
     }
 
+    // an optimized version without allocating interim Terminal objects
+    @Override
+    public ByteBuffer bindAndGet(QueryOptions options)
+    {
+        try
+        {
+            ByteBuffer bytes = options.getValue(bindIndex);
+            if (bytes == null)
+                return null;
+
+            if (bytes == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                return ByteBufferUtil.UNSET_BYTE_BUFFER;
+
+            if (receiver.type instanceof MultiElementType<?>)
+            {
+                return MultiElements.Value.fromSerialized(bytes, 
(MultiElementType<?>) receiver.type).get();
+            }
+
+            receiver.type.validate(bytes);
+            return bytes;
+        }
+        catch (MarshalException e)
+        {
+            throw new InvalidRequestException(e.getMessage(), e);
+        }
+    }
+
     public boolean isByteArrayGetSupported(QueryOptions options)
     {
         return options.isByteArrayValuesGetSupported();
diff --git a/src/java/org/apache/cassandra/cql3/terms/Terms.java 
b/src/java/org/apache/cassandra/cql3/terms/Terms.java
index 3f411f901f..10f0c5d137 100644
--- a/src/java/org/apache/cassandra/cql3/terms/Terms.java
+++ b/src/java/org/apache/cassandra/cql3/terms/Terms.java
@@ -116,6 +116,17 @@ public interface Terms
      */
     List<ByteBuffer> bindAndGet(QueryOptions options);
 
+    default boolean isSingleTerm(QueryOptions options)
+    {
+        return false;
+    }
+
+    default ByteBuffer bindAndGetSingleTermValue(QueryOptions options)
+    {
+        throw new IllegalStateException("bindAndGetSingleTermValue() method is 
not implemented, " +
+                                        "isSingleTerm() must be always checked 
before invoking this method");
+    }
+
     /**
      * A shorter for {@code bind(options).getElements()}.
      * We expose it mainly because for constants it can avoid allocating a 
temporary
@@ -627,6 +638,19 @@ public interface Terms
                     return Collections.singletonList(term.bindAndGet(options));
                 }
 
+                @Override
+                public boolean isSingleTerm(QueryOptions options)
+                {
+                    return true;
+                }
+
+                @Override
+                public ByteBuffer bindAndGetSingleTermValue(QueryOptions 
options)
+                {
+                    return term.bindAndGet(options);
+                }
+
+
                 @Override
                 public List<List<ByteBuffer>> bindAndGetElements(QueryOptions 
options)
                 {
diff --git a/src/java/org/apache/cassandra/db/MultiCBuilder.java 
b/src/java/org/apache/cassandra/db/MultiCBuilder.java
index 6e9cf5cd5b..ef50d79534 100644
--- a/src/java/org/apache/cassandra/db/MultiCBuilder.java
+++ b/src/java/org/apache/cassandra/db/MultiCBuilder.java
@@ -191,6 +191,11 @@ public final class MultiCBuilder
         if (hasMissingElements)
             return BTreeSet.empty(comparator);
 
+        return build(comparator, clusterings);
+    }
+
+    public static NavigableSet<Clustering<?>> build(ClusteringComparator 
comparator, List<ClusteringElements> clusterings)
+    {
         if (clusterings.isEmpty())
             return BTreeSet.of(comparator, Clustering.EMPTY);
 
diff --git a/src/java/org/apache/cassandra/db/rows/SerializationHelper.java 
b/src/java/org/apache/cassandra/db/rows/SerializationHelper.java
index ff097111aa..1ba8ebd4fc 100644
--- a/src/java/org/apache/cassandra/db/rows/SerializationHelper.java
+++ b/src/java/org/apache/cassandra/db/rows/SerializationHelper.java
@@ -39,12 +39,13 @@ public class SerializationHelper
     private BTreeSearchIterator<ColumnMetadata, ColumnMetadata> regulars = 
null;
 
     // reusable fields to avoid extra allocation during cells processing
-    // within 
org.apache.cassandra.db.rows.UnfilteredSerializer.serializeRowBody
+    // within 
org.apache.cassandra.db.rows.UnfilteredSerializer.serializeRowBody and 
serializedRowBodySize
     int flags;
     LivenessInfo pkLiveness;
 
     DataOutputPlus out;
     SearchIterator<ColumnMetadata, ColumnMetadata> si;
+    boolean hasComplexDeletion;
 
     public SerializationHelper(SerializationHeader header)
     {
diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java 
b/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
index 02e235bab9..112c1e4c6b 100644
--- a/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
+++ b/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
@@ -369,18 +369,24 @@ public class UnfilteredSerializer
             size += Columns.serializer.serializedSubsetSize(row.columns(), 
header.columns(isStatic));
 
         SearchIterator<ColumnMetadata, ColumnMetadata> si = 
helper.iterator(isStatic);
-        return row.accumulate((data, v) -> {
-            ColumnMetadata column = si.next(data.column());
-            assert column != null;
+        helper.si = si;
+        helper.pkLiveness = pkLiveness;
+        helper.hasComplexDeletion = hasComplexDeletion;
+        return row.accumulate(UnfilteredSerializer::serializedColumnDataSize, 
helper, size);
+    }
 
-            if (data.column.isSimple())
-                return v + Cell.serializer.serializedSize((Cell<?>) data, 
column, pkLiveness, header);
-            else
-                return v + sizeOfComplexColumn((ComplexColumnData) data, 
column, hasComplexDeletion, pkLiveness, header);
-        }, size);
+    private static long serializedColumnDataSize(SerializationHelper helper, 
ColumnData data, long v)
+    {
+        ColumnMetadata column = helper.si.next(data.column());
+        assert column != null;
+
+        if (data.column.isSimple())
+            return v + Cell.serializer.serializedSize((Cell<?>) data, column, 
helper.pkLiveness, helper.header);
+        else
+            return v + sizeOfComplexColumn((ComplexColumnData) data, column, 
helper.hasComplexDeletion, helper.pkLiveness, helper.header);
     }
 
-    private long sizeOfComplexColumn(ComplexColumnData data, ColumnMetadata 
column, boolean hasComplexDeletion, LivenessInfo rowLiveness, 
SerializationHeader header)
+    private static long sizeOfComplexColumn(ComplexColumnData data, 
ColumnMetadata column, boolean hasComplexDeletion, LivenessInfo rowLiveness, 
SerializationHeader header)
     {
         long size = 0;
 
diff --git a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java 
b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
index 1919d530d2..361a387046 100644
--- a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
+++ b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
@@ -34,6 +34,8 @@ import java.util.function.Function;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.primitives.Longs;
 
+import net.nicoulaj.compilecommand.annotations.Inline;
+
 import accord.primitives.Ranges;
 
 import org.apache.cassandra.db.DecoratedKey;
@@ -398,6 +400,7 @@ public class Murmur3Partitioner implements IPartitioner
         return value ^ 0x8000000000000000L;
     }
 
+    @Inline // inline to help JIT with escape analysis and long[] heap 
allocation elimination
     private long[] getHash(ByteBuffer key)
     {
         long[] hash = new long[2];
@@ -405,6 +408,7 @@ public class Murmur3Partitioner implements IPartitioner
         return hash;
     }
 
+    @Inline // inline to help JIT with escape analysis and long[] heap 
allocation elimination
     private void populateHash(ByteBuffer key, long[] hash)
     {
         MurmurHash.hash3_x64_128(key, key.position(), key.remaining(), 0, 
hash);
diff --git a/src/java/org/apache/cassandra/utils/MurmurHash.java 
b/src/java/org/apache/cassandra/utils/MurmurHash.java
index e5d488d663..79f38b9917 100644
--- a/src/java/org/apache/cassandra/utils/MurmurHash.java
+++ b/src/java/org/apache/cassandra/utils/MurmurHash.java
@@ -22,6 +22,8 @@ import java.util.BitSet;
 
 import com.google.common.primitives.Longs;
 
+import net.nicoulaj.compilecommand.annotations.Inline;
+
 /**
  * This is a very fast, non-cryptographic hash suitable for general hash-based
  * lookup. See http://murmurhash.googlepages.com/ for more details.
@@ -245,6 +247,10 @@ public class MurmurHash
         return k;
     }
 
+
+    // inline mostly to help JIT with escape analysis and long[] heap 
allocation elimination
+    // by default this method is not inlined as too big
+    @Inline
     public static void hash3_x64_128(ByteBuffer key, int offset, int length, 
long seed, long[] result)
     {
         final int nblocks = length >> 4; // Process as 128-bit blocks.


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


Reply via email to