mirror of https://github.com/apache/druid.git
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:
parent
c713116a75
commit
486c6f3cf9
|
@ -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(",")) +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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(",")) +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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(",")) +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 +
|
||||
'}';
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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(),
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -156,7 +156,7 @@ public class CoordinatorRuleManager
|
|||
)
|
||||
);
|
||||
|
||||
log.info("Got [%,d] rules", newRules.size());
|
||||
log.debug("Got [%,d] rules", newRules.size());
|
||||
|
||||
rules.set(newRules);
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue