This is an automated email from the ASF dual-hosted git repository.
yihua pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git
The following commit(s) were added to refs/heads/master by this push:
new 3c44050a2db3 chore: Add SamplingLogger utility for reducing log volume
while maintaining observability (#14354)
3c44050a2db3 is described below
commit 3c44050a2db35df02d3fdaa171e39428e31bb9d2
Author: Shubham Patel <[email protected]>
AuthorDate: Fri Dec 12 23:13:39 2025 -0800
chore: Add SamplingLogger utility for reducing log volume while maintaining
observability (#14354)
---
.../table/view/AbstractTableFileSystemView.java | 23 ++++-
.../apache/hudi/common/util/SamplingLogger.java | 90 +++++++++++++++++
.../hudi/common/util/TestSamplingLogger.java | 109 +++++++++++++++++++++
3 files changed, 219 insertions(+), 3 deletions(-)
diff --git
a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java
b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java
index 6cc18f0f6b46..36611771fe16 100644
---
a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java
+++
b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java
@@ -40,6 +40,7 @@ import org.apache.hudi.common.util.ClusteringUtils;
import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.HoodieTimer;
import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.SamplingLogger;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
@@ -107,6 +108,9 @@ public abstract class AbstractTableFileSystemView
implements SyncableFileSystemV
// Used to concurrently load and populate partition views
private final ConcurrentHashMap<String, Boolean> addedPartitions = new
ConcurrentHashMap<>(4096);
+ // Sampling logger for replaced file groups read logs (log at INFO once
every 5 times)
+ private final SamplingLogger replacedFileGroupsReadSamplingLogger = new
SamplingLogger(LOG, 5);
+
// Locks to control concurrency. Sync operations use write-lock blocking all
fetch operations.
// For the common-case, we allow concurrent read of single or multiple
partitions
private final ReentrantReadWriteLock globalLock = new
ReentrantReadWriteLock();
@@ -294,7 +298,10 @@ public abstract class AbstractTableFileSystemView
implements SyncableFileSystemV
Map<HoodieFileGroupId, HoodieInstant> replacedFileGroups =
resultStream.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue,
(instance1, instance2) -> compareTimestamps(instance1.requestedTime(),
LESSER_THAN, instance2.requestedTime()) ? instance2 : instance1));
resetReplacedFileGroups(replacedFileGroups);
- LOG.info("Took {} ms to read {} instants, {} replaced file groups",
hoodieTimer.endTimer(), replacedTimeline.countInstants(),
replacedFileGroups.size());
+ // Sample log: log at INFO once every 5 times to track latencies,
otherwise DEBUG
+ replacedFileGroupsReadSamplingLogger.logInfoOrDebug(
+ "Took {} ms to read {} instants, {} replaced file groups",
+ () -> new Object[]{hoodieTimer.endTimer(),
replacedTimeline.countInstants(), replacedFileGroups.size()});
}
@Override
@@ -390,7 +397,12 @@ public abstract class AbstractTableFileSystemView
implements SyncableFileSystemV
long beginTs = System.currentTimeMillis();
// Not loaded yet
try {
- LOG.debug("Building file system view for partitions: {}",
partitionSet);
+ // For metadata table, log at DEBUG. For data table, log at INFO.
+ if (metaClient.isMetadataTable()) {
+ LOG.debug("Building file system view for {} partition(s)",
partitionSet.size());
+ } else {
+ LOG.info("Building file system view for {} partition(s)",
partitionSet.size());
+ }
// Pairs of relative partition path and absolute partition path
List<Pair<String, StoragePath>> absolutePartitionPathList =
partitionSet.stream()
@@ -453,7 +465,12 @@ public abstract class AbstractTableFileSystemView
implements SyncableFileSystemV
if (!isPartitionAvailableInStore(partitionPathStr)) {
// Not loaded yet
try {
- LOG.info("Building file system view for partition ({})",
partitionPathStr);
+ // For metadata table, log at DEBUG. For data table, log at INFO.
+ if (metaClient.isMetadataTable()) {
+ LOG.debug("Building file system view for partition ({})",
partitionPathStr);
+ } else {
+ LOG.info("Building file system view for partition ({})",
partitionPathStr);
+ }
List<HoodieFileGroup> groups = addFilesToView(partitionPathStr,
getAllFilesInPartition(partitionPathStr));
if (groups.isEmpty()) {
storePartitionView(partitionPathStr, new ArrayList<>());
diff --git
a/hudi-common/src/main/java/org/apache/hudi/common/util/SamplingLogger.java
b/hudi-common/src/main/java/org/apache/hudi/common/util/SamplingLogger.java
new file mode 100644
index 000000000000..77e96b86063e
--- /dev/null
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/SamplingLogger.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.util;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+/**
+ * A sampling logger that logs at INFO level once every N times, otherwise at
DEBUG level.
+ * This is useful for reducing log volume while still maintaining periodic
visibility
+ * into frequently executed code paths.
+ */
+public class SamplingLogger implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ private final String loggerName;
+ private transient Logger logger;
+ private final int sampleFrequency;
+ private final AtomicInteger counter;
+
+ /**
+ * Creates a new SamplingLogger.
+ *
+ * @param logger the underlying SLF4J logger to use
+ * @param sampleFrequency log at INFO level once every N calls (e.g., 5
means every 5th call)
+ */
+ public SamplingLogger(Logger logger, int sampleFrequency) {
+ this.loggerName = logger.getName();
+ this.logger = logger;
+ this.sampleFrequency = sampleFrequency;
+ this.counter = new AtomicInteger(0);
+ }
+
+ /**
+ * Gets the logger, recreating it if necessary after deserialization.
+ */
+ private Logger getLogger() {
+ if (logger == null) {
+ logger = LoggerFactory.getLogger(loggerName);
+ }
+ return logger;
+ }
+
+ /**
+ * Returns true if the next log call should be at INFO level based on
sampling frequency.
+ * This method increments the internal counter.
+ *
+ * @return true if the caller should log at INFO level, false for DEBUG level
+ */
+ @VisibleForTesting
+ boolean shouldLogAtInfo() {
+ return counter.incrementAndGet() % sampleFrequency == 0;
+ }
+
+ /**
+ * Logs a message at INFO level if it's time to sample, otherwise at DEBUG
level.
+ * Uses a Supplier to lazily evaluate arguments, avoiding computation when
logging is disabled.
+ *
+ * @param message the log message (can contain {} placeholders)
+ * @param argsSupplier a supplier that provides the arguments to substitute
into the message
+ */
+ public void logInfoOrDebug(String message, Supplier<Object[]> argsSupplier) {
+ if (shouldLogAtInfo()) {
+ getLogger().info(message, argsSupplier.get());
+ } else if (getLogger().isDebugEnabled()) {
+ getLogger().debug(message, argsSupplier.get());
+ }
+ }
+}
\ No newline at end of file
diff --git
a/hudi-common/src/test/java/org/apache/hudi/common/util/TestSamplingLogger.java
b/hudi-common/src/test/java/org/apache/hudi/common/util/TestSamplingLogger.java
new file mode 100644
index 000000000000..40adbbc4f59d
--- /dev/null
+++
b/hudi-common/src/test/java/org/apache/hudi/common/util/TestSamplingLogger.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hudi.common.util;
+
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class TestSamplingLogger {
+
+ @Test
+ public void testShouldLogAtInfoWithFrequency5() {
+ Logger mockLogger = mock(Logger.class);
+ when(mockLogger.getName()).thenReturn("TestLogger");
+ SamplingLogger samplingLogger = new SamplingLogger(mockLogger, 5);
+
+ // Calls 1-4 should return false (DEBUG)
+ assertFalse(samplingLogger.shouldLogAtInfo());
+ assertFalse(samplingLogger.shouldLogAtInfo());
+ assertFalse(samplingLogger.shouldLogAtInfo());
+ assertFalse(samplingLogger.shouldLogAtInfo());
+
+ // Call 5 should return true (INFO)
+ assertTrue(samplingLogger.shouldLogAtInfo());
+
+ // Calls 6-9 should return false (DEBUG)
+ assertFalse(samplingLogger.shouldLogAtInfo());
+ assertFalse(samplingLogger.shouldLogAtInfo());
+ assertFalse(samplingLogger.shouldLogAtInfo());
+ assertFalse(samplingLogger.shouldLogAtInfo());
+
+ // Call 10 should return true (INFO)
+ assertTrue(samplingLogger.shouldLogAtInfo());
+ }
+
+ @Test
+ public void testShouldLogAtInfoWithFrequency1() {
+ Logger mockLogger = mock(Logger.class);
+ when(mockLogger.getName()).thenReturn("TestLogger");
+ SamplingLogger samplingLogger = new SamplingLogger(mockLogger, 1);
+
+ // Every call should return true (always INFO)
+ assertTrue(samplingLogger.shouldLogAtInfo());
+ assertTrue(samplingLogger.shouldLogAtInfo());
+ assertTrue(samplingLogger.shouldLogAtInfo());
+ }
+
+ @Test
+ public void testLogInfoOrDebugWithSupplierOnlyEvaluatesWhenNeeded() {
+ Logger mockLogger = mock(Logger.class);
+ when(mockLogger.getName()).thenReturn("TestLogger");
+ when(mockLogger.isDebugEnabled()).thenReturn(false);
+
+ SamplingLogger samplingLogger = new SamplingLogger(mockLogger, 5);
+ AtomicInteger supplierCallCount = new AtomicInteger(0);
+
+ for (int i = 0; i < 10; i++) {
+ samplingLogger.logInfoOrDebug("Test message {}", () -> {
+ supplierCallCount.incrementAndGet();
+ return new Object[]{"arg"};
+ });
+ }
+
+ assertEquals(2, supplierCallCount.get());
+ }
+
+ @Test
+ public void testLogInfoOrDebugWithSupplierEvaluatesForDebugWhenEnabled() {
+ Logger mockLogger = mock(Logger.class);
+ when(mockLogger.getName()).thenReturn("TestLogger");
+ when(mockLogger.isDebugEnabled()).thenReturn(true);
+
+ SamplingLogger samplingLogger = new SamplingLogger(mockLogger, 5);
+ AtomicInteger supplierCallCount = new AtomicInteger(0);
+
+ for (int i = 0; i < 10; i++) {
+ samplingLogger.logInfoOrDebug("Test message {}", () -> {
+ supplierCallCount.incrementAndGet();
+ return new Object[]{"arg"};
+ });
+ }
+
+ assertEquals(10, supplierCallCount.get());
+ }
+}