exceptionfactory commented on a change in pull request #5101: URL: https://github.com/apache/nifi/pull/5101#discussion_r640037705
########## File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/controller/reporting/LogComponentStatuses.java ########## @@ -0,0 +1,205 @@ +/* + * 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.reporting; + +import org.apache.nifi.controller.Counter; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.flow.FlowManager; +import org.apache.nifi.controller.repository.CounterRepository; +import org.apache.nifi.controller.repository.FlowFileEvent; +import org.apache.nifi.controller.repository.FlowFileEventRepository; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.util.FormatUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class LogComponentStatuses implements Runnable { + private static final Logger logger = LoggerFactory.getLogger(LogComponentStatuses.class); + + private static final String PROCESSOR_LINE_FORMAT = "| %1$-30.30s | %2$-36.36s | %3$-30.30s | %4$28.28s | %5$30.30s | %6$14.14s | %714.14s | %8$28.28s |\n"; + private static final String COUNTER_LINE_FORMAT = "| %1$-36.36s | %2$-36.36s | %3$28.28s | %4$28.28s |\n"; + + private final FlowFileEventRepository flowFileEventRepository; + private final CounterRepository counterRepository; + private final FlowManager flowManager; + + private final String processorHeader; + private final String processorBorderLine; + private final String counterHeader; + private final String counterBorderLine; + + private final Map<String, Long> previousCounterValues = new ConcurrentHashMap<>(); + private volatile long lastTriggerTime = System.currentTimeMillis(); + + public LogComponentStatuses(final FlowFileEventRepository flowFileEventRepository, final CounterRepository counterRepository, final FlowManager flowManager) { + this.flowFileEventRepository = flowFileEventRepository; + this.counterRepository = counterRepository; + this.flowManager = flowManager; + + processorHeader = String.format(PROCESSOR_LINE_FORMAT, "Processor Name", "Processor ID", "Processor Type", "Bytes Read/sec", "Bytes Written/sec", "Tasks/sec", "Nanos/Task", + "Percent of Processing Time"); + processorBorderLine = createLine(processorHeader); + + counterHeader = String.format(COUNTER_LINE_FORMAT, "Counter Context", "Counter Name", "Counter Value", "Increase/sec"); + counterBorderLine = createLine(counterHeader); + } + + private String createLine(final String valueToUnderscore) { + final StringBuilder processorBorderBuilder = new StringBuilder(valueToUnderscore.length()); + for (int i = 0; i < valueToUnderscore.length(); i++) { + processorBorderBuilder.append('-'); + } + return processorBorderBuilder.toString(); + } + + @Override + public void run() { + try { + if (!logger.isInfoEnabled()) { + return; + } + + logFlowFileEvents(); + logCounters(); + } catch (final Exception e) { + logger.error("Failed to log component statuses", e); + } + } + + private void logFlowFileEvents() { + final long timestamp = System.currentTimeMillis(); + final ProcessGroup rootGroup = flowManager.getRootGroup(); + final List<ProcessorNode> allProcessors = rootGroup.findAllProcessors(); + + long totalNanos = 0L; + final List<ProcessorAndEvent> processorsAndEvents = new ArrayList<>(); + for (final ProcessorNode processorNode : allProcessors) { + final FlowFileEvent flowFileEvent = flowFileEventRepository.reportTransferEvents(processorNode.getIdentifier(), timestamp); + if (flowFileEvent == null) { + continue; + } + + processorsAndEvents.add(new ProcessorAndEvent(processorNode, flowFileEvent)); + totalNanos += flowFileEvent.getProcessingNanoseconds(); + } + + final Comparator<ProcessorAndEvent> comparator = Comparator.comparing(procAndEvent -> procAndEvent.getEvent().getProcessingNanoseconds()); + processorsAndEvents.sort(comparator.reversed()); + + final StringBuilder builder = new StringBuilder(); + builder.append("Processor Statuses:\n"); Review comment: Good point about existing usage, in that case, leaving it as-is sounds good. ########## File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/controller/reporting/LogComponentStatuses.java ########## @@ -0,0 +1,205 @@ +/* + * 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.reporting; + +import org.apache.nifi.controller.Counter; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.flow.FlowManager; +import org.apache.nifi.controller.repository.CounterRepository; +import org.apache.nifi.controller.repository.FlowFileEvent; +import org.apache.nifi.controller.repository.FlowFileEventRepository; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.util.FormatUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class LogComponentStatuses implements Runnable { + private static final Logger logger = LoggerFactory.getLogger(LogComponentStatuses.class); + + private static final String PROCESSOR_LINE_FORMAT = "| %1$-30.30s | %2$-36.36s | %3$-30.30s | %4$28.28s | %5$30.30s | %6$14.14s | %714.14s | %8$28.28s |\n"; + private static final String COUNTER_LINE_FORMAT = "| %1$-36.36s | %2$-36.36s | %3$28.28s | %4$28.28s |\n"; + + private final FlowFileEventRepository flowFileEventRepository; + private final CounterRepository counterRepository; + private final FlowManager flowManager; + + private final String processorHeader; + private final String processorBorderLine; + private final String counterHeader; + private final String counterBorderLine; + + private final Map<String, Long> previousCounterValues = new ConcurrentHashMap<>(); + private volatile long lastTriggerTime = System.currentTimeMillis(); + + public LogComponentStatuses(final FlowFileEventRepository flowFileEventRepository, final CounterRepository counterRepository, final FlowManager flowManager) { + this.flowFileEventRepository = flowFileEventRepository; + this.counterRepository = counterRepository; + this.flowManager = flowManager; + + processorHeader = String.format(PROCESSOR_LINE_FORMAT, "Processor Name", "Processor ID", "Processor Type", "Bytes Read/sec", "Bytes Written/sec", "Tasks/sec", "Nanos/Task", + "Percent of Processing Time"); + processorBorderLine = createLine(processorHeader); + + counterHeader = String.format(COUNTER_LINE_FORMAT, "Counter Context", "Counter Name", "Counter Value", "Increase/sec"); + counterBorderLine = createLine(counterHeader); + } + + private String createLine(final String valueToUnderscore) { + final StringBuilder processorBorderBuilder = new StringBuilder(valueToUnderscore.length()); + for (int i = 0; i < valueToUnderscore.length(); i++) { + processorBorderBuilder.append('-'); + } + return processorBorderBuilder.toString(); + } + + @Override + public void run() { + try { + if (!logger.isInfoEnabled()) { + return; + } + + logFlowFileEvents(); + logCounters(); Review comment: Thanks for the background, sounds good. ########## File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlow.java ########## @@ -212,23 +217,39 @@ public void initialize() { logger.info("Successfully initialized components in {} millis ({} millis to perform validation, {} millis for services to enable)", initializationMillis, validationMillis, serviceEnableMillis); - runDataflowExecutor = Executors.newFixedThreadPool(1, r -> { - final Thread thread = Executors.defaultThreadFactory().newThread(r); - final String flowName = dataflowDefinition.getFlowName(); - if (flowName == null) { - thread.setName("Run Dataflow"); - } else { - thread.setName("Run Dataflow " + flowName); - } + // Create executor for dataflow + final String flowName = dataflowDefinition.getFlowName(); + final String threadName = (flowName == null) ? "Run Dataflow" : "Run Dataflow " + flowName; + runDataflowExecutor = Executors.newFixedThreadPool(1, createNamedThreadFactory(threadName, false)); - return thread; - }); + // Periodically log component statuses + backgroundTaskExecutor = Executors.newScheduledThreadPool(1, createNamedThreadFactory("Background Tasks", true)); + backgroundTasks.forEach(task -> backgroundTaskExecutor.scheduleWithFixedDelay(task.getTask(), task.getSchedulingPeriod(), task.getSchedulingPeriod(), task.getSchedulingUnit())); } catch (final Throwable t) { processScheduler.shutdown(); throw t; } } + private ThreadFactory createNamedThreadFactory(final String name, final boolean daemon) { + return (Runnable r) -> { + final Thread thread = Executors.defaultThreadFactory().newThread(r); + thread.setName(name); + thread.setDaemon(daemon); + return thread; + }; + } + + /** + * Schedules the given background task to run periodically after the dataflow has been initialized until it has been shutdown + * @param task the task to run + * @param period how often to run it + * @param unit the unit for the time period + */ + public void scheduleBackgroundTask(final Runnable task, final long period, final TimeUnit unit) { Review comment: That makes sense, thanks. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected]
