[
https://issues.apache.org/jira/browse/CASSANDRA-7276?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14934399#comment-14934399
]
Paulo Motta commented on CASSANDRA-7276:
----------------------------------------
A more elegant approach would be to use logback
[MDC|http://logback.qos.ch/manual/mdc.html] feature, which allows to
transparently add thread-local contexts to log statements (similar to the
solution mentioned by [~odpeer]).
We could add new CF and KS MDC placeholders to the appender layout pattern on
logback.xml (they will be empty if not set), and set them when necessary. We
could start by setting on the following places:
* VerbHandlers which contains KS and CF info
* Flush
* Compaction
Some helper methods would be nice to provide encapsulated and consistent access
to MDC. Are you still willing to take this [~nitzanv]?
> 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
> Components: Core
> 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
>
>
> 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)