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 org.apache.druid.timeline.DataSegment;
|
||||||
|
|
||||||
import java.util.Set;
|
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
|
* 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()
|
public String toString()
|
||||||
{
|
{
|
||||||
return "SegmentInsertAction{" +
|
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()
|
public String toString()
|
||||||
{
|
{
|
||||||
return "SegmentMetadataUpdateAction{" +
|
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()
|
public String toString()
|
||||||
{
|
{
|
||||||
return "SegmentNukeAction{" +
|
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()
|
public String toString()
|
||||||
{
|
{
|
||||||
return "SegmentInsertAction{" +
|
return "SegmentInsertAction{" +
|
||||||
"segments=" + segments +
|
"segments=" + segments.stream().map(DataSegment::getIdentifier).collect(Collectors.joining(",")) +
|
||||||
", startMetadata=" + startMetadata +
|
", startMetadata=" + startMetadata +
|
||||||
", endMetadata=" + endMetadata +
|
", endMetadata=" + endMetadata +
|
||||||
'}';
|
'}';
|
||||||
|
|
|
@ -397,6 +397,8 @@ public interface IndexMerger
|
||||||
{
|
{
|
||||||
pQueue = new PriorityQueue<>(dimValueLookups.length, NULLS_FIRST_PEEKING_COMPARATOR);
|
pQueue = new PriorityQueue<>(dimValueLookups.length, NULLS_FIRST_PEEKING_COMPARATOR);
|
||||||
conversions = new IntBuffer[dimValueLookups.length];
|
conversions = new IntBuffer[dimValueLookups.length];
|
||||||
|
|
||||||
|
long mergeBufferTotalSize = 0;
|
||||||
for (int i = 0; i < conversions.length; i++) {
|
for (int i = 0; i < conversions.length; i++) {
|
||||||
if (dimValueLookups[i] == null) {
|
if (dimValueLookups[i] == null) {
|
||||||
continue;
|
continue;
|
||||||
|
@ -404,12 +406,14 @@ public interface IndexMerger
|
||||||
Indexed<String> indexed = dimValueLookups[i];
|
Indexed<String> indexed = dimValueLookups[i];
|
||||||
if (useDirect) {
|
if (useDirect) {
|
||||||
int allocationSize = indexed.size() * Integer.BYTES;
|
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);
|
final ByteBuffer conversionDirectBuffer = ByteBuffer.allocateDirect(allocationSize);
|
||||||
conversions[i] = conversionDirectBuffer.asIntBuffer();
|
conversions[i] = conversionDirectBuffer.asIntBuffer();
|
||||||
directBufferAllocations.add(new Pair<>(conversionDirectBuffer, allocationSize));
|
directBufferAllocations.add(new Pair<>(conversionDirectBuffer, allocationSize));
|
||||||
} else {
|
} else {
|
||||||
conversions[i] = IntBuffer.allocate(indexed.size());
|
conversions[i] = IntBuffer.allocate(indexed.size());
|
||||||
|
mergeBufferTotalSize += indexed.size();
|
||||||
}
|
}
|
||||||
|
|
||||||
final PeekingIterator<String> iter = Iterators.peekingIterator(
|
final PeekingIterator<String> iter = Iterators.peekingIterator(
|
||||||
|
@ -422,6 +426,7 @@ public interface IndexMerger
|
||||||
pQueue.add(Pair.of(i, iter));
|
pQueue.add(Pair.of(i, iter));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
log.info("Allocated [%,d] bytes of dictionary merging direct buffers", mergeBufferTotalSize);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -482,10 +487,13 @@ public interface IndexMerger
|
||||||
@Override
|
@Override
|
||||||
public void close()
|
public void close()
|
||||||
{
|
{
|
||||||
|
long mergeBufferTotalSize = 0;
|
||||||
for (Pair<ByteBuffer, Integer> bufferAllocation : directBufferAllocations) {
|
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);
|
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()) {
|
if (!segmentsToDrop.isEmpty()) {
|
||||||
currentlyProcessing = segmentsToDrop.firstEntry().getValue();
|
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()) {
|
} else if (!segmentsToLoad.isEmpty()) {
|
||||||
currentlyProcessing = segmentsToLoad.firstEntry().getValue();
|
currentlyProcessing = segmentsToLoad.firstEntry().getValue();
|
||||||
log.info("Server[%s] loading [%s]", basePath, currentlyProcessing.getSegmentIdentifier());
|
log.debug("Server[%s] loading [%s]", basePath, currentlyProcessing.getSegmentIdentifier());
|
||||||
} else {
|
} else {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
@ -386,7 +386,7 @@ public class CuratorLoadQueuePeon extends LoadQueuePeon
|
||||||
);
|
);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
log.info(
|
log.debug(
|
||||||
"Server[%s] done processing %s of segment [%s]",
|
"Server[%s] done processing %s of segment [%s]",
|
||||||
basePath,
|
basePath,
|
||||||
currentlyProcessing.getType() == LOAD ? "load" : "drop",
|
currentlyProcessing.getType() == LOAD ? "load" : "drop",
|
||||||
|
|
|
@ -334,7 +334,7 @@ public class DruidCoordinator
|
||||||
|
|
||||||
public void removeSegment(DataSegment segment)
|
public void removeSegment(DataSegment segment)
|
||||||
{
|
{
|
||||||
log.info("Removing Segment[%s]", segment);
|
log.info("Removing Segment[%s]", segment.getIdentifier());
|
||||||
metadataSegmentManager.removeSegment(segment.getDataSource(), segment.getIdentifier());
|
metadataSegmentManager.removeSegment(segment.getDataSource(), segment.getIdentifier());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -223,7 +223,7 @@ public class HttpLoadQueuePeon extends LoadQueuePeon
|
||||||
handleResponseStatus(e.getRequest(), e.getStatus());
|
handleResponseStatus(e.getRequest(), e.getStatus());
|
||||||
break;
|
break;
|
||||||
case PENDING:
|
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;
|
break;
|
||||||
default:
|
default:
|
||||||
scheduleNextRunImmediately = false;
|
scheduleNextRunImmediately = false;
|
||||||
|
@ -521,7 +521,7 @@ public class HttpLoadQueuePeon extends LoadQueuePeon
|
||||||
|
|
||||||
public void requestSucceeded()
|
public void requestSucceeded()
|
||||||
{
|
{
|
||||||
log.info(
|
log.debug(
|
||||||
"Server[%s] Successfully processed segment[%s] request[%s].",
|
"Server[%s] Successfully processed segment[%s] request[%s].",
|
||||||
serverId,
|
serverId,
|
||||||
segment.getIdentifier(),
|
segment.getIdentifier(),
|
||||||
|
|
|
@ -155,11 +155,11 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
|
||||||
moveSegment(segmentToMoveHolder, destinationHolder.getServer(), params);
|
moveSegment(segmentToMoveHolder, destinationHolder.getServer(), params);
|
||||||
moved++;
|
moved++;
|
||||||
} else {
|
} else {
|
||||||
log.info("Segment [%s] is 'optimally' placed.", segmentToMove.getIdentifier());
|
log.debug("Segment [%s] is 'optimally' placed.", segmentToMove.getIdentifier());
|
||||||
unmoved++;
|
unmoved++;
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
log.info(
|
log.debug(
|
||||||
"No valid movement destinations for segment [%s].",
|
"No valid movement destinations for segment [%s].",
|
||||||
segmentToMove.getIdentifier()
|
segmentToMove.getIdentifier()
|
||||||
);
|
);
|
||||||
|
@ -207,7 +207,7 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
|
||||||
if (!toPeon.getSegmentsToLoad().contains(segmentToMove) &&
|
if (!toPeon.getSegmentsToLoad().contains(segmentToMove) &&
|
||||||
(toServer.getSegment(segmentName) == null) &&
|
(toServer.getSegment(segmentName) == null) &&
|
||||||
new ServerHolder(toServer, toPeon).getAvailableSize() > segmentToMove.getSize()) {
|
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;
|
LoadPeonCallback callback = null;
|
||||||
try {
|
try {
|
||||||
|
|
|
@ -160,7 +160,11 @@ public class DruidCoordinatorSegmentCompactor implements DruidCoordinatorHelper
|
||||||
config.getTuningConfig(),
|
config.getTuningConfig(),
|
||||||
config.getTaskContext()
|
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) {
|
} else if (segmentsToCompact.size() == 1) {
|
||||||
throw new ISE("Found one segments[%s] to compact", segmentsToCompact);
|
throw new ISE("Found one segments[%s] to compact", segmentsToCompact);
|
||||||
} else {
|
} 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);
|
rules.set(newRules);
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue