[ https://issues.apache.org/jira/browse/CASSANDRA-7276?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15187713#comment-15187713 ]
Paulo Motta commented on CASSANDRA-7276: ---------------------------------------- Thanks and sorry for the delay. I'm a bit worried that the repetition of log statements with the \[ks.cf\] prefix is error prone and cumbersome so may not be followed by other developers, so I think we can improve this by providing a {{ContextualizedLogger}} that will wrap a {{Logger}} object and provide utillity methods to automatically add the \[ks.cf\] prefix. This will allow us to change the prefix format if necessary and easily add new contextualized log statements types in the future. I thought of something along those lines (these are just prototypes/suggestions): {noformat} public class ContextualizedLogger implements Logger { private final Logger logger; public static ContextualizedLogger getLogger(Class clazz) { return new ContextualizedLogger(LoggerFactory.getLogger(clazz)); } private ContextualizedLogger(Logger logger) { this.logger = logger; } public void debug(ColumnFamilyStore cfs, String msg, Object... args) { debug(cfs.name, cfs.keyspace.getName(), msg, args); } public void debug(CFMetaData cfm, String msg, Object... args) { debug(cfm.ksName, cfm.cfName, msg, args); } public void debug(String ks, String table, String msg, Object... args) { logger.debug(extendMsg(msg), extendArgs(ks, table, args)); } private String extendMsg(String msg) { return String.format("[{}.{}] %s", msg); } private Object[] extendArgs(String ks, String table, Object[] args) { Object[] newArgs = new Object[args.length+2]; newArgs[0] = ks; newArgs[1] = table; System.arraycopy(args, 0, newArgs, 2, args.length); return newArgs; } public String getName() { return logger.getName(); } public boolean isTraceEnabled() { return logger.isTraceEnabled(); } public void trace(String s) { logger.trace(s); } public boolean isTraceEnabled(Marker marker) { return logger.isTraceEnabled(marker); } } {noformat} So this, could be used like this, for instance: {noformat} public class ColumnFamilyStore implements ColumnFamilyStoreMBean { private static final ContextualizedLogger logger = ContextualizedLogger.getLogger(ColumnFamilyStore.class); private void logFlush() { logger.debug(this, "Enqueuing flush: {}", String.format("%d (%.0f%%) on-heap, %d (%.0f%%) off-heap", onHeapTotal, onHeapRatio * 100, offHeapTotal, offHeapRatio * 100)); } } {noformat} or {noformat} public class CompactionManager implements CompactionManagerMBean { private static final ContextualizedLogger logger = ContextualizedLogger.getLogger(CompactionManager.class); public Future<?> submitCacheWrite(final AutoSavingCache.Writer writer) { Runnable runnable = new Runnable() { public void run() { if (!AutoSavingCache.flushInProgress.add(writer.cacheType())) { CFMetaData cfm = writer.getCompactionInfo().getCFMetaData(); logger.debug(cfm, "Cache flushing was already in progress: skipping {}", writer.getCompactionInfo()); return; } } } } } {noformat} WDYT about this approach? If you agree please add support to this and use it on all log statements of {{ColumnFamilyStore}}, {{CompactionManager}}, {{Memtable}}, {{CompactionTask}} (and subclasses). You may need to update log statements that already mention ks/cf info to only show those only in the prefix, for example: * {{Loading new SSTables and building secondary indexes for ks/cf: sstable_XYZ}} would become {{\[ks.cf\] Loading new SSTables and building secondary indexes: sstable_XYZ}} We can also improve {{ContextualizedException}} to add the {{ks/cf}} info to the Exception message itself, so it will be automatically print when the stack trace is print by the uncaught exception handler. I modified the class to do this so you can reuse it on your next patch: {noformat} diff --git a/src/java/org/apache/cassandra/exceptions/ContextualizedException.java b/src/java/org/apache/cassandra/exceptions/ContextualizedException.java index 68c3e54..9cc5a93 100644 --- a/src/java/org/apache/cassandra/exceptions/ContextualizedException.java +++ b/src/java/org/apache/cassandra/exceptions/ContextualizedException.java @@ -19,6 +19,9 @@ package org.apache.cassandra.exceptions; import java.util.List; import java.util.Optional; +import java.util.stream.Collectors; + +import org.apache.commons.lang3.StringUtils; /** * This exception is used by MessageDeliveryTask to @@ -49,4 +52,28 @@ public class ContextualizedException extends RuntimeException { this.keyspace = keyspace; this.tables = tables; } + + public String getPrettyKeyspaceAndTables() + { + if (keyspace.isPresent() && !tables.isPresent()) + { + return String.format("Exception on keyspace %s: ", keyspace.get()); + } + + if (keyspace.isPresent() && tables.isPresent() && !tables.get().isEmpty()) + { + List<String> ksTables = tables.get().stream().map(t -> String.format("%s.%s", keyspace.get(), t)).collect(Collectors.toList()); + if (ksTables.size() == 1) + return String.format("Exception on table %s: ", ksTables.get(0)); + else + return String.format("Exception on tables %s: ", StringUtils.join(ksTables, ",")); + } + + return ""; + } + + public String getMessage() + { + return String.format("%s%s", getPrettyKeyspaceAndTables(), super.getMessage()); + } } {noformat} Another improvement we can make is to add a constructor to {{ContextualizedException}} that receives a keyspace, a table and a message, and replace the occurrences of {{RuntimeException}} on {{ColumnFamilyStore}} with a {{ContextualizedException}} that will automatically print the message in the correct format without the need to add a prefix to the exception message as we've been doing before. Some other nits from the previous patch: * Fix typo {{IContexualizedVerbHandler}} -> {{IContextualizedVerbHandler}} * Rename {{getColumnFamilyNames}} on {{CounterMutation}} and {{Mutation}} to {{getTableNames}}. * Use a ternary if on {{RepairMessageVerbHandler.getKeyspace}} and {{getTables}}. * On {{Mutation.toString()}} you're printing the {{getTableNames()}} {{Optional}} but you should check if it's empty first. > 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-KS-and-CF-consistently.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)