This is an automated email from the ASF dual-hosted git repository.

pvillard pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/main by this push:
     new dd2f0010b21 NIFI-15626 Added Record Gauge support to Batched Sessions
dd2f0010b21 is described below

commit dd2f0010b21701a661d35c3eadc13f79c5a36c04
Author: exceptionfactory <[email protected]>
AuthorDate: Thu Feb 19 08:33:20 2026 -0600

    NIFI-15626 Added Record Gauge support to Batched Sessions
    
    - Updated BatchingSessionFactory.HighThroughputSession to delegate record 
gauge calls to StandardProcessSession
    
    This closes #10920.
    
    Signed-off-by: Pierre Villard <[email protected]>
---
 .../repository/BatchingSessionFactory.java         |  6 ++
 .../repository/BatchingSessionFactoryTest.java     | 85 ++++++++++++++++++++++
 2 files changed, 91 insertions(+)

diff --git 
a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
 
b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
index fc4fe0a8d51..11ce752ea5b 100644
--- 
a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
+++ 
b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
@@ -27,6 +27,7 @@ import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.io.InputStreamCallback;
 import org.apache.nifi.processor.io.OutputStreamCallback;
 import org.apache.nifi.processor.io.StreamCallback;
+import org.apache.nifi.processor.metrics.CommitTiming;
 import org.apache.nifi.provenance.ProvenanceReporter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -115,6 +116,11 @@ public class BatchingSessionFactory implements 
ProcessSessionFactory {
             session.adjustCounter(name, delta, immediate);
         }
 
+        @Override
+        public void recordGauge(final String name, final double value, final 
CommitTiming commitTiming) {
+            session.recordGauge(name, value, commitTiming);
+        }
+
         @Override
         public FlowFile get() {
             return session.get();
diff --git 
a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/BatchingSessionFactoryTest.java
 
b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/BatchingSessionFactoryTest.java
new file mode 100644
index 00000000000..eced182de10
--- /dev/null
+++ 
b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/BatchingSessionFactoryTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.nifi.controller.repository;
+
+import org.apache.nifi.connectable.Connectable;
+import org.apache.nifi.controller.lifecycle.TaskTermination;
+import org.apache.nifi.controller.metrics.GaugeRecord;
+import org.apache.nifi.controller.repository.metrics.NopPerformanceTracker;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.metrics.CommitTiming;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@ExtendWith(MockitoExtension.class)
+class BatchingSessionFactoryTest {
+    private static final String CONNECTABLE_ID = "connectable-id";
+
+    private static final String GAUGE_NAME = "recording";
+
+    private static final double GAUGE_VALUE = 64.5;
+
+    private final TaskTermination taskTermination = () -> false;
+
+    @Mock
+    private RepositoryContext repositoryContext;
+
+    @Mock
+    private Connectable connectable;
+
+    @Captor
+    private ArgumentCaptor<GaugeRecord> gaugeRecordCaptor;
+
+    private BatchingSessionFactory factory;
+
+    @BeforeEach
+    void setFactory() {
+        when(repositoryContext.getConnectable()).thenReturn(connectable);
+        when(connectable.getIdentifier()).thenReturn(CONNECTABLE_ID);
+        final StandardProcessSession standardProcessSession = new 
StandardProcessSession(repositoryContext, taskTermination, new 
NopPerformanceTracker());
+        factory = new BatchingSessionFactory(standardProcessSession);
+    }
+
+    @Test
+    void testCreateSession() {
+        final ProcessSession session = factory.createSession();
+        assertNotNull(session);
+    }
+
+    @Test
+    void testCreateSessionRecordGauge() {
+        final ProcessSession session = factory.createSession();
+        assertNotNull(session);
+
+        session.recordGauge(GAUGE_NAME, GAUGE_VALUE, CommitTiming.NOW);
+
+        verify(repositoryContext).recordGauge(gaugeRecordCaptor.capture());
+        final GaugeRecord gaugeRecord = gaugeRecordCaptor.getValue();
+        assertEquals(GAUGE_NAME, gaugeRecord.name());
+        assertEquals(GAUGE_VALUE, gaugeRecord.value());
+    }
+}

Reply via email to