Skip to content

Commit facaac7

Browse files
inespotines potier
and
ines potier
committed
Add additional logging to segment creation, recycling and replay (#660)
Co-authored-by: ines potier <[email protected]>
1 parent 9e951b7 commit facaac7

File tree

2 files changed

+17
-7
lines changed

2 files changed

+17
-7
lines changed

src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java

Lines changed: 9 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@
3030
import java.util.concurrent.Future;
3131
import java.util.concurrent.atomic.AtomicInteger;
3232

33+
import com.palantir.logsafe.SafeArg;
3334
import com.google.common.base.Predicate;
3435
import com.google.common.collect.HashMultimap;
3536
import com.google.common.collect.Iterables;
@@ -136,7 +137,9 @@ public static CommitLogReplayer construct(CommitLog commitLog)
136137
}
137138
if (globalPosition == null)
138139
globalPosition = ReplayPosition.firstNotCovered(cfPersisted.values());
139-
logger.debug("Global replay position is {} from columnfamilies {}", globalPosition, FBUtilities.toString(cfPersisted));
140+
logger.debug("Global replay position {} is from columnfamilies {}",
141+
SafeArg.of("globalPosition", globalPosition),
142+
SafeArg.of("columnFamilies", FBUtilities.toString(cfPersisted)));
140143
return new CommitLogReplayer(commitLog, globalPosition, cfPersisted, replayFilter);
141144
}
142145

@@ -399,8 +402,10 @@ public void recover(File file, boolean tolerateTruncation) throws IOException
399402
{
400403
int replayPos = replayEnd + CommitLogSegment.SYNC_MARKER_SIZE;
401404

402-
if (logger.isTraceEnabled())
403-
logger.trace("Replaying {} between {} and {}", file, reader.getFilePointer(), end);
405+
logger.debug("Replaying {} between {} and {}",
406+
SafeArg.of("file", file.getName()),
407+
SafeArg.of("start", reader.getFilePointer()),
408+
SafeArg.of("end", end));
404409
if (compressor != null)
405410
{
406411
int uncompressedLength = reader.readInt();
@@ -473,7 +478,7 @@ public boolean logAndCheckIfShouldSkip(File file, CommitLogDescriptor desc)
473478

474479
if (globalPosition.segment > desc.id)
475480
{
476-
logger.trace("skipping replay of fully-flushed {}", file);
481+
logger.debug("Skipping replay of fully-flushed {}", SafeArg.of("file", file.getName()));
477482
return true;
478483
}
479484
return false;

src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java

Lines changed: 8 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@
3434
import java.util.concurrent.TimeUnit;
3535
import java.util.concurrent.atomic.AtomicLong;
3636

37+
import com.palantir.logsafe.SafeArg;
3738
import com.google.common.annotations.VisibleForTesting;
3839
import com.google.common.collect.Iterables;
3940
import com.google.common.util.concurrent.*;
@@ -118,9 +119,11 @@ public void runMayThrow() throws Exception
118119
// if we have no more work to do, check if we should create a new segment
119120
if (availableSegments.isEmpty() && (activeSegments.isEmpty() || createReserveSegments))
120121
{
121-
logger.trace("No segments in reserve; creating a fresh one");
122122
// TODO : some error handling in case we fail to create a new segment
123-
availableSegments.add(CommitLogSegment.createSegment(commitLog));
123+
CommitLogSegment newSegment = CommitLogSegment.createSegment(commitLog);
124+
availableSegments.add(newSegment);
125+
logger.info("No segments in reserve; created a fresh one",
126+
SafeArg.of("segment", newSegment.id));
124127
hasAvailableSegments.signalAll();
125128
}
126129

@@ -374,7 +377,9 @@ void recycleSegment(final File file)
374377
*/
375378
private void discardSegment(final CommitLogSegment segment, final boolean deleteFile)
376379
{
377-
logger.trace("Segment {} is no longer active and will be deleted {}", segment, deleteFile ? "now" : "by the archive script");
380+
logger.debug("Segment {} is no longer active and will be deleted {}",
381+
SafeArg.of("segment", segment.id),
382+
SafeArg.of("deleteNow", deleteFile));
378383

379384
segmentManagementTasks.add(new Runnable()
380385
{

0 commit comments

Comments
 (0)