dcapwell commented on a change in pull request #1180:
URL: https://github.com/apache/cassandra/pull/1180#discussion_r706401521



##########
File path: 
src/java/org/apache/cassandra/service/reads/trackwarnings/CoordinatorTrackWarnings.java
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.reads.trackwarnings;
+
+import java.util.AbstractMap;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.netty.util.concurrent.FastThreadLocal;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.ReadCommand;
+import org.apache.cassandra.metrics.TableMetrics;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.reads.ReadCallback;
+
+public class CoordinatorTrackWarnings
+{
+    private static final Logger logger = 
LoggerFactory.getLogger(CoordinatorTrackWarnings.class);
+    private static final boolean ENABLE_DEFENSIVE_CHECKS = 
Boolean.getBoolean("cassandra.track_warnings.coordinator.devensive_checks_enabled");
+
+    // when .init() is called set the STATE to be INIT; this is to lazy 
allocate the map only when warnings are generated
+    private static final Map<ReadCommand, TrackWarningsSnapshot> INIT = 
Collections.emptyMap();
+    private static final FastThreadLocal<Map<ReadCommand, 
TrackWarningsSnapshot>> STATE = new FastThreadLocal<>();
+
+    private CoordinatorTrackWarnings() {}
+
+    public static void init()
+    {
+        logger.trace("CoordinatorTrackWarnings.init()");
+        if (STATE.get() != null)
+        {
+            if (ENABLE_DEFENSIVE_CHECKS)
+                throw new AssertionError("CoordinatorTrackWarnings.init called 
while state is not null: " + STATE.get());
+            return;
+        }
+        STATE.set(INIT);
+    }
+
+    public static void update(ReadCommand cmd, TrackWarningsSnapshot snapshot)
+    {
+        logger.trace("CoordinatorTrackWarnings.update({}, {})", 
cmd.metadata(), snapshot);
+        Map<ReadCommand, TrackWarningsSnapshot> map = mutable();
+        TrackWarningsSnapshot previous = map.get(cmd);
+        TrackWarningsSnapshot update = TrackWarningsSnapshot.merge(previous, 
snapshot);
+        if (update == null) // null happens when the merge had null input or 
EMPTY input... remove the command from the map
+            map.remove(cmd);
+        else
+            map.put(cmd, update);
+    }
+
+    public static void done()
+    {
+        Map<ReadCommand, TrackWarningsSnapshot> map = readonly();
+        logger.trace("CoordinatorTrackWarnings.done() with state {}", map);
+        map.forEach((command, merged) -> {
+            ColumnFamilyStore cfs = 
Schema.instance.getColumnFamilyStoreInstance(command.metadata().id);
+            // race condition when dropping tables, also happens in unit tests 
as Schema may be bypassed
+            if (cfs == null)
+                return;
+
+            String cql = command.toCQLString();
+            String loggableTokens = command.loggableTokens();
+            trackAborts(merged.tombstones, cql, loggableTokens, 
cfs.metric.clientTombstoneAborts, ReadCallback::tombstoneAbortMessage);
+            trackWarnings(merged.tombstones, cql, loggableTokens, 
cfs.metric.clientTombstoneWarnings, ReadCallback::tombstoneWarnMessage);
+
+            trackAborts(merged.localReadSize, cql, loggableTokens, 
cfs.metric.localReadSizeAborts, ReadCallback::localReadSizeAbortMessage);
+            trackWarnings(merged.localReadSize, cql, loggableTokens, 
cfs.metric.localReadSizeWarnings, ReadCallback::localReadSizeWarnMessage);
+
+            trackAborts(merged.rowIndexTooLarge, cql, loggableTokens, 
cfs.metric.rowIndexSizeAborts, ReadCallback::rowIndexSizeAbortMessage);
+            trackWarnings(merged.rowIndexTooLarge, cql, loggableTokens, 
cfs.metric.rowIndexSizeWarnings, ReadCallback::rowIndexSizeWarnMessage);
+        });
+
+        STATE.set(null);
+    }
+
+    private static Map<ReadCommand, TrackWarningsSnapshot> mutable()
+    {
+        Map<ReadCommand, TrackWarningsSnapshot> map = STATE.get();
+        if (map == null)
+        {
+            if (ENABLE_DEFENSIVE_CHECKS)
+                throw new AssertionError("CoordinatorTrackWarnings.mutable 
calling without calling .init() first");
+            // set map to an "ignore" map; dropping all mutations
+            // this does not update the state as it is not known if .clear() 
will be called; so safter to leave state null
+            map = IgnoreMap.get();

Review comment:
       the argument for Ignore is to not fail code paths which call outside of 
the thread.  So if you enable failures for this feature we block, else we 
silently ignore.  The issue with `assert` is it is NEVER safe to disable in 
C*... so you can never opt out of failing in this case.
   
   My thinking is really just that, if we have a bug in C* why fail in 
production?  The user negative to silently ignore is we do not update 
logs/metrics, but their queries still work; why fail the query as well?




-- 
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