[ 
https://issues.apache.org/jira/browse/CASSANDRA-7276?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15027414#comment-15027414
 ] 

Ariel Weisberg commented on CASSANDRA-7276:
-------------------------------------------

I think that it's kind of a no win scenario from a complexity perspective. 
There are two problem scenarios here. One is log statements and the other is 
exceptions.

The problem with log statements is that it is error prone to populate the KS 
and CF name in all of them and some of the things that log might not have 
access to the KS and CF name. I suspect that most of them will have access to 
the KS and CF name.

The problem with exceptions is that we don't throw all of them so we can't 
ensure the message contains the KS and CF name and it has the same issue where 
the code throwing the exception might not have access to the KS and CF name. 

Exceptions are easier to deal with because any place doing the MDC shuffling 
right now can be converted to a try/catch/rethrow with a wrapper exception 
containing the KS and CF name or maybe modifying the message of the exception. 
This is also error prone and a maintenance headache, but there are at least a 
manageable number of paths to instrument. I don't see a free lunch. Maybe 
someone else does.

The log statements are harder to automate, but my guess is that most of them do 
know what KS and CF names they should have. If we did a pass to make sure log 
statements were good about using KS and CF name, and then kept an eye out it 
might be good enough.

> Include keyspace and table names in logs where possible
> -------------------------------------------------------
>
>                 Key: CASSANDRA-7276
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-7276
>             Project: Cassandra
>          Issue Type: Improvement
>            Reporter: Tyler Hobbs
>            Assignee: Nitzan Volman
>            Priority: Minor
>              Labels: bootcamp, lhf
>             Fix For: 2.1.x
>
>         Attachments: 2.1-CASSANDRA-7276-v1.txt, 
> cassandra-2.1-7276-compaction.txt, cassandra-2.1-7276.txt, 
> cassandra-2.1.9-7276-v2.txt, cassandra-2.1.9-7276.txt
>
>
> Most error messages and stacktraces give you no clue as to what keyspace or 
> table was causing the problem.  For example:
> {noformat}
> ERROR [MutationStage:61648] 2014-05-20 12:05:45,145 CassandraDaemon.java 
> (line 198) Exception in thread Thread[MutationStage:61648,5,main]
> java.lang.IllegalArgumentException
>     at java.nio.Buffer.limit(Unknown Source)
>     at 
> org.apache.cassandra.db.marshal.AbstractCompositeType.getBytes(AbstractCompositeType.java:63)
>     at 
> org.apache.cassandra.db.marshal.AbstractCompositeType.getWithShortLength(AbstractCompositeType.java:72)
>     at 
> org.apache.cassandra.db.marshal.AbstractCompositeType.compare(AbstractCompositeType.java:98)
>     at 
> org.apache.cassandra.db.marshal.AbstractCompositeType.compare(AbstractCompositeType.java:35)
>     at 
> edu.stanford.ppl.concurrent.SnapTreeMap$1.compareTo(SnapTreeMap.java:538)
>     at 
> edu.stanford.ppl.concurrent.SnapTreeMap.attemptUpdate(SnapTreeMap.java:1108)
>     at 
> edu.stanford.ppl.concurrent.SnapTreeMap.updateUnderRoot(SnapTreeMap.java:1059)
>     at edu.stanford.ppl.concurrent.SnapTreeMap.update(SnapTreeMap.java:1023)
>     at 
> edu.stanford.ppl.concurrent.SnapTreeMap.putIfAbsent(SnapTreeMap.java:985)
>     at 
> org.apache.cassandra.db.AtomicSortedColumns$Holder.addColumn(AtomicSortedColumns.java:328)
>     at 
> org.apache.cassandra.db.AtomicSortedColumns.addAllWithSizeDelta(AtomicSortedColumns.java:200)
>     at org.apache.cassandra.db.Memtable.resolve(Memtable.java:226)
>     at org.apache.cassandra.db.Memtable.put(Memtable.java:173)
>     at 
> org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:893)
>     at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:368)
>     at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:333)
>     at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:206)
>     at 
> org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:56)
>     at 
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:60)
>     at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
>     at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>     at java.lang.Thread.run(Unknown Source)
> {noformat}
> We should try to include info on the keyspace and column family in the error 
> messages or logs whenever possible.  This includes reads, writes, 
> compactions, flushes, repairs, and probably more.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to