emit logs that are only useful for debugging at debug level (#6741)

* make logs that are only useful for debugging be at debug level so log volume is much more chill

* info level messages for total merge buffer allocated/free

* more chill compaction logs
This commit is contained in:
Clint Wylie 2018-12-16 22:20:28 -08:00 committed by Benedict Jin
parent c713116a75
commit 486c6f3cf9
11 changed files with 30 additions and 17 deletions

View File

@ -27,6 +27,7 @@ import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.timeline.DataSegment;
import java.util.Set;
import java.util.stream.Collectors;
/**
* Insert segments into metadata storage. The segment versions must all be less than or equal to a lock held by
@ -83,7 +84,7 @@ public class SegmentInsertAction implements TaskAction<Set<DataSegment>>
public String toString()
{
return "SegmentInsertAction{" +
"segments=" + segments +
"segments=" + segments.stream().map(DataSegment::getIdentifier).collect(Collectors.joining(",")) +
'}';
}
}

View File

@ -114,7 +114,7 @@ public class SegmentMetadataUpdateAction implements TaskAction<Void>
public String toString()
{
return "SegmentMetadataUpdateAction{" +
"segments=" + segments +
"segments=" + segments.stream().map(DataSegment::getIdentifier).collect(Collectors.joining(",")) +
'}';
}
}

View File

@ -116,7 +116,7 @@ public class SegmentNukeAction implements TaskAction<Void>
public String toString()
{
return "SegmentNukeAction{" +
"segments=" + segments +
"segments=" + segments.stream().map(DataSegment::getIdentifier).collect(Collectors.joining(",")) +
'}';
}
}

View File

@ -154,7 +154,7 @@ public class SegmentTransactionalInsertAction implements TaskAction<SegmentPubli
public String toString()
{
return "SegmentInsertAction{" +
"segments=" + segments +
"segments=" + segments.stream().map(DataSegment::getIdentifier).collect(Collectors.joining(",")) +
", startMetadata=" + startMetadata +
", endMetadata=" + endMetadata +
'}';

View File

@ -397,6 +397,8 @@ public interface IndexMerger
{
pQueue = new PriorityQueue<>(dimValueLookups.length, NULLS_FIRST_PEEKING_COMPARATOR);
conversions = new IntBuffer[dimValueLookups.length];
long mergeBufferTotalSize = 0;
for (int i = 0; i < conversions.length; i++) {
if (dimValueLookups[i] == null) {
continue;
@ -404,12 +406,14 @@ public interface IndexMerger
Indexed<String> indexed = dimValueLookups[i];
if (useDirect) {
int allocationSize = indexed.size() * Integer.BYTES;
log.info("Allocating dictionary merging direct buffer with size[%,d]", allocationSize);
log.debug("Allocating dictionary merging direct buffer with size[%,d]", allocationSize);
mergeBufferTotalSize += allocationSize;
final ByteBuffer conversionDirectBuffer = ByteBuffer.allocateDirect(allocationSize);
conversions[i] = conversionDirectBuffer.asIntBuffer();
directBufferAllocations.add(new Pair<>(conversionDirectBuffer, allocationSize));
} else {
conversions[i] = IntBuffer.allocate(indexed.size());
mergeBufferTotalSize += indexed.size();
}
final PeekingIterator<String> iter = Iterators.peekingIterator(
@ -422,6 +426,7 @@ public interface IndexMerger
pQueue.add(Pair.of(i, iter));
}
}
log.info("Allocated [%,d] bytes of dictionary merging direct buffers", mergeBufferTotalSize);
}
@Override
@ -482,10 +487,13 @@ public interface IndexMerger
@Override
public void close()
{
long mergeBufferTotalSize = 0;
for (Pair<ByteBuffer, Integer> bufferAllocation : directBufferAllocations) {
log.info("Freeing dictionary merging direct buffer with size[%,d]", bufferAllocation.rhs);
log.debug("Freeing dictionary merging direct buffer with size[%,d]", bufferAllocation.rhs);
mergeBufferTotalSize += bufferAllocation.rhs;
ByteBufferUtils.free(bufferAllocation.lhs);
}
log.info("Freed [,%d] bytes of dictionary merging direct buffers", mergeBufferTotalSize);
}
}
}

View File

@ -235,10 +235,10 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon
if (!segmentsToDrop.isEmpty()) {
currentlyProcessing = segmentsToDrop.firstEntry().getValue();
log.info("Server[%s] dropping [%s]", basePath, currentlyProcessing.getSegmentIdentifier());
log.debug("Server[%s] dropping [%s]", basePath, currentlyProcessing.getSegmentIdentifier());
} else if (!segmentsToLoad.isEmpty()) {
currentlyProcessing = segmentsToLoad.firstEntry().getValue();
log.info("Server[%s] loading [%s]", basePath, currentlyProcessing.getSegmentIdentifier());
log.debug("Server[%s] loading [%s]", basePath, currentlyProcessing.getSegmentIdentifier());
} else {
return;
}
@ -386,7 +386,7 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon
);
return;
}
log.info(
log.debug(
"Server[%s] done processing %s of segment [%s]",
basePath,
currentlyProcessing.getType() == LOAD ? "load" : "drop",

View File

@ -334,7 +334,7 @@ public class DruidCoordinator
public void removeSegment(DataSegment segment)
{
log.info("Removing Segment[%s]", segment);
log.info("Removing Segment[%s]", segment.getIdentifier());
metadataSegmentManager.removeSegment(segment.getDataSource(), segment.getIdentifier());
}

View File

@ -223,7 +223,7 @@ public class HttpLoadQueuePeon extends LoadQueuePeon
handleResponseStatus(e.getRequest(), e.getStatus());
break;
case PENDING:
log.info("Request[%s] is still pending on server[%s].", e.getRequest(), serverId);
log.debug("Request[%s] is still pending on server[%s].", e.getRequest(), serverId);
break;
default:
scheduleNextRunImmediately = false;
@ -521,7 +521,7 @@ public class HttpLoadQueuePeon extends LoadQueuePeon
public void requestSucceeded()
{
log.info(
log.debug(
"Server[%s] Successfully processed segment[%s] request[%s].",
serverId,
segment.getIdentifier(),

View File

@ -155,11 +155,11 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
moveSegment(segmentToMoveHolder, destinationHolder.getServer(), params);
moved++;
} else {
log.info("Segment [%s] is 'optimally' placed.", segmentToMove.getIdentifier());
log.debug("Segment [%s] is 'optimally' placed.", segmentToMove.getIdentifier());
unmoved++;
}
} else {
log.info(
log.debug(
"No valid movement destinations for segment [%s].",
segmentToMove.getIdentifier()
);
@ -207,7 +207,7 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
if (!toPeon.getSegmentsToLoad().contains(segmentToMove) &&
(toServer.getSegment(segmentName) == null) &&
new ServerHolder(toServer, toPeon).getAvailableSize() > segmentToMove.getSize()) {
log.info("Moving [%s] from [%s] to [%s]", segmentName, fromServer.getName(), toServer.getName());
log.debug("Moving [%s] from [%s] to [%s]", segmentName, fromServer.getName(), toServer.getName());
LoadPeonCallback callback = null;
try {

View File

@ -160,7 +160,11 @@ public class DruidCoordinatorSegmentCompactor implements DruidCoordinatorHelper
config.getTuningConfig(),
config.getTaskContext()
);
LOG.info("Submitted a compactTask[%s] for segments[%s]", taskId, segmentsToCompact);
LOG.info(
"Submitted a compactTask[%s] for segments[%s]",
taskId,
segmentsToCompact.stream().map(DataSegment::getIdentifier).collect(Collectors.joining(","))
);
} else if (segmentsToCompact.size() == 1) {
throw new ISE("Found one segments[%s] to compact", segmentsToCompact);
} else {

View File

@ -156,7 +156,7 @@ public class CoordinatorRuleManager
)
);
log.info("Got [%,d] rules", newRules.size());
log.debug("Got [%,d] rules", newRules.size());
rules.set(newRules);
}