improve tracing output, moving less-useful debug messages to trace and adding others as needed
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/36b40be6 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/36b40be6 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/36b40be6 Branch: refs/heads/trunk Commit: 36b40be64f56d1bffb31fb75b9a0f11b586686b4 Parents: 880334b Author: Jonathan Ellis <[email protected]> Authored: Tue Oct 23 17:03:10 2012 -0500 Committer: Jonathan Ellis <[email protected]> Committed: Tue Oct 23 20:58:05 2012 -0500 ---------------------------------------------------------------------- .../concurrent/DebuggableThreadPoolExecutor.java | 29 +---- .../org/apache/cassandra/db/ColumnFamilyStore.java | 17 ++-- .../org/apache/cassandra/db/ReadVerbHandler.java | 8 +- .../cassandra/db/RowMutationVerbHandler.java | 9 +- src/java/org/apache/cassandra/db/Table.java | 6 +- .../org/apache/cassandra/net/MessagingService.java | 6 +- .../apache/cassandra/net/ResponseVerbHandler.java | 6 +- .../cassandra/service/IndexScanVerbHandler.java | 3 +- .../cassandra/service/RangeSliceVerbHandler.java | 3 +- .../org/apache/cassandra/service/ReadCallback.java | 4 +- .../org/apache/cassandra/service/StorageProxy.java | 96 +++++++-------- src/java/org/apache/cassandra/tracing/Tracing.java | 1 - .../org/apache/cassandra/utils/SlabAllocator.java | 2 +- 13 files changed, 76 insertions(+), 114 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/36b40be6/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java b/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java index 0d5c428..9aff0bf 100644 --- a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java +++ b/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java @@ -167,10 +167,10 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor if (r instanceof TraceSessionWrapper) { - logger.debug("completed executing {}", r); + TraceSessionWrapper tsw = (TraceSessionWrapper) r; // we have to reset trace state as its presence is what denotes the current thread is tracing // and if left this thread might start tracing unrelated tasks - ((TraceSessionWrapper)r).reset(); + tsw.reset(); } logExceptionsAfterExecute(r, t); @@ -255,25 +255,20 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor * * @param <T> */ - private static class TraceSessionWrapper<T> extends FutureTask<T> implements Callable<T> + private static class TraceSessionWrapper<T> extends FutureTask<T> { private final TraceState state; - private Callable<T> callable; - - // Using initializer because the ctor's provided by the FutureTask<> are all we need - { - state = Tracing.instance().get(); - } public TraceSessionWrapper(Runnable runnable, T result) { super(runnable, result); + state = Tracing.instance().get(); } public TraceSessionWrapper(Callable<T> callable) { super(callable); - this.callable = callable; + state = Tracing.instance().get(); } private void setupContext() @@ -285,19 +280,5 @@ public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor { Tracing.instance().set(null); } - - public T call() throws Exception - { - return callable.call(); - } - - @Override - public String toString() - { - return "TraceSessionWrapper{" + - "state=" + state + - ", callable=" + callable + - '}'; - } } } http://git-wip-us.apache.org/repos/asf/cassandra/blob/36b40be6/src/java/org/apache/cassandra/db/ColumnFamilyStore.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index 361e362..5f326d5 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -1185,6 +1185,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean ColumnFamily getColumnFamily(QueryFilter filter, int gcBefore) { assert columnFamily.equals(filter.getColumnFamilyName()) : filter.getColumnFamilyName(); + logger.debug("Executing single-partition query"); ColumnFamily result = null; @@ -1210,14 +1211,14 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean UUID cfId = Schema.instance.getId(table.name, columnFamily); if (cfId == null) { - logger.debug("no id found for {}.{}", table.name, columnFamily); + logger.trace("no id found for {}.{}", table.name, columnFamily); return null; } ColumnFamily cached = getThroughCache(cfId, filter); if (cached == null) { - logger.debug("cached row is empty"); + logger.trace("cached row is empty"); return null; } @@ -1229,7 +1230,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean metric.readLatency.addNano(System.nanoTime() - start); } - logger.debug("Read {} columns", result == null ? 0 : result.getColumnCount()); + logger.debug("Read {} cells", result == null ? 0 : result.getColumnCount()); return result; } @@ -1406,8 +1407,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean if (!range.contains(key)) return computeNext(); - if (logger.isDebugEnabled()) - logger.debug("scanned " + key); + logger.trace("scanned {}", key); // TODO this is necessary because when we collate supercolumns together, we don't check // their subcolumns for relevance, so we need to do a second prune post facto here. @@ -1438,6 +1438,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean public List<Row> getRangeSlice(ByteBuffer superColumn, final AbstractBounds<RowPosition> range, int maxResults, IFilter columnFilter, List<IndexExpression> rowFilter, boolean maxIsColumns, boolean isPaging) { + logger.debug("Executing seq scan"); return filter(getSequentialIterator(superColumn, range, columnFilter), ExtendedFilter.create(this, columnFilter, rowFilter, maxResults, maxIsColumns, isPaging)); } @@ -1448,13 +1449,13 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean public List<Row> search(List<IndexExpression> clause, AbstractBounds<RowPosition> range, int maxResults, IFilter dataFilter, boolean maxIsColumns) { + logger.debug("Executing indexed scan"); return indexManager.search(clause, range, maxResults, dataFilter, maxIsColumns); } public List<Row> filter(AbstractScanIterator rowIterator, ExtendedFilter filter) { - if (logger.isDebugEnabled()) - logger.debug("Filtering {} for rows matching {}", rowIterator, filter); + logger.trace("Filtering {} for rows matching {}", rowIterator, filter); List<Row> rows = new ArrayList<Row>(); int columnsCount = 0; try @@ -1480,7 +1481,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean if (!filter.isSatisfiedBy(data, null)) continue; - logger.debug("{} satisfies all filter expressions", data); + logger.trace("{} satisfies all filter expressions", data); // cut the resultset back to what was requested, if necessary data = filter.prune(data); } http://git-wip-us.apache.org/repos/asf/cassandra/blob/36b40be6/src/java/org/apache/cassandra/db/ReadVerbHandler.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/ReadVerbHandler.java b/src/java/org/apache/cassandra/db/ReadVerbHandler.java index 680c5ad..7477993 100644 --- a/src/java/org/apache/cassandra/db/ReadVerbHandler.java +++ b/src/java/org/apache/cassandra/db/ReadVerbHandler.java @@ -49,9 +49,7 @@ public class ReadVerbHandler implements IVerbHandler<ReadCommand> MessageOut<ReadResponse> reply = new MessageOut<ReadResponse>(MessagingService.Verb.REQUEST_RESPONSE, getResponse(command, row), ReadResponse.serializer); - if (logger.isDebugEnabled()) - logger.debug(String.format("Read key %s; sending response to %s@%s", - ByteBufferUtil.bytesToHex(command.key), id, message.from)); + logger.debug("Sending response to {}", message.from); MessagingService.instance().sendReply(reply, id, message.from); } catch (IOException ex) @@ -64,8 +62,8 @@ public class ReadVerbHandler implements IVerbHandler<ReadCommand> { if (command.isDigestQuery()) { - if (logger.isDebugEnabled()) - logger.debug("digest is " + ByteBufferUtil.bytesToHex(ColumnFamily.digest(row.cf))); + if (logger.isTraceEnabled()) + logger.trace("digest is " + ByteBufferUtil.bytesToHex(ColumnFamily.digest(row.cf))); return new ReadResponse(ColumnFamily.digest(row.cf)); } else http://git-wip-us.apache.org/repos/asf/cassandra/blob/36b40be6/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java b/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java index fad83bc..a8cdfdc 100644 --- a/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java +++ b/src/java/org/apache/cassandra/db/RowMutationVerbHandler.java @@ -36,8 +36,7 @@ public class RowMutationVerbHandler implements IVerbHandler<RowMutation> try { RowMutation rm = message.payload; - if (logger.isDebugEnabled()) - logger.debug("Applying " + rm); + logger.debug("Applying mutation"); // Check if there were any forwarding headers in this message InetAddress replyTo = message.from; @@ -55,8 +54,7 @@ public class RowMutationVerbHandler implements IVerbHandler<RowMutation> rm.apply(); WriteResponse response = new WriteResponse(); - if (logger.isDebugEnabled()) - logger.debug(rm + " applied. Sending response to " + id + "@" + replyTo); + logger.debug("Sending response to {}", replyTo); MessagingService.instance().sendReply(response.createMessage(), id, replyTo); } catch (IOException e) @@ -81,8 +79,7 @@ public class RowMutationVerbHandler implements IVerbHandler<RowMutation> // Send a message to each of the addresses on our Forward List InetAddress address = CompactEndpointSerializationHelper.deserialize(dis); String id = dis.readUTF(); - if (logger.isDebugEnabled()) - logger.debug("Forwarding message to " + address + " with= ID: " + id); + logger.debug("Forwarding message to {}@{}", id, address); // Let the response go back to the coordinator MessagingService.instance().sendOneWay(message, id, address); } http://git-wip-us.apache.org/repos/asf/cassandra/blob/36b40be6/src/java/org/apache/cassandra/db/Table.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/Table.java b/src/java/org/apache/cassandra/db/Table.java index f489ac9..942b79d 100644 --- a/src/java/org/apache/cassandra/db/Table.java +++ b/src/java/org/apache/cassandra/db/Table.java @@ -358,14 +358,17 @@ public class Table public void apply(RowMutation mutation, boolean writeCommitLog, boolean updateIndexes) { if (!mutation.getTable().equals(Tracing.TRACE_KS)) - logger.debug("applying mutation"); + logger.debug("Acquiring switchLock"); // write the mutation to the commitlog and memtables switchLock.readLock().lock(); try { if (writeCommitLog) + { + logger.debug("Appending to commitlog"); CommitLog.instance.add(mutation); + } DecoratedKey key = StorageService.getPartitioner().decorateKey(mutation.key()); for (ColumnFamily cf : mutation.getColumnFamilies()) @@ -377,6 +380,7 @@ public class Table continue; } + logger.debug("Adding to memtable"); cfs.apply(key, cf, updateIndexes ? cfs.indexManager.updaterFor(key, true) : SecondaryIndexManager.nullUpdater); } } http://git-wip-us.apache.org/repos/asf/cassandra/blob/36b40be6/src/java/org/apache/cassandra/net/MessagingService.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java index 42ad356..46ac6dd 100644 --- a/src/java/org/apache/cassandra/net/MessagingService.java +++ b/src/java/org/apache/cassandra/net/MessagingService.java @@ -590,7 +590,7 @@ public final class MessagingService implements MessagingServiceMBean logger.trace(FBUtilities.getBroadcastAddress() + " sending " + message.verb + " to " + id + "@" + to); if (to.equals(FBUtilities.getBroadcastAddress())) - logger.debug("Message-to-self {} going over MessagingService", message); + logger.trace("Message-to-self {} going over MessagingService", message); // message sinks are a testing hook MessageOut processedMessage = SinkManager.processOutboundMessage(message, id, to); @@ -692,9 +692,7 @@ public final class MessagingService implements MessagingServiceMBean public void receive(MessageIn message, String id, long timestamp) { Tracing.instance().initializeFromMessage(message); - - if (logger.isTraceEnabled()) - logger.trace(FBUtilities.getBroadcastAddress() + " received " + message.verb + " from " + id + "@" + message.from); + logger.debug("Request received from {}", message.from); message = SinkManager.processInboundMessage(message, id); if (message == null) http://git-wip-us.apache.org/repos/asf/cassandra/blob/36b40be6/src/java/org/apache/cassandra/net/ResponseVerbHandler.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/net/ResponseVerbHandler.java b/src/java/org/apache/cassandra/net/ResponseVerbHandler.java index b62a452..5b51058 100644 --- a/src/java/org/apache/cassandra/net/ResponseVerbHandler.java +++ b/src/java/org/apache/cassandra/net/ResponseVerbHandler.java @@ -39,14 +39,12 @@ public class ResponseVerbHandler implements IVerbHandler if (cb instanceof IAsyncCallback) { - if (logger.isDebugEnabled()) - logger.debug("Processing response on a callback from " + id + "@" + message.from); + logger.debug("Processing response from {}", message.from); ((IAsyncCallback) cb).response(message); } else { - if (logger.isDebugEnabled()) - logger.debug("Processing response on an async result from " + id + "@" + message.from); + logger.debug("Processing result from {}", message.from); ((IAsyncResult) cb).result(message); } } http://git-wip-us.apache.org/repos/asf/cassandra/blob/36b40be6/src/java/org/apache/cassandra/service/IndexScanVerbHandler.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/IndexScanVerbHandler.java b/src/java/org/apache/cassandra/service/IndexScanVerbHandler.java index 536f74a..774e046 100644 --- a/src/java/org/apache/cassandra/service/IndexScanVerbHandler.java +++ b/src/java/org/apache/cassandra/service/IndexScanVerbHandler.java @@ -44,8 +44,7 @@ public class IndexScanVerbHandler implements IVerbHandler<IndexScanCommand> command.index_clause.count, ThriftValidation.asIFilter(command.predicate, cfs.getComparator())); RangeSliceReply reply = new RangeSliceReply(rows); - if (logger.isDebugEnabled()) - logger.debug("Sending " + reply+ " to " + id + "@" + message.from); + logger.debug("Sending response to {}", message.from); MessagingService.instance().sendReply(reply.createMessage(), id, message.from); } catch (Exception ex) http://git-wip-us.apache.org/repos/asf/cassandra/blob/36b40be6/src/java/org/apache/cassandra/service/RangeSliceVerbHandler.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/RangeSliceVerbHandler.java b/src/java/org/apache/cassandra/service/RangeSliceVerbHandler.java index ef057e6..fbbf88a 100644 --- a/src/java/org/apache/cassandra/service/RangeSliceVerbHandler.java +++ b/src/java/org/apache/cassandra/service/RangeSliceVerbHandler.java @@ -55,8 +55,7 @@ public class RangeSliceVerbHandler implements IVerbHandler<RangeSliceCommand> throw new RuntimeException("Cannot service reads while bootstrapping!"); } RangeSliceReply reply = new RangeSliceReply(executeLocally(message.payload)); - if (logger.isDebugEnabled()) - logger.debug("Sending " + reply+ " to " + id + "@" + message.from); + logger.debug("Sending response to {}", message.from); MessagingService.instance().sendReply(reply.createMessage(), id, message.from); } catch (Exception ex) http://git-wip-us.apache.org/repos/asf/cassandra/blob/36b40be6/src/java/org/apache/cassandra/service/ReadCallback.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/ReadCallback.java b/src/java/org/apache/cassandra/service/ReadCallback.java index f6cf0bd..25affe9 100644 --- a/src/java/org/apache/cassandra/service/ReadCallback.java +++ b/src/java/org/apache/cassandra/service/ReadCallback.java @@ -79,8 +79,8 @@ public class ReadCallback<TMessage, TResolved> implements IAsyncCallback<TMessag this.consistencyLevel = consistencyLevel; sortForConsistencyLevel(endpoints); this.endpoints = resolver instanceof RowRepairResolver ? endpoints : filterEndpoints(endpoints); - if (logger.isDebugEnabled()) - logger.debug(String.format("Blockfor is %s; setting up requests to %s", blockfor, StringUtils.join(this.endpoints, ","))); + if (logger.isTraceEnabled()) + logger.trace(String.format("Blockfor is %s; setting up requests to %s", blockfor, StringUtils.join(this.endpoints, ","))); } /** http://git-wip-us.apache.org/repos/asf/cassandra/blob/36b40be6/src/java/org/apache/cassandra/service/StorageProxy.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java index 69e1467..57deaad 100644 --- a/src/java/org/apache/cassandra/service/StorageProxy.java +++ b/src/java/org/apache/cassandra/service/StorageProxy.java @@ -132,8 +132,8 @@ public class StorageProxy implements StorageProxyMBean ConsistencyLevel consistency_level) throws IOException { - if (logger.isDebugEnabled()) - logger.debug("insert writing local & replicate " + mutation.toString(true)); + if (logger.isTraceEnabled()) + logger.trace("insert writing local & replicate " + mutation.toString(true)); Runnable runnable = counterWriteTask(mutation, targets, responseHandler, localDataCenter, consistency_level); runnable.run(); @@ -149,8 +149,8 @@ public class StorageProxy implements StorageProxyMBean ConsistencyLevel consistency_level) throws IOException { - if (logger.isDebugEnabled()) - logger.debug("insert writing local & replicate " + mutation.toString(true)); + if (logger.isTraceEnabled()) + logger.trace("insert writing local & replicate " + mutation.toString(true)); Runnable runnable = counterWriteTask(mutation, targets, responseHandler, localDataCenter, consistency_level); StageManager.getStage(Stage.MUTATION).execute(runnable); @@ -170,7 +170,7 @@ public class StorageProxy implements StorageProxyMBean public static void mutate(Collection<? extends IMutation> mutations, ConsistencyLevel consistency_level) throws UnavailableException, OverloadedException, WriteTimeoutException { - logger.debug("Mutations/ConsistencyLevel are {}/{}", mutations, consistency_level); + logger.trace("Mutations/ConsistencyLevel are {}/{}", mutations, consistency_level); final String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress()); long startTime = System.nanoTime(); @@ -248,9 +248,7 @@ public class StorageProxy implements StorageProxyMBean throws UnavailableException, OverloadedException, WriteTimeoutException { long startTime = System.nanoTime(); - - if (logger.isDebugEnabled()) - logger.debug("Mutations/ConsistencyLevel are {}/{}", mutations, consistency_level); + logger.trace("Mutations/ConsistencyLevel are {}/{}", mutations, consistency_level); List<WriteResponseHandlerWrapper> wrappers = new ArrayList<WriteResponseHandlerWrapper>(mutations.size()); String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress()); @@ -508,8 +506,8 @@ public class StorageProxy implements StorageProxyMBean else { // belongs on a different server - if (logger.isDebugEnabled()) - logger.debug("insert writing key " + ByteBufferUtil.bytesToHex(rm.key()) + " to " + destination); + if (logger.isTraceEnabled()) + logger.trace("insert writing key " + ByteBufferUtil.bytesToHex(rm.key()) + " to " + destination); String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(destination); Multimap<MessageOut, InetAddress> messages = dcMessages.get(dc); @@ -550,8 +548,7 @@ public class StorageProxy implements StorageProxyMBean { public void runMayThrow() throws IOException { - if (logger.isDebugEnabled()) - logger.debug("Adding hint for " + target); + logger.debug("Adding hint for {}", target); try { @@ -636,20 +633,19 @@ public class StorageProxy implements StorageProxyMBean CompactEndpointSerializationHelper.serialize(destination, dos); String id = MessagingService.instance().addCallback(handler, message, destination, message.getTimeout()); dos.writeUTF(id); - if (logger.isDebugEnabled()) - logger.debug("Adding FWD message to: " + destination + " with ID " + id); + logger.trace("Adding FWD message to {}@{}", id, destination); } message = message.withParameter(RowMutation.FORWARD_TO, bos.toByteArray()); // send the combined message + forward headers String id = MessagingService.instance().sendRR(message, target, handler); - if (logger.isDebugEnabled()) - logger.debug("Sending message to: " + target + " with ID " + id); + logger.trace("Sending message to {}@{}", id, target); } private static void insertLocal(final RowMutation rm, final AbstractWriteResponseHandler responseHandler) { - if (logger.isDebugEnabled()) - logger.debug("insert writing local " + rm.toString(true)); + if (logger.isTraceEnabled()) + logger.trace("insert writing local " + rm.toString(true)); + Runnable runnable = new DroppableRunnable(MessagingService.Verb.MUTATION) { public void runMayThrow() throws IOException @@ -697,8 +693,8 @@ public class StorageProxy implements StorageProxyMBean // Forward the actual update to the chosen leader replica AbstractWriteResponseHandler responseHandler = new WriteResponseHandler(endpoint, WriteType.COUNTER); - if (logger.isDebugEnabled()) - logger.debug("forwarding counter update of key " + ByteBufferUtil.bytesToHex(cm.key()) + " to " + endpoint); + if (logger.isTraceEnabled()) + logger.trace("forwarding counter update of key " + ByteBufferUtil.bytesToHex(cm.key()) + " to " + endpoint); MessagingService.instance().sendRR(cm.makeMutationMessage(), endpoint, responseHandler); return responseHandler; } @@ -868,7 +864,7 @@ public class StorageProxy implements StorageProxyMBean { ReadCommand command = commands.get(i); assert !command.isDigestQuery(); - logger.debug("Command/ConsistencyLevel is {}/{}", command, consistency_level); + logger.trace("Command/ConsistencyLevel is {}/{}", command, consistency_level); List<InetAddress> endpoints = StorageService.instance.getLiveNaturalEndpoints(command.table, command.key); @@ -884,12 +880,12 @@ public class StorageProxy implements StorageProxyMBean InetAddress dataPoint = handler.endpoints.get(0); if (dataPoint.equals(FBUtilities.getBroadcastAddress()) && OPTIMIZE_LOCAL_REQUESTS) { - logger.debug("reading data locally"); + logger.trace("reading data locally"); StageManager.getStage(Stage.READ).execute(new LocalReadRunnable(command, handler)); } else { - logger.debug("reading data from {}", dataPoint); + logger.trace("reading data from {}", dataPoint); MessagingService.instance().sendRR(command.createMessage(), dataPoint, handler); } @@ -902,14 +898,14 @@ public class StorageProxy implements StorageProxyMBean MessageOut message = null; for (InetAddress digestPoint : handler.endpoints.subList(1, handler.endpoints.size())) { - if (digestPoint.equals(FBUtilities.getBroadcastAddress())) + if (digestPoint.equals(FBUtilities.getBroadcastAddress()) && OPTIMIZE_LOCAL_REQUESTS) { - logger.debug("reading digest locally"); + logger.trace("reading digest locally"); StageManager.getStage(Stage.READ).execute(new LocalReadRunnable(digestCommand, handler)); } else { - logger.debug("reading digest from {}", digestPoint); + logger.trace("reading digest from {}", digestPoint); // (We lazy-construct the digest Message object since it may not be necessary if we // are doing a local digest read, or no digest reads at all.) if (message == null) @@ -935,9 +931,6 @@ public class StorageProxy implements StorageProxyMBean command.maybeTrim(row); rows.add(row); } - - if (logger.isDebugEnabled()) - logger.debug("Read: " + (System.currentTimeMillis() - startTime2) + " ms."); } catch (ReadTimeoutException ex) { @@ -947,8 +940,7 @@ public class StorageProxy implements StorageProxyMBean } catch (DigestMismatchException ex) { - if (logger.isDebugEnabled()) - logger.debug("Digest mismatch: {}", ex.toString()); + logger.debug("Digest mismatch: {}", ex.toString()); RowRepairResolver resolver = new RowRepairResolver(command.table, command.key); RepairCallback repairHandler = new RepairCallback(resolver, handler.endpoints); @@ -1005,7 +997,7 @@ public class StorageProxy implements StorageProxyMBean ReadCommand retryCommand = command.maybeGenerateRetryCommand(handler, row); if (retryCommand != null) { - logger.debug("issuing retry for read command"); + logger.debug("Issuing retry for read command"); if (commandsToRetry == Collections.EMPTY_LIST) commandsToRetry = new ArrayList<ReadCommand>(); commandsToRetry.add(retryCommand); @@ -1039,8 +1031,7 @@ public class StorageProxy implements StorageProxyMBean protected void runMayThrow() throws IOException { - if (logger.isDebugEnabled()) - logger.debug("LocalReadRunnable reading " + command); + logger.trace("LocalReadRunnable reading {}", command); Table table = Table.open(command.table); Row r = command.getRow(table); @@ -1065,8 +1056,7 @@ public class StorageProxy implements StorageProxyMBean protected void runMayThrow() throws ExecutionException, InterruptedException { - if (logger.isDebugEnabled()) - logger.debug("LocalReadRunnable reading " + command); + logger.trace("LocalReadRunnable reading {}", command); RangeSliceReply result = new RangeSliceReply(RangeSliceVerbHandler.executeLocally(command)); MessagingService.instance().addLatency(FBUtilities.getBroadcastAddress(), System.currentTimeMillis() - start); @@ -1086,8 +1076,8 @@ public class StorageProxy implements StorageProxyMBean public static List<Row> getRangeSlice(RangeSliceCommand command, ConsistencyLevel consistency_level) throws IOException, UnavailableException, ReadTimeoutException { - if (logger.isDebugEnabled()) - logger.debug("Command/ConsistencyLevel is {}/{}", command.toString(), consistency_level); + logger.debug("Determining replicas to query"); + logger.trace("Command/ConsistencyLevel is {}/{}", command.toString(), consistency_level); long startTime = System.nanoTime(); List<Row> rows; // now scan until we have enough results @@ -1116,9 +1106,11 @@ public class StorageProxy implements StorageProxyMBean ReadCallback<RangeSliceReply, Iterable<Row>> handler = getReadCallback(resolver, nodeCmd, consistency_level, liveEndpoints); handler.assureSufficientLiveNodes(); resolver.setSources(handler.endpoints); - if (handler.endpoints.size() == 1 && handler.endpoints.get(0).equals(FBUtilities.getBroadcastAddress())) + if (handler.endpoints.size() == 1 + && handler.endpoints.get(0).equals(FBUtilities.getBroadcastAddress()) + && OPTIMIZE_LOCAL_REQUESTS) { - logger.debug("reading data locally"); + logger.trace("reading data locally"); StageManager.getStage(Stage.READ).execute(new LocalRangeSliceRunnable(nodeCmd, handler)); } else @@ -1127,8 +1119,7 @@ public class StorageProxy implements StorageProxyMBean for (InetAddress endpoint : handler.endpoints) { MessagingService.instance().sendRR(message, endpoint, handler); - if (logger.isDebugEnabled()) - logger.debug("reading " + nodeCmd + " from " + endpoint); + logger.trace("reading {} from {}", nodeCmd, endpoint); } } @@ -1138,14 +1129,13 @@ public class StorageProxy implements StorageProxyMBean { rows.add(row); columnsCount += row.getLiveColumnCount(); - logger.debug("range slices read {}", row.key); + logger.trace("range slices read {}", row.key); } FBUtilities.waitOnFutures(resolver.repairResults, DatabaseDescriptor.getWriteRpcTimeout()); } catch (TimeoutException ex) { - if (logger.isDebugEnabled()) - logger.debug("Range slice timeout: {}", ex.toString()); + logger.debug("Range slice timeout: {}", ex.toString()); // We actually got all response at that point int blockFor = consistency_level.blockFor(command.keyspace); throw new ReadTimeoutException(consistency_level, blockFor, blockFor, true); @@ -1194,7 +1184,7 @@ public class StorageProxy implements StorageProxyMBean public void response(MessageIn<UUID> message) { // record the response from the remote node. - logger.debug("Received schema check response from {}", message.from.getHostAddress()); + logger.trace("Received schema check response from {}", message.from.getHostAddress()); versions.put(message.from, message.payload); latch.countDown(); } @@ -1219,7 +1209,7 @@ public class StorageProxy implements StorageProxyMBean throw new AssertionError("This latch shouldn't have been interrupted."); } - logger.debug("My version is {}", myVersion); + logger.trace("My version is {}", myVersion); // maps versions to hosts that are on that version. Map<String, List<String>> results = new HashMap<String, List<String>>(); @@ -1263,8 +1253,7 @@ public class StorageProxy implements StorageProxyMBean // special case for bounds containing exactly 1 (non-minimum) token if (queryRange instanceof Bounds && queryRange.left.equals(queryRange.right) && !queryRange.left.isMinimum(StorageService.getPartitioner())) { - if (logger.isDebugEnabled()) - logger.debug("restricted single token match for query {}", queryRange); + logger.trace("restricted single token match for query {}", queryRange); return Collections.singletonList(queryRange); } @@ -1300,7 +1289,7 @@ public class StorageProxy implements StorageProxyMBean } ranges.add(remainder); if (logger.isDebugEnabled()) - logger.debug("restricted ranges for query {} are {}", queryRange, ranges); + logger.trace("restricted ranges for query {} are {}", queryRange, ranges); return ranges; } @@ -1407,7 +1396,7 @@ public class StorageProxy implements StorageProxyMBean boolean hintWindowExpired = Gossiper.instance.getEndpointDowntime(ep) > DatabaseDescriptor.getMaxHintWindow(); if (hintWindowExpired) - logger.debug("not hinting {} which has been down {}ms", ep, Gossiper.instance.getEndpointDowntime(ep)); + logger.trace("not hinting {} which has been down {}ms", ep, Gossiper.instance.getEndpointDowntime(ep)); return !hintWindowExpired; } @@ -1438,16 +1427,15 @@ public class StorageProxy implements StorageProxyMBean final TruncateResponseHandler responseHandler = new TruncateResponseHandler(blockFor); // Send out the truncate calls and track the responses with the callbacks. - logger.debug("Starting to send truncate messages to hosts {}", allEndpoints); + logger.trace("Starting to send truncate messages to hosts {}", allEndpoints); final Truncation truncation = new Truncation(keyspace, cfname); MessageOut<Truncation> message = truncation.createMessage(); for (InetAddress endpoint : allEndpoints) MessagingService.instance().sendRR(message, endpoint, responseHandler); // Wait for all - logger.debug("Sent all truncate messages, now waiting for {} responses", blockFor); + logger.trace("Sent all truncate messages, now waiting for {} responses", blockFor); responseHandler.get(); - logger.debug("truncate done"); } /** http://git-wip-us.apache.org/repos/asf/cassandra/blob/36b40be6/src/java/org/apache/cassandra/tracing/Tracing.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/tracing/Tracing.java b/src/java/org/apache/cassandra/tracing/Tracing.java index 98a0cd4..5e6b3d0 100644 --- a/src/java/org/apache/cassandra/tracing/Tracing.java +++ b/src/java/org/apache/cassandra/tracing/Tracing.java @@ -250,5 +250,4 @@ public class Tracing checkState(sessionBytes.length == 16); state.set(new TraceState(message.from, UUIDGen.getUUID(ByteBuffer.wrap(sessionBytes)))); } - } http://git-wip-us.apache.org/repos/asf/cassandra/blob/36b40be6/src/java/org/apache/cassandra/utils/SlabAllocator.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/utils/SlabAllocator.java b/src/java/org/apache/cassandra/utils/SlabAllocator.java index 96b6e54..9d273e2 100644 --- a/src/java/org/apache/cassandra/utils/SlabAllocator.java +++ b/src/java/org/apache/cassandra/utils/SlabAllocator.java @@ -95,7 +95,7 @@ public class SlabAllocator extends Allocator // we won race - now we need to actually do the expensive allocation step region.init(); regionCount++; - logger.debug("{} regions now allocated in {}", regionCount, this); + logger.trace("{} regions now allocated in {}", regionCount, this); return region; } // someone else won race - that's fine, we'll try to grab theirs
