Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

IGNITE-24277 Add necessary log.isDebugEnabled() checks in ignite-page-memory module #5090

Merged
merged 3 commits into from
Jan 22, 2025
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,7 @@ public abstract class PagesList extends DataStructure {

/** Logger. */
protected final IgniteLogger log;
private final boolean debugLogEnabled;
tkalkirill marked this conversation as resolved.
Show resolved Hide resolved

/** Basket sizes. */
protected final AtomicLongArray bucketsSize;
Expand Down Expand Up @@ -163,7 +164,7 @@ public Boolean run(

int newBucket = getBucketIndex(freeSpace);

if (newBucket != oldBucket) {
if (newBucket != oldBucket && debugLogEnabled) {
log.debug("Bucket changed when moving from heap to PageMemory [list={}, oldBucket={}, newBucket={}, pageId={}]",
name(), oldBucket, newBucket, pageId);
}
Expand Down Expand Up @@ -201,7 +202,7 @@ protected PagesList(
super(pageListNamePrefix, grpId, null, partId, pageMem, lockLsnr, FLAG_AUX);

this.log = log;

this.debugLogEnabled = log.isDebugEnabled();
this.buckets = buckets;
this.metaPageId = metaPageId;

Expand Down Expand Up @@ -376,12 +377,16 @@ private void flushBucketsCache(IoStatisticsHolder statHolder) throws IgniteInter
LongArrayList pages = pagesCache.flush();

if (pages != null) {
log.debug("Move pages from heap to PageMemory [list={}, bucket={}, pages={}]", name(), bucket, pages);
if (debugLogEnabled) {
log.debug("Move pages from heap to PageMemory [list={}, bucket={}, pages={}]", name(), bucket, pages);
}

for (int i = 0; i < pages.size(); i++) {
long pageId = pages.getLong(i);

log.debug("Move page from heap to PageMemory [list={}, bucket={}, pageId={}]", name(), bucket, pageId);
if (debugLogEnabled) {
log.debug("Move page from heap to PageMemory [list={}, bucket={}, pageId={}]", name(), bucket, pageId);
}

Boolean res = write(pageId, putBucket, bucket, null, statHolder);

Expand All @@ -399,7 +404,9 @@ private void flushBucketsCache(IoStatisticsHolder statHolder) throws IgniteInter
}

if (lockedPages != 0) {
log.info("Several pages were locked and weren't flushed on disk [grp={}, lockedPages={}]", grpName, lockedPages);
if (log.isInfoEnabled()) {
tkalkirill marked this conversation as resolved.
Show resolved Hide resolved
log.info("Several pages were locked and weren't flushed on disk [grp={}, lockedPages={}]", grpName, lockedPages);
}

pageCacheChanged = true;
}
Expand Down Expand Up @@ -628,7 +635,7 @@ private boolean updateTail(int bucket, long oldTailId, long newTailId) {
for (; ; ) {
Stripe[] tails = getBucket(bucket);

if (log.isDebugEnabled()) {
if (debugLogEnabled) {
log.debug("Update tail [list={}, bucket={}, oldTailId={}, newTailId={}, tails={}]",
name(), bucket, oldTailId, newTailId, Arrays.toString(tails));
}
Expand Down Expand Up @@ -805,7 +812,9 @@ protected final void put(

if (bag == null && onheapListCachingEnabled && putDataPage(getBucketCache(bucket, true), dataId, dataAddr, bucket)) {
// Successfully put page to the onheap pages list cache.
log.debug("Put page to pages list cache [list={}, bucket={}, dataId={}]", name(), bucket, dataId);
if (debugLogEnabled) {
log.debug("Put page to pages list cache [list={}, bucket={}, dataId={}]", name(), bucket, dataId);
}

return;
}
Expand Down Expand Up @@ -870,7 +879,9 @@ assert getType(tailAddr) == T_PAGE_LIST_NODE
putDataPage(tailId, tailAddr, io, dataId, dataAddr, bucket, statHolder);

if (ok) {
log.debug("Put page to pages list [list={}, bucket={}, dataId={}, tailId={}]", name(), bucket, dataId, tailId);
if (debugLogEnabled) {
log.debug("Put page to pages list [list={}, bucket={}, dataId={}, tailId={}]", name(), bucket, dataId, tailId);
}

stripe.empty = false;

Expand Down Expand Up @@ -1198,7 +1209,9 @@ protected long takeEmptyPage(
if (pagesCache != null && (pageId = pagesCache.poll()) != 0L) {
decrementBucketSize(bucket);

log.debug("Take page from pages list cache [list={}, bucket={}, pageId={}]", name(), bucket, pageId);
if (debugLogEnabled) {
log.debug("Take page from pages list cache [list={}, bucket={}, pageId={}]", name(), bucket, pageId);
}

assert !isReuseBucket(bucket) : "reuse bucket detected";

Expand Down Expand Up @@ -1338,7 +1351,12 @@ assert getType(tailAddr) == T_PAGE_LIST_NODE
reuseList.addForRecycle(new SingletonReuseBag(recycleId));
}

log.debug("Take page from pages list [list={}, bucket={}, dataPageId={}, tailId={}]", name(), bucket, dataPageId, tailId);
if (debugLogEnabled) {
log.debug(
"Take page from pages list [list={}, bucket={}, dataPageId={}, tailId={}]",
name(), bucket, dataPageId, tailId
);
}

return dataPageId;
} finally {
Expand Down Expand Up @@ -1459,20 +1477,26 @@ protected final boolean removeDataPage(

// Pages cache can be null here if page was taken for put from free list concurrently.
if (pagesCache == null || !pagesCache.removePage(dataId)) {
log.debug("Remove page from pages list cache failed [list={}, bucket={}, dataId={}, reason={}]",
name(), bucket, dataId, ((pagesCache == null) ? "cache is null" : "page not found"));
if (debugLogEnabled) {
log.debug("Remove page from pages list cache failed [list={}, bucket={}, dataId={}, reason={}]",
name(), bucket, dataId, ((pagesCache == null) ? "cache is null" : "page not found"));
}

return false;
}

decrementBucketSize(bucket);

log.debug("Remove page from pages list cache [list={}, bucket={}, dataId={}]", name(), bucket, dataId);
if (debugLogEnabled) {
log.debug("Remove page from pages list cache [list={}, bucket={}, dataId={}]", name(), bucket, dataId);
}

return true;
}

log.debug("Remove page from pages list [list={}, bucket={}, dataId={}, pageId={}]", name(), bucket, dataId, pageId);
if (debugLogEnabled) {
log.debug("Remove page from pages list [list={}, bucket={}, dataId={}, pageId={}]", name(), bucket, dataId, pageId);
}

final long page = acquirePage(pageId, statHolder);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -185,7 +185,9 @@ public void waitUnlock(FullPageId id) {
boolean interrupted = false;

while (locked.contains(id)) {
log.debug("Found replaced page which is being written to page store, wait for finish replacement [id={}]", id);
if (log.isDebugEnabled()) {
tkalkirill marked this conversation as resolved.
Show resolved Hide resolved
log.debug("Found replaced page which is being written to page store, wait for finish replacement [id={}]", id);
}

try {
// Uninterruptable wait.
Expand Down