[
https://issues.apache.org/jira/browse/CASSANDRA-7276?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15032775#comment-15032775
]
Paulo Motta commented on CASSANDRA-7276:
----------------------------------------
Thanks for your input [~aweisberg], I agree MDC will not get us many benefits
here, since we will pretty much need to manage it in the same way as we would
without it with extra complexity and dependencies.
There was a lot of back-and-forth on this ticket looking for a more transparent
way of doing this, but it seems we will need to handle it explicitly on a
case-by-case basis. In order to wrap-up, I propose doing the following (open to
suggestions):
* Manually add ks/cf information to relevant existing log statements in
{{ColumnFamilyStore}} and {{CompactionManager}}, and anywhere else that
uses/access {{ColumnFamilyStore}}.
** Flush and compaction log messages are a good place to start.
* Create a new wrapped {{ContextualizedException}} (or better name) that
extends {{RuntimeException}} and saves keyspace and CF information. This
exception could have the following methods:
{noformat}
String getKeyspace()
Optional<List<String>> getTables()
{noformat}
** Modify the logger of the default uncaught exception handler on
{{CassandraDaemon.setup()}} to print ks/cf information when the uncatched
exception is of type {{ContextualizedException}}.
* In order to allow retrieving keyspace and table information transparently
when there is an error on {{MessageDeliveryTask}}, create
{{TableAwareVerbHandler}} which would implement the following methods:
{noformat}
public interface KeyspaceAwareVerbHandler<T>
{
public String getKeyspace(MessageIn<T> message);
public Optional<List<String>> getTables(MessageIn<T> message);
}
{noformat}
** Handlers such as {{ReadVerbHandler}}, {{MutationVerbHandler}},
{{TruncateVerbHandler}} and others would implement the
{{KeyspaceAwareVerbHandler}} to provide the interface methods (similar to what
was done on [Eiti's initial
patch|https://issues.apache.org/jira/secure/attachment/12668599/2.1-CASSANDRA-7276-v1.txt]
but in a more generic way).
** When there is an uncatch exception on {{MessageDeliveryTask}} and the verb
handler is a {{KeyspaceAwareVerbHandler}} propagate a
{{ContextualizedException}} with the correct ks/cf information so it will be
printed correctly by the uncaught exception handler.
[~nitzanv] [~eitikimura] I will leave this unassigned so feel free to assign to
you if you still want to work on this. Things should move faster now that the
scope is better defined.
> 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)