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

Paulo Motta commented on CASSANDRA-7276:
----------------------------------------

Thanks for the fixes. Looking better, just a few more nits:
* Missing space here: {{logger.trace("\[{}.{}\]Scheduling flush in {} ms", 
this.keyspace.getName(), this.name, period);}}
* Rename {{IContexualizedAwareVerbHandler}} to {{IContexualizedVerbHandler}}
* You can use the same format {{\[ks.cf\]}} on {{RuntimeException("Keyspace:" + 
cfs.keyspace.getName() + " Table:" + cfs.name + " Cannot start multiple repair 
sessions over the same sstables")}}
* On {{maybeGetKeyspaceAndTableInfo}} if there are multiple ks/cfs you'll print 
"\[ks.cf1,cf2,cf3\]", what can be more troublesome to parse, I think it's 
better to have "\[ks.cf1,ks.cf2,ks.cf3\]"
* Fix git apply warnings:
{noformat}
Applying: Better Logging for KS and CF
/home/paulo/Workspace/cassandra/cassandra2/.git/rebase-apply/patch:520: 
trailing whitespace.
 * This exception is used by MessageDeliveryTask to 
/home/paulo/Workspace/cassandra/cassandra2/.git/rebase-apply/patch:532: 
trailing whitespace.
    public String getKeyspace() 
/home/paulo/Workspace/cassandra/cassandra2/.git/rebase-apply/patch:537: 
trailing whitespace.
    public Optional<List<String>> getTables() 
warning: 3 lines add whitespace errors.
{noformat}
* bike-shedding: replace {noformat}
+        if (cfnames.isEmpty())
+        {
+            return Optional.empty();
+        }
+        else
+        {
+            return Optional.of(cfnames);
+        }
{noformat}
with {{return cfnames.isEmpty()? Optional.empty() : Optional.of(cfnames)}} 
3-nary if #ftw :P
** You can use the same trick to retrieve ks and cf names on 
{{RepairMessageVerbHandler}}
* Since {{RepairMessageVerbHandler}} can return null keyspace, I think it's 
safer to modify the {{IContexualizedVerbHandler}} interface to return an 
{{Optional<String>>}} instead.
* Please also rebase to latest trunk to avoid conflicts.

> 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
>            Priority: Minor
>              Labels: bootcamp, lhf
>             Fix For: 2.1.x
>
>         Attachments: 0001-Better-Logging-for-KS-and-CF.patch, 
> 0001-Logging-for-Keyspace-and-Tables.patch, 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