krummas commented on code in PR #4556:
URL: https://github.com/apache/cassandra/pull/4556#discussion_r2883824168


##########
src/java/org/apache/cassandra/service/thresholds/ThresholdCounter.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.service.thresholds;
+
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.locator.InetAddressAndPort;
+
+/**
+ * Immutable counter tracking which replicas reported warnings/aborts and the 
maximum value.
+ * Shared by both read and write threshold tracking.

Review Comment:
   looks like this is only used by read thresholds, should it move back to an 
inner class?



##########
src/java/org/apache/cassandra/service/writes/thresholds/WriteThresholdCounter.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.service.writes.thresholds;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.cassandra.schema.TableId;
+
+public class WriteThresholdCounter
+{
+    private static final WriteThresholdCounter EMPTY = new 
WriteThresholdCounter(Collections.emptyMap());
+    public final Map<TableId, Long> tableValues;

Review Comment:
   since this is `public` it should probably be an `ImmutableMap` to make it 
clear we allow no external changes



##########
src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java:
##########
@@ -79,9 +84,15 @@ public void onResponse(Message<T> message)
     {
         try
         {
+            InetAddressAndPort from = message == null ? 
FBUtilities.getBroadcastAddressAndPort() : message.from();

Review Comment:
   unnecessary, there is a null check for this below



##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -5482,6 +5470,40 @@ public static void 
setRowIndexReadSizeFailThreshold(@Nullable DataStorageSpec.Lo
         conf.row_index_read_size_fail_threshold = value;
     }
 
+    public static boolean getWriteThresholdsEnabled()
+    {
+        return conf.write_thresholds_enabled;
+    }
+
+    public static void setWriteThresholdsEnabled(boolean enabled)
+    {
+        logger.info("updating write_thresholds_enabled to {}", enabled);
+        conf.write_thresholds_enabled = enabled;
+    }
+
+    @Nullable
+    public static DataStorageSpec.LongBytesBound getWriteSizeWarnThreshold()
+    {
+        return conf.write_size_warn_threshold;
+    }
+
+    public static void setWriteSizeWarnThreshold(@Nullable 
DataStorageSpec.LongBytesBound value)
+    {
+        logger.info("updating write_size_warn_threshold to {}", value);
+        conf.write_size_warn_threshold = value;
+    }
+
+    public static int getWriteTombstoneWarnThreshold()
+    {
+        return conf.write_tombstone_warn_threshold;
+    }
+
+    public static void setWriteTombstoneWarnThreshold(int threshold)
+    {
+        logger.info("updating write_tombstone_warn_threshold to {}", 
threshold);
+        conf.write_tombstone_warn_threshold = threshold;

Review Comment:
   nit; check that `threshold > min_tracked_partition_tombstone_count`



##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -5482,6 +5470,40 @@ public static void 
setRowIndexReadSizeFailThreshold(@Nullable DataStorageSpec.Lo
         conf.row_index_read_size_fail_threshold = value;
     }
 
+    public static boolean getWriteThresholdsEnabled()
+    {
+        return conf.write_thresholds_enabled;
+    }
+
+    public static void setWriteThresholdsEnabled(boolean enabled)
+    {
+        logger.info("updating write_thresholds_enabled to {}", enabled);
+        conf.write_thresholds_enabled = enabled;

Review Comment:
   nit; verify that `top_partitions_enabled` is true



##########
src/java/org/apache/cassandra/service/writes/thresholds/WarnCounter.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.service.writes.thresholds;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.cassandra.schema.TableId;
+
+public class WarnCounter
+{
+    private final ConcurrentHashMap<TableId, AtomicLong> tableValues = new 
ConcurrentHashMap<>();
+
+    void addWarning(Map<TableId, Long> incoming)
+    {
+        for (Map.Entry<TableId, Long> entry : incoming.entrySet())
+            tableValues.computeIfAbsent(entry.getKey(), k -> new AtomicLong())
+                       .accumulateAndGet(entry.getValue(), Math::max);
+    }
+
+    public WriteThresholdCounter snapshot()
+    {
+        Map<TableId, Long> copy = new ConcurrentHashMap<>();

Review Comment:
   no need to use ConcurrentHashMap here, I think we can use an `ImmutableMap`
   
   also no need to copy it again in WriteThresholdCounter.create(...)
   
   



##########
src/java/org/apache/cassandra/db/WriteThresholds.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.db;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DataStorageSpec;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.net.ParamType;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.NoSpamLogger;
+
+/**
+ * Utility class for checking write threshold warnings on replicas.
+ * CASSANDRA-17258: paxos and accord do complex thread hand off and custom 
write logic which makes this patch complex, so was deferred
+ */
+public class WriteThresholds
+{
+    private static final Logger logger = 
LoggerFactory.getLogger(WriteThresholds.class);
+    private static final NoSpamLogger noSpamLogger = 
NoSpamLogger.getLogger(logger, 1, TimeUnit.MINUTES);
+
+    /**
+     * Check write thresholds for all partition updates in a mutation.
+     * This method iterates through all partition updates in the mutation.

Review Comment:
   it doesn't anymore - it iterates the tables



##########
src/java/org/apache/cassandra/service/writes/thresholds/WriteThresholdMapSerializer.java:
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.service.writes.thresholds;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableId;
+
+public class WriteThresholdMapSerializer implements 
IVersionedSerializer<Map<TableId, Long>>
+{
+    public static final WriteThresholdMapSerializer serializer = new 
WriteThresholdMapSerializer();
+
+    @Override
+    public void serialize(Map<TableId, Long> t, DataOutputPlus out, int 
version) throws IOException
+    {
+        out.writeUnsignedVInt32(t.size());
+        for (Map.Entry<TableId, Long> entry : t.entrySet())
+        {
+            entry.getKey().serialize(out);
+            out.writeLong(entry.getValue());
+        }
+    }
+
+    @Override
+    public Map<TableId, Long> deserialize(DataInputPlus in, int version) 
throws IOException
+    {
+        int size = in.readUnsignedVInt32();
+        Map<TableId, Long> result = new HashMap<>(size);

Review Comment:
   nit; the size param here is not the expected size, it is the initial 
capacity - use `Maps.newHashMapWithExpectedSize(size);` instead



##########
test/distributed/org/apache/cassandra/distributed/test/thresholds/WriteSizeWarningTest.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.distributed.test.thresholds;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.util.List;
+
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DataStorageSpec;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static 
org.apache.cassandra.config.DataStorageSpec.DataStorageUnit.MEBIBYTES;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Distributed tests for write size threshold warnings.
+ * Tests that writes to large partitions (tracked in TopPartitionTracker) 
trigger warnings.
+ */
+public class WriteSizeWarningTest extends AbstractWriteThresholdWarning
+{
+    private static final long WARN_THRESHOLD_BYTES = 5 * 1024 * 1024; // 5MB
+    private static final Logger log = 
LoggerFactory.getLogger(WriteSizeWarningTest.class);
+
+    @BeforeClass
+    public static void setupClass() throws IOException
+    {
+        AbstractWriteThresholdWarning.setupClass();
+
+        // Setup write size threshold after cluster init
+        CLUSTER.stream().forEach(i -> i.runOnInstance(() -> {
+            DatabaseDescriptor.setWriteSizeWarnThreshold(new 
DataStorageSpec.LongBytesBound(5, MEBIBYTES));
+            // Set minimum tracked partition size to ensure partitions are 
tracked
+            // This should be lower than the test value (10MB) to allow 
tracking
+            DatabaseDescriptor.setMinTrackedPartitionSizeInBytes(new 
DataStorageSpec.LongBytesBound(1, MEBIBYTES));
+            DatabaseDescriptor.setWriteThresholdsEnabled(true);
+        }));
+    }
+
+    @Override
+    protected long getWarnThreshold()
+    {
+        return WARN_THRESHOLD_BYTES;
+    }
+
+    @Override
+    protected void populateTopPartitions(int pk, long sizeBytes)
+    {
+        CLUSTER.stream().forEach(node -> node.runOnInstance(() -> {
+            // Get the DecoratedKey for the partition
+            var key = 
Murmur3Partitioner.instance.decorateKey(ByteBufferUtil.bytes(pk));
+
+            // Get the ColumnFamilyStore
+            ColumnFamilyStore cfs = 
Keyspace.open(KEYSPACE).getColumnFamilyStore("tbl");
+
+            // If topPartitions is null, create it using reflection
+            if (cfs.topPartitions == null)
+            {
+                try

Review Comment:
   I don't think this can be null? And if it is we should configure top 
partitions to be enabled instead



##########
src/java/org/apache/cassandra/service/writes/thresholds/WriteWarningContext.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.service.writes.thresholds;
+
+
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.cassandra.net.ParamType;
+import org.apache.cassandra.schema.TableId;
+
+/**
+ * Accumulates write warning information from replica responses.
+ * Similar to WarningContext but for write operations (warnings only, no 
aborts).
+ */
+public class WriteWarningContext
+{
+    private static final EnumSet<ParamType> SUPPORTED = EnumSet.of(
+        ParamType.WRITE_SIZE_WARN,
+        ParamType.WRITE_TOMBSTONE_WARN
+    );
+
+    final WarnCounter writeSize = new WarnCounter();
+    final WarnCounter writeTombstone = new WarnCounter();
+
+    public static boolean isSupported(Set<ParamType> keys)
+    {
+        return !Collections.disjoint(keys, SUPPORTED);
+    }
+
+    /**
+     * Update counters from replica response parameters. Writes never abort, 
so this always returns without throwing.
+     */
+    public void updateCounters(Map<ParamType, Object> params)
+    {
+        for (Map.Entry<ParamType, Object> entry : params.entrySet())

Review Comment:
   Instead of iterating the entries, we can just:
   
   ```
       public void updateCounters(Map<ParamType, Object> params)
       {
           Object val = params.get(ParamType.WRITE_SIZE_WARN);
           if (val != null)
               writeSize.addWarning((Map<TableId, Long>) val);
           val = params.get(ParamType.WRITE_TOMBSTONE_WARN);
           if (val != null)
               writeTombstone.addWarning((Map<TableId, Long>) val);
       }
   ```
   
   I know the read warnings iterates them, but we only have 2 params



##########
src/java/org/apache/cassandra/service/writes/thresholds/WriteThresholdMapSerializer.java:
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.service.writes.thresholds;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableId;
+
+public class WriteThresholdMapSerializer implements 
IVersionedSerializer<Map<TableId, Long>>
+{
+    public static final WriteThresholdMapSerializer serializer = new 
WriteThresholdMapSerializer();
+
+    @Override
+    public void serialize(Map<TableId, Long> t, DataOutputPlus out, int 
version) throws IOException
+    {
+        out.writeUnsignedVInt32(t.size());
+        for (Map.Entry<TableId, Long> entry : t.entrySet())
+        {
+            entry.getKey().serialize(out);
+            out.writeLong(entry.getValue());

Review Comment:
   use `out.writeUnsignedVInt(..)` to save some space



##########
src/java/org/apache/cassandra/config/DatabaseDescriptor.java:
##########
@@ -5482,6 +5470,40 @@ public static void 
setRowIndexReadSizeFailThreshold(@Nullable DataStorageSpec.Lo
         conf.row_index_read_size_fail_threshold = value;
     }
 
+    public static boolean getWriteThresholdsEnabled()
+    {
+        return conf.write_thresholds_enabled;
+    }
+
+    public static void setWriteThresholdsEnabled(boolean enabled)
+    {
+        logger.info("updating write_thresholds_enabled to {}", enabled);
+        conf.write_thresholds_enabled = enabled;
+    }
+
+    @Nullable
+    public static DataStorageSpec.LongBytesBound getWriteSizeWarnThreshold()
+    {
+        return conf.write_size_warn_threshold;
+    }
+
+    public static void setWriteSizeWarnThreshold(@Nullable 
DataStorageSpec.LongBytesBound value)
+    {
+        logger.info("updating write_size_warn_threshold to {}", value);
+        conf.write_size_warn_threshold = value;

Review Comment:
   nit; we should make sure `value > min_tracked_partition_size`



##########
src/java/org/apache/cassandra/service/writes/thresholds/WriteThresholdMapSerializer.java:
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.service.writes.thresholds;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableId;
+
+public class WriteThresholdMapSerializer implements 
IVersionedSerializer<Map<TableId, Long>>
+{
+    public static final WriteThresholdMapSerializer serializer = new 
WriteThresholdMapSerializer();
+
+    @Override
+    public void serialize(Map<TableId, Long> t, DataOutputPlus out, int 
version) throws IOException
+    {
+        out.writeUnsignedVInt32(t.size());
+        for (Map.Entry<TableId, Long> entry : t.entrySet())
+        {
+            entry.getKey().serialize(out);
+            out.writeLong(entry.getValue());
+        }
+    }
+
+    @Override
+    public Map<TableId, Long> deserialize(DataInputPlus in, int version) 
throws IOException
+    {
+        int size = in.readUnsignedVInt32();
+        Map<TableId, Long> result = new HashMap<>(size);
+        for (int i = 0; i < size; i++)
+            result.put(TableId.deserialize(in), in.readLong());
+        return result;
+    }
+
+    @Override
+    public long serializedSize(Map<TableId, Long> t, int version)
+    {
+        return TypeSizes.sizeofUnsignedVInt(t.size()) + (long) t.size() * 24;

Review Comment:
   this should iterate the map and use serializedSize on the tableids
   
   ```
           long size = TypeSizes.sizeofUnsignedVInt(t.size());
           for (Map.Entry<TableId, Long> entry : t.entrySet())
           {
               size += entry.getKey().serializedSize();
               size += TypeSizes.sizeofUnsignedVInt(entry.getValue());
           }
           return size;
   ```



##########
src/java/org/apache/cassandra/metrics/TableMetrics.java:
##########
@@ -907,6 +911,10 @@ public Long getValue()
         tooManySSTableIndexesReadWarnings = 
createTableMeter("TooManySSTableIndexesReadWarnings", 
cfs.keyspace.metric.tooManySSTableIndexesReadWarnings);
         tooManySSTableIndexesReadAborts = 
createTableMeter("TooManySSTableIndexesReadAborts", 
cfs.keyspace.metric.tooManySSTableIndexesReadAborts);
 
+        writeSizeWarnings = createTableMeter("WriteSizeWarnings", 
cfs.keyspace.metric.writeSizeWarnings);
+        writeSize = createTableHistogram("WriteSize", 
cfs.keyspace.metric.writeSize, false);

Review Comment:
   this is not really WriteSize - it is the size of the large partitions 
getting written to, and we can already see that in `nodetool tablestats` - can 
probably remove this metric



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


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

Reply via email to